From 0ad911c413dd9c37f3db61e1af7e613df8af40fa Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 3 Oct 2022 11:24:33 +0000 Subject: [PATCH 001/554] 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 002/554] 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 003/554] 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 004/554] 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 005/554] 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 006/554] 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 007/554] 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 008/554] 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 009/554] 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 010/554] 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 011/554] 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 012/554] 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 4f27391a72d3c87b351c5dbe13bd3c1c8aaeba08 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 31 Mar 2023 13:37:41 +0300 Subject: [PATCH 013/554] 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 014/554] 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 015/554] 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 016/554] 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 017/554] 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 018/554] 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 019/554] 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 020/554] 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 021/554] 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 022/554] 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: Fri, 14 Apr 2023 13:10:09 +0200 Subject: [PATCH 023/554] 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 024/554] 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 025/554] 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 2b7a403752b523e5384cfe42d76907a3c097386a Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Thu, 27 Apr 2023 14:55:23 +0300 Subject: [PATCH 026/554] 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 027/554] 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 dd5e95e2e36fa4c781e7a12cef2264dee67c5c97 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 1 May 2023 08:23:33 +0300 Subject: [PATCH 028/554] 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 029/554] 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 030/554] 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 031/554] 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 032/554] 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 033/554] 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 034/554] 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 a8d56b2290cf0ada1a3598f295ed45575778afe4 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 8 May 2023 10:26:52 +0300 Subject: [PATCH 035/554] 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 036/554] 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 037/554] 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 038/554] 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 446cf3c847eb77608fd01a1020fb16bda8f4ccf5 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Wed, 10 May 2023 18:17:26 +0300 Subject: [PATCH 039/554] 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 abf1d459b6d76536855ba6ad62b23bd84fe06709 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Mon, 15 May 2023 09:38:59 +0300 Subject: [PATCH 040/554] 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 041/554] 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 042/554] 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 043/554] 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 e6d6d41f22102e3e6b3f60996e0baec34e3b0432 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Mon, 22 May 2023 12:23:31 +0800 Subject: [PATCH 044/554] 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 15b847b41058de86ee1e85ef7acb61902c08713b Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 24 May 2023 09:49:12 +0800 Subject: [PATCH 045/554] 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 046/554] 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 bc7c67e33f20a07602715788a1426492c5a26349 Mon Sep 17 00:00:00 2001 From: Sergey Katkovskiy Date: Thu, 25 May 2023 23:29:16 +0300 Subject: [PATCH 047/554] 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 048/554] 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 049/554] 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 050/554] 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 051/554] 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 052/554] 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 053/554] 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 054/554] 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 055/554] 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 056/554] 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 057/554] 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 058/554] 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 059/554] 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 6328811097204e87abc31973adcec84ddf8f0fc4 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 8 Jun 2023 09:26:30 +0200 Subject: [PATCH 060/554] 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 061/554] 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 062/554] 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 95d447d81fad93770cdddf1ba508466ac7a1d886 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 12 Jun 2023 10:10:09 +0200 Subject: [PATCH 063/554] 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 064/554] 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 2d0152e93d0bb64d077b510886cdb26924c9711a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 12 Jun 2023 13:38:01 +0200 Subject: [PATCH 065/554] 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 8650586131f8cedabb682d4c4191442d8cd30f3e Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 13 Jun 2023 10:51:35 +0200 Subject: [PATCH 066/554] 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 067/554] 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 1234fbe6f1adf64161e384eab7820c9ebfc1fe2f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 15 Jun 2023 08:41:37 +0200 Subject: [PATCH 068/554] 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 83840da5353a1c4b98ea1f688a0d250d5a1039ee Mon Sep 17 00:00:00 2001 From: LiuYangkuan Date: Sat, 17 Jun 2023 09:48:58 +0800 Subject: [PATCH 069/554] Allow using cluster, clusterAllReplicas, remote, and remoteSecure without table name. --- .../sql-reference/table-functions/cluster.md | 10 +- .../sql-reference/table-functions/remote.md | 10 +- src/TableFunctions/TableFunctionRemote.cpp | 92 +++++++++++-------- ...0171_shard_array_of_tuple_remote.reference | 2 + .../00171_shard_array_of_tuple_remote.sql | 1 + ...als_in_subquery_remote_and_limit.reference | 3 + ...th_totals_in_subquery_remote_and_limit.sql | 1 + .../01121_remote_scalar_subquery.reference | 2 + .../01121_remote_scalar_subquery.sql | 1 + ...4_cluster_all_replicas_shard_num.reference | 9 ++ .../01754_cluster_all_replicas_shard_num.sql | 6 ++ .../0_stateless/01787_map_remote.reference | 2 + .../queries/0_stateless/01787_map_remote.sql | 1 + .../queries/0_stateless/01880_remote_ipv6.sql | 11 +++ ...00_table_function_cluster_macros.reference | 2 + .../02000_table_function_cluster_macros.sql | 2 + ...ous_functions_in_order_by_remote.reference | 2 + ...onotonous_functions_in_order_by_remote.sql | 4 + .../02596_build_set_and_remote.reference | 16 ++++ .../02596_build_set_and_remote.sql | 6 ++ 20 files changed, 136 insertions(+), 47 deletions(-) diff --git a/docs/en/sql-reference/table-functions/cluster.md b/docs/en/sql-reference/table-functions/cluster.md index 904c678750c..cff8402a200 100644 --- a/docs/en/sql-reference/table-functions/cluster.md +++ b/docs/en/sql-reference/table-functions/cluster.md @@ -16,14 +16,14 @@ All available clusters are listed in the [system.clusters](../../operations/syst **Syntax** ``` sql -cluster('cluster_name', db.table[, sharding_key]) -cluster('cluster_name', db, table[, sharding_key]) -clusterAllReplicas('cluster_name', db.table[, sharding_key]) -clusterAllReplicas('cluster_name', db, table[, sharding_key]) +cluster(['cluster_name', db.table, sharding_key]) +cluster(['cluster_name', db, table, sharding_key]) +clusterAllReplicas(['cluster_name', db.table, sharding_key]) +clusterAllReplicas(['cluster_name', db, table, sharding_key]) ``` **Arguments** -- `cluster_name` – Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. +- `cluster_name` – Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers, set `default` if not specify. - `db.table` or `db`, `table` - Name of a database and a table. - `sharding_key` - A sharding key. Optional. Needs to be specified if the cluster has more than one shard. diff --git a/docs/en/sql-reference/table-functions/remote.md b/docs/en/sql-reference/table-functions/remote.md index bf0abd49fc6..ae0ce7bf6d2 100644 --- a/docs/en/sql-reference/table-functions/remote.md +++ b/docs/en/sql-reference/table-functions/remote.md @@ -13,10 +13,10 @@ Both functions can be used in `SELECT` and `INSERT` queries. ## Syntax ``` sql -remote('addresses_expr', db, table[, 'user'[, 'password'], sharding_key]) -remote('addresses_expr', db.table[, 'user'[, 'password'], sharding_key]) -remoteSecure('addresses_expr', db, table[, 'user'[, 'password'], sharding_key]) -remoteSecure('addresses_expr', db.table[, 'user'[, 'password'], sharding_key]) +remote('addresses_expr', [db, table, 'user'[, 'password'], sharding_key]) +remote('addresses_expr', [db.table, 'user'[, 'password'], sharding_key]) +remoteSecure('addresses_expr', [db, table, 'user'[, 'password'], sharding_key]) +remoteSecure('addresses_expr', [db.table, 'user'[, 'password'], sharding_key]) ``` ## Parameters @@ -29,6 +29,8 @@ remoteSecure('addresses_expr', db.table[, 'user'[, 'password'], sharding_key]) The port is required for an IPv6 address. + If only specify this parameter, `db` and `table` will use `system.one` by default. + Type: [String](../../sql-reference/data-types/string.md). - `db` — Database name. Type: [String](../../sql-reference/data-types/string.md). diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index b2f09adf773..3f3f6677e20 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -37,7 +37,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr String cluster_name; String cluster_description; - String database, table, username = "default", password; + String database = "system", table = "one", username = "default", password; if (args_func.size() != 1) throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -86,7 +86,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr else { /// Supported signatures: - /// + /// remote('addresses_expr') /// remote('addresses_expr', db.table) /// remote('addresses_expr', 'db', 'table') /// remote('addresses_expr', db.table, 'user') @@ -102,6 +102,8 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr /// /// remoteSecure() - same as remote() /// + /// cluster() + /// cluster('cluster_name') /// cluster('cluster_name', db.table) /// cluster('cluster_name', 'db', 'table') /// cluster('cluster_name', db.table, sharding_key) @@ -109,7 +111,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr /// /// clusterAllReplicas() - same as cluster() - if (args.size() < 2 || args.size() > max_args) + if ((!is_cluster_function && args.size() < 1) || args.size() > max_args) throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); size_t arg_num = 0; @@ -128,8 +130,15 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr if (is_cluster_function) { - args[arg_num] = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context); - cluster_name = checkAndGetLiteralArgument(args[arg_num], "cluster_name"); + if (args.size() > 0) + { + args[arg_num] = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context); + cluster_name = checkAndGetLiteralArgument(args[arg_num], "cluster_name"); + } + else + { + cluster_name = "default"; + } } else { @@ -141,44 +150,49 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr } ++arg_num; - const auto * function = args[arg_num]->as(); - if (function && TableFunctionFactory::instance().isTableFunctionName(function->name)) - { - remote_table_function_ptr = args[arg_num]; - ++arg_num; - } - else - { - args[arg_num] = evaluateConstantExpressionForDatabaseName(args[arg_num], context); - database = checkAndGetLiteralArgument(args[arg_num], "database"); - ++arg_num; - - auto qualified_name = QualifiedTableName::parseFromString(database); - if (qualified_name.database.empty()) + /// Names of database and table is not necessary. + if (arg_num < args.size()) + { + const auto * function = args[arg_num]->as(); + if (function && TableFunctionFactory::instance().isTableFunctionName(function->name)) { - if (arg_num >= args.size()) + remote_table_function_ptr = args[arg_num]; + ++arg_num; + } + else + { + args[arg_num] = evaluateConstantExpressionForDatabaseName(args[arg_num], context); + database = checkAndGetLiteralArgument(args[arg_num], "database"); + + ++arg_num; + + auto qualified_name = QualifiedTableName::parseFromString(database); + if (qualified_name.database.empty()) { - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + if (arg_num >= args.size()) + { + throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + else + { + std::swap(qualified_name.database, qualified_name.table); + args[arg_num] = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context); + qualified_name.table = checkAndGetLiteralArgument(args[arg_num], "table"); + ++arg_num; + } } - else + + database = std::move(qualified_name.database); + table = std::move(qualified_name.table); + + /// Cluster function may have sharding key for insert + if (is_cluster_function && arg_num < args.size()) { - std::swap(qualified_name.database, qualified_name.table); - args[arg_num] = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context); - qualified_name.table = checkAndGetLiteralArgument(args[arg_num], "table"); + sharding_key = args[arg_num]; ++arg_num; } } - - database = std::move(qualified_name.database); - table = std::move(qualified_name.table); - - /// Cluster function may have sharding key for insert - if (is_cluster_function && arg_num < args.size()) - { - sharding_key = args[arg_num]; - ++arg_num; - } } /// Username and password parameters are prohibited in cluster version of the function @@ -329,11 +343,13 @@ TableFunctionRemote::TableFunctionRemote(const std::string & name_, bool secure_ { is_cluster_function = (name == "cluster" || name == "clusterAllReplicas"); help_message = PreformattedMessage::create( - "Table function '{}' requires from 2 to {} parameters: " - ", , {}", + "Table function '{}' requires from {} to {} parameters: " + "{}", name, + is_cluster_function ? 0 : 1, is_cluster_function ? 4 : 6, - is_cluster_function ? " [, sharding_key]" : " [, username[, password], sharding_key]"); + is_cluster_function ? "[, , ] [, sharding_key]" + : " [, , ] [, username[, password], sharding_key]"); } void registerTableFunctionRemote(TableFunctionFactory & factory) diff --git a/tests/queries/0_stateless/00171_shard_array_of_tuple_remote.reference b/tests/queries/0_stateless/00171_shard_array_of_tuple_remote.reference index cd4823e219f..ece1f5aa525 100644 --- a/tests/queries/0_stateless/00171_shard_array_of_tuple_remote.reference +++ b/tests/queries/0_stateless/00171_shard_array_of_tuple_remote.reference @@ -1,2 +1,4 @@ [(1,4),(2,5),(3,6)] [(1,4),(2,5),(3,6)] +[(1,4),(2,5),(3,6)] +[(1,4),(2,5),(3,6)] diff --git a/tests/queries/0_stateless/00171_shard_array_of_tuple_remote.sql b/tests/queries/0_stateless/00171_shard_array_of_tuple_remote.sql index b28f4e65487..0189d3a63f5 100644 --- a/tests/queries/0_stateless/00171_shard_array_of_tuple_remote.sql +++ b/tests/queries/0_stateless/00171_shard_array_of_tuple_remote.sql @@ -1,3 +1,4 @@ -- Tags: shard SELECT arrayMap((x, y) -> (x, y), [1, 2, 3], [4, 5, 6]) FROM remote('127.0.0.{2,3}', system.one) ORDER BY rand(); +SELECT arrayMap((x, y) -> (x, y), [1, 2, 3], [4, 5, 6]) FROM remote('127.0.0.{2,3}') ORDER BY rand(); diff --git a/tests/queries/0_stateless/00220_shard_with_totals_in_subquery_remote_and_limit.reference b/tests/queries/0_stateless/00220_shard_with_totals_in_subquery_remote_and_limit.reference index a594e1495c1..d3cd76be236 100644 --- a/tests/queries/0_stateless/00220_shard_with_totals_in_subquery_remote_and_limit.reference +++ b/tests/queries/0_stateless/00220_shard_with_totals_in_subquery_remote_and_limit.reference @@ -1,3 +1,6 @@ 1 1 +1 + +1 diff --git a/tests/queries/0_stateless/00220_shard_with_totals_in_subquery_remote_and_limit.sql b/tests/queries/0_stateless/00220_shard_with_totals_in_subquery_remote_and_limit.sql index 80a35a4855a..a1e8d907a35 100644 --- a/tests/queries/0_stateless/00220_shard_with_totals_in_subquery_remote_and_limit.sql +++ b/tests/queries/0_stateless/00220_shard_with_totals_in_subquery_remote_and_limit.sql @@ -1,3 +1,4 @@ -- Tags: shard SELECT x FROM (SELECT count() AS x FROM remote('127.0.0.2', system.one) WITH TOTALS) LIMIT 1; +SELECT x FROM (SELECT count() AS x FROM remote('127.0.0.2') WITH TOTALS) LIMIT 1; diff --git a/tests/queries/0_stateless/01121_remote_scalar_subquery.reference b/tests/queries/0_stateless/01121_remote_scalar_subquery.reference index 6ed281c757a..98fb6a68656 100644 --- a/tests/queries/0_stateless/01121_remote_scalar_subquery.reference +++ b/tests/queries/0_stateless/01121_remote_scalar_subquery.reference @@ -1,2 +1,4 @@ 1 1 +1 +1 diff --git a/tests/queries/0_stateless/01121_remote_scalar_subquery.sql b/tests/queries/0_stateless/01121_remote_scalar_subquery.sql index eada5ed4b59..2d0c842c5b4 100644 --- a/tests/queries/0_stateless/01121_remote_scalar_subquery.sql +++ b/tests/queries/0_stateless/01121_remote_scalar_subquery.sql @@ -1 +1,2 @@ SELECT (SELECT 1) FROM remote('127.0.0.{1,2}', system.one); +SELECT (SELECT 1) FROM remote('127.0.0.{1,2}'); diff --git a/tests/queries/0_stateless/01754_cluster_all_replicas_shard_num.reference b/tests/queries/0_stateless/01754_cluster_all_replicas_shard_num.reference index d308efd8662..8c39c3bf41a 100644 --- a/tests/queries/0_stateless/01754_cluster_all_replicas_shard_num.reference +++ b/tests/queries/0_stateless/01754_cluster_all_replicas_shard_num.reference @@ -1,9 +1,18 @@ 1 1 1 +1 +1 +2 +1 +2 +1 2 1 2 1 1 +1 +2 +1 2 diff --git a/tests/queries/0_stateless/01754_cluster_all_replicas_shard_num.sql b/tests/queries/0_stateless/01754_cluster_all_replicas_shard_num.sql index 59e0ca60f8f..ea88e304c42 100644 --- a/tests/queries/0_stateless/01754_cluster_all_replicas_shard_num.sql +++ b/tests/queries/0_stateless/01754_cluster_all_replicas_shard_num.sql @@ -1,10 +1,16 @@ -- Tags: replica, shard SELECT _shard_num FROM cluster('test_shard_localhost', system.one); +SELECT _shard_num FROM cluster('test_shard_localhost'); SELECT _shard_num FROM clusterAllReplicas('test_shard_localhost', system.one); +SELECT _shard_num FROM clusterAllReplicas('test_shard_localhost'); SELECT _shard_num FROM cluster('test_cluster_two_shards', system.one) ORDER BY _shard_num; +SELECT _shard_num FROM cluster('test_cluster_two_shards') ORDER BY _shard_num; SELECT _shard_num FROM clusterAllReplicas('test_cluster_two_shards', system.one) ORDER BY _shard_num; +SELECT _shard_num FROM clusterAllReplicas('test_cluster_two_shards') ORDER BY _shard_num; SELECT _shard_num FROM cluster('test_cluster_one_shard_two_replicas', system.one) ORDER BY _shard_num; +SELECT _shard_num FROM cluster('test_cluster_one_shard_two_replicas') ORDER BY _shard_num; SELECT _shard_num FROM clusterAllReplicas('test_cluster_one_shard_two_replicas', system.one) ORDER BY _shard_num; +SELECT _shard_num FROM clusterAllReplicas('test_cluster_one_shard_two_replicas') ORDER BY _shard_num; diff --git a/tests/queries/0_stateless/01787_map_remote.reference b/tests/queries/0_stateless/01787_map_remote.reference index 1c488d4418e..c7828769f9f 100644 --- a/tests/queries/0_stateless/01787_map_remote.reference +++ b/tests/queries/0_stateless/01787_map_remote.reference @@ -1,2 +1,4 @@ {'a':1,'b':2} {'a':1,'b':2} +{'a':1,'b':2} +{'a':1,'b':2} diff --git a/tests/queries/0_stateless/01787_map_remote.sql b/tests/queries/0_stateless/01787_map_remote.sql index 748316c8044..217308e5141 100644 --- a/tests/queries/0_stateless/01787_map_remote.sql +++ b/tests/queries/0_stateless/01787_map_remote.sql @@ -1 +1,2 @@ SELECT map('a', 1, 'b', 2) FROM remote('127.0.0.{1,2}', system, one); +SELECT map('a', 1, 'b', 2) FROM remote('127.0.0.{1,2}'); diff --git a/tests/queries/0_stateless/01880_remote_ipv6.sql b/tests/queries/0_stateless/01880_remote_ipv6.sql index 057b3ad7ec6..7f15449e556 100644 --- a/tests/queries/0_stateless/01880_remote_ipv6.sql +++ b/tests/queries/0_stateless/01880_remote_ipv6.sql @@ -10,3 +10,14 @@ SELECT * FROM remote('::1', system.one) FORMAT Null; -- { serverError 36 } SELECT * FROM remote('[::1][::1]', system.one) FORMAT Null; -- { serverError 36 } SELECT * FROM remote('[::1][::1', system.one) FORMAT Null; -- { serverError 36 } SELECT * FROM remote('[::1]::1]', system.one) FORMAT Null; -- { serverError 36 } + +SELECT * FROM remote('[::1]') FORMAT Null; +SELECT * FROM remote('[::1]:9000') FORMAT Null; + +SELECT * FROM remote('[::1') FORMAT Null; -- { serverError 36 } +SELECT * FROM remote('::1]') FORMAT Null; -- { serverError 36 } +SELECT * FROM remote('::1') FORMAT Null; -- { serverError 36 } + +SELECT * FROM remote('[::1][::1]') FORMAT Null; -- { serverError 36 } +SELECT * FROM remote('[::1][::1') FORMAT Null; -- { serverError 36 } +SELECT * FROM remote('[::1]::1]') FORMAT Null; -- { serverError 36 } diff --git a/tests/queries/0_stateless/02000_table_function_cluster_macros.reference b/tests/queries/0_stateless/02000_table_function_cluster_macros.reference index 6ed281c757a..98fb6a68656 100644 --- a/tests/queries/0_stateless/02000_table_function_cluster_macros.reference +++ b/tests/queries/0_stateless/02000_table_function_cluster_macros.reference @@ -1,2 +1,4 @@ 1 1 +1 +1 diff --git a/tests/queries/0_stateless/02000_table_function_cluster_macros.sql b/tests/queries/0_stateless/02000_table_function_cluster_macros.sql index f1bc1358b55..d133f5fdc2c 100644 --- a/tests/queries/0_stateless/02000_table_function_cluster_macros.sql +++ b/tests/queries/0_stateless/02000_table_function_cluster_macros.sql @@ -1,2 +1,4 @@ SELECT _shard_num FROM cluster("{default_cluster_macro}", system.one); +SELECT _shard_num FROM cluster("{default_cluster_macro}"); SELECT _shard_num FROM clusterAllReplicas("{default_cluster_macro}", system.one); +SELECT _shard_num FROM clusterAllReplicas("{default_cluster_macro}"); diff --git a/tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.reference b/tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.reference index aa47d0d46d4..44e0be8e356 100644 --- a/tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.reference +++ b/tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.reference @@ -1,2 +1,4 @@ 0 0 +0 +0 diff --git a/tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.sql b/tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.sql index 6a5e4a0ae65..f0085b7660f 100644 --- a/tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.sql +++ b/tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.sql @@ -4,3 +4,7 @@ SET optimize_monotonous_functions_in_order_by = 1; SELECT * FROM cluster(test_cluster_two_shards_localhost, system, one) ORDER BY toDateTime(dummy); + +SELECT * +FROM cluster(test_cluster_two_shards_localhost) +ORDER BY toDateTime(dummy) diff --git a/tests/queries/0_stateless/02596_build_set_and_remote.reference b/tests/queries/0_stateless/02596_build_set_and_remote.reference index 8d12196ae33..b506eaf574f 100644 --- a/tests/queries/0_stateless/02596_build_set_and_remote.reference +++ b/tests/queries/0_stateless/02596_build_set_and_remote.reference @@ -4,16 +4,32 @@ SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM system.one; SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}', system.one); 1 1 +SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}'); +1 +1 SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}', system.one) GROUP BY NULL; 1 +SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}') GROUP BY NULL; +1 SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}', system.one) GROUP BY 1; 1 +SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}') GROUP BY 1; +1 SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}', system.one) GROUP BY 'A'; 1 +SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}') GROUP BY 'A'; +1 SELECT 1 IN ( SELECT 1 ) FROM remote('127.0.0.{1,2}', system.one) GROUP BY dummy; 1 +SELECT 1 IN ( SELECT 1 ) FROM remote('127.0.0.{1,2}') GROUP BY dummy; +1 SELECT 1000.0001, toUInt64(arrayJoin([NULL, 257, 65536, NULL])), arrayExists(x -> (x IN (SELECT '2.55')), [-9223372036854775808]) FROM remote('127.0.0.{1,2}', system.one) GROUP BY NULL, NULL, NULL, NULL; 1000.0001 \N 0 1000.0001 257 0 1000.0001 65536 0 1000.0001 \N 0 +SELECT 1000.0001, toUInt64(arrayJoin([NULL, 257, 65536, NULL])), arrayExists(x -> (x IN (SELECT '2.55')), [-9223372036854775808]) FROM remote('127.0.0.{1,2}') GROUP BY NULL, NULL, NULL, NULL; +1000.0001 \N 0 +1000.0001 257 0 +1000.0001 65536 0 +1000.0001 \N 0 diff --git a/tests/queries/0_stateless/02596_build_set_and_remote.sql b/tests/queries/0_stateless/02596_build_set_and_remote.sql index 7a904344c91..4785446c1ab 100644 --- a/tests/queries/0_stateless/02596_build_set_and_remote.sql +++ b/tests/queries/0_stateless/02596_build_set_and_remote.sql @@ -2,13 +2,19 @@ SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM system.one; SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}', system.one); +SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}'); SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}', system.one) GROUP BY NULL; +SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}') GROUP BY NULL; SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}', system.one) GROUP BY 1; +SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}') GROUP BY 1; SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}', system.one) GROUP BY 'A'; +SELECT arrayExists(x -> (x IN (SELECT '2')), [2]) FROM remote('127.0.0.{2,3}') GROUP BY 'A'; SELECT 1 IN ( SELECT 1 ) FROM remote('127.0.0.{1,2}', system.one) GROUP BY dummy; +SELECT 1 IN ( SELECT 1 ) FROM remote('127.0.0.{1,2}') GROUP BY dummy; SELECT 1000.0001, toUInt64(arrayJoin([NULL, 257, 65536, NULL])), arrayExists(x -> (x IN (SELECT '2.55')), [-9223372036854775808]) FROM remote('127.0.0.{1,2}', system.one) GROUP BY NULL, NULL, NULL, NULL; +SELECT 1000.0001, toUInt64(arrayJoin([NULL, 257, 65536, NULL])), arrayExists(x -> (x IN (SELECT '2.55')), [-9223372036854775808]) FROM remote('127.0.0.{1,2}') GROUP BY NULL, NULL, NULL, NULL; From 08b5196070fcece2cb904790c8327dfc76c20319 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Sun, 18 Jun 2023 19:54:31 +0200 Subject: [PATCH 070/554] Updated to support passing sas token authentication --- src/Storages/StorageAzureBlob.cpp | 78 +++++++++---------- .../TableFunctionAzureBlobStorage.cpp | 2 +- 2 files changed, 40 insertions(+), 40 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 3ee176a68b7..41179d1e641 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -86,7 +86,7 @@ const std::unordered_set optional_configuration_keys = { bool isConnectionString(const std::string & candidate) { - return candidate.starts_with("DefaultEndpointsProtocol"); + return !candidate.starts_with("http"); } } @@ -320,52 +320,52 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co } else { + std::shared_ptr storage_shared_key_credential; + std::unique_ptr blob_service_client; if (configuration.account_name.has_value() && configuration.account_key.has_value()) { - auto storage_shared_key_credential = std::make_shared(*configuration.account_name, *configuration.account_key); - auto blob_service_client = std::make_unique(configuration.connection_url, storage_shared_key_credential); - 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) - { - auto final_url = configuration.connection_url - + (configuration.connection_url.back() == '/' ? "" : "/") - + configuration.container; - - result = std::make_unique(final_url, storage_shared_key_credential); - } - else - { - throw; - } - } + storage_shared_key_credential + = std::make_shared(*configuration.account_name, *configuration.account_key); + blob_service_client = std::make_unique(configuration.connection_url, storage_shared_key_credential); } else { - auto managed_identity_credential = std::make_shared(); - auto blob_service_client = std::make_unique(configuration.connection_url, managed_identity_credential); - 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) - { - auto final_url = configuration.connection_url - + (configuration.connection_url.back() == '/' ? "" : "/") - + configuration.container; + blob_service_client = std::make_unique(configuration.connection_url); + } - result = std::make_unique(final_url, managed_identity_credential); + + try + { + blob_service_client = std::make_unique(configuration.connection_url); + 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) + { + 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 - { - throw; - } + 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 + result = std::make_unique(final_url); + } + else + { + throw; } } } diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp index d2a96173491..d51942d133d 100644 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp @@ -39,7 +39,7 @@ namespace bool isConnectionString(const std::string & candidate) { - return candidate.starts_with("DefaultEndpointsProtocol"); + return !candidate.starts_with("http"); } } From 9a5d917cb561fefb67dfc22fdf95f71970d54726 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 19 Jun 2023 14:59:21 +0200 Subject: [PATCH 071/554] Fixed clang build issue --- 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 41179d1e641..4eb60db0b99 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -344,7 +344,7 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co if (e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict) { std::string final_url; - size_t pos = configuration.connection_url.find("?"); + size_t pos = configuration.connection_url.find('?'); if (pos != std::string::npos) { auto url_without_sas = configuration.connection_url.substr(0, pos); From 11220777974125135c6054655a8231cd17877170 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 19 Jun 2023 13:36:01 +0000 Subject: [PATCH 072/554] Automatic style fix --- .../test_storage_azure_blob_storage/test_cluster.py | 3 ++- 1 file changed, 2 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 2a9ef95776b..15ac5590cbe 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_cluster.py +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -257,6 +257,7 @@ 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" @@ -281,4 +282,4 @@ def test_partition_parallel_readig_withcluster(cluster): """ ) - assert azure_cluster == "3\n" \ No newline at end of file + assert azure_cluster == "3\n" From 0de983e32b7268d05f4518824a9bd82c69f17412 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 19 Jun 2023 22:55:34 +0200 Subject: [PATCH 073/554] Added azureBlobStorageCluster to aspell-dict.txt --- 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 73ec64e2f30..39cf02850a7 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1084,6 +1084,7 @@ avro avx aws azureBlobStorage +azureBlobStorageCluster backend backoff backticks From 3ed56120c1793ced3e237a50f66a1e8e6ffdd2b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Jun 2023 00:24:58 +0300 Subject: [PATCH 074/554] Update TableFunctionRemote.cpp --- src/TableFunctions/TableFunctionRemote.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 3f3f6677e20..885ae7a08db 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -37,7 +37,10 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr String cluster_name; String cluster_description; - String database = "system", table = "one", username = "default", password; + String database = "system"; + String table = "one"; /// The table containing one row is used by default for queries without explicit table specification. + String username = "default"; + String password; if (args_func.size() != 1) throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); From 50002653fe66851a4c4f2af349db29d06c65fe87 Mon Sep 17 00:00:00 2001 From: "ducle.canh" Date: Wed, 21 Jun 2023 12:11:13 +0800 Subject: [PATCH 075/554] add a test to limit client max opening fd --- .../0_stateless/02790_client_max_opening_fd.reference | 1 + tests/queries/0_stateless/02790_client_max_opening_fd.sh | 9 +++++++++ 2 files changed, 10 insertions(+) create mode 100644 tests/queries/0_stateless/02790_client_max_opening_fd.reference create mode 100755 tests/queries/0_stateless/02790_client_max_opening_fd.sh diff --git a/tests/queries/0_stateless/02790_client_max_opening_fd.reference b/tests/queries/0_stateless/02790_client_max_opening_fd.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02790_client_max_opening_fd.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02790_client_max_opening_fd.sh b/tests/queries/0_stateless/02790_client_max_opening_fd.sh new file mode 100755 index 00000000000..289486ce389 --- /dev/null +++ b/tests/queries/0_stateless/02790_client_max_opening_fd.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +ulimit -n 1024 + +${CLICKHOUSE_CLIENT} --query "SELECT 1" From cc01b81a5ff607ea2c18f68cfc74bb07d58d8bfa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Jun 2023 01:26:04 +0300 Subject: [PATCH 076/554] Update 02790_client_max_opening_fd.sh --- tests/queries/0_stateless/02790_client_max_opening_fd.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02790_client_max_opening_fd.sh b/tests/queries/0_stateless/02790_client_max_opening_fd.sh index 289486ce389..ecc05d32050 100755 --- a/tests/queries/0_stateless/02790_client_max_opening_fd.sh +++ b/tests/queries/0_stateless/02790_client_max_opening_fd.sh @@ -4,6 +4,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh +# Ensure that clickhouse-client does not open a large number of files. ulimit -n 1024 - ${CLICKHOUSE_CLIENT} --query "SELECT 1" From 4060beae4928865d6f836f828769f2c8e3478e97 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 22 Jun 2023 17:58:57 +0000 Subject: [PATCH 077/554] Structure to CapnProto/Protobuf schema take 1 --- src/Core/Settings.h | 6 +- src/Core/SettingsEnums.cpp | 8 +- src/Core/SettingsEnums.h | 2 +- src/DataTypes/NestedUtils.cpp | 15 +- src/DataTypes/NestedUtils.h | 3 + src/Formats/CapnProtoSchema.cpp | 298 ++++ .../{CapnProtoUtils.h => CapnProtoSchema.h} | 13 +- src/Formats/CapnProtoSerializer.cpp | 1220 +++++++++++++++++ src/Formats/CapnProtoSerializer.h | 25 + src/Formats/CapnProtoUtils.cpp | 734 ---------- src/Formats/FormatFactory.cpp | 3 + src/Formats/FormatSchemaInfo.cpp | 82 ++ src/Formats/FormatSchemaInfo.h | 21 + src/Formats/FormatSettings.h | 9 +- src/Formats/ProtobufSerializer.cpp | 4 +- src/Formats/StructureToCapnProtoSchema.cpp | 236 ++++ src/Formats/StructureToCapnProtoSchema.h | 16 + src/Formats/StructureToFormatSchemaUtils.cpp | 113 ++ src/Formats/StructureToFormatSchemaUtils.h | 27 + src/Formats/StructureToProtobufSchema.cpp | 214 +++ src/Formats/StructureToProtobufSchema.h | 16 + src/Functions/generateRandomStructure.cpp | 507 +++++++ src/Functions/structureToFormatSchema.cpp | 137 ++ .../Formats/Impl/CapnProtoRowInputFormat.cpp | 252 +--- .../Formats/Impl/CapnProtoRowInputFormat.h | 12 +- .../Formats/Impl/CapnProtoRowOutputFormat.cpp | 275 +--- .../Formats/Impl/CapnProtoRowOutputFormat.h | 21 +- .../Formats/Impl/ProtobufListInputFormat.cpp | 6 +- .../Formats/Impl/ProtobufListInputFormat.h | 2 +- .../Formats/Impl/ProtobufListOutputFormat.cpp | 7 +- .../Formats/Impl/ProtobufListOutputFormat.h | 4 +- .../Formats/Impl/ProtobufRowInputFormat.cpp | 6 +- .../Formats/Impl/ProtobufRowInputFormat.h | 2 +- .../Formats/Impl/ProtobufRowOutputFormat.cpp | 7 +- .../Formats/Impl/ProtobufRowOutputFormat.h | 4 +- src/Storages/StorageGenerateRandom.cpp | 4 +- .../TableFunctionGenerateRandom.cpp | 6 +- src/TableFunctions/TableFunctionS3.cpp | 1 + 38 files changed, 3029 insertions(+), 1289 deletions(-) create mode 100644 src/Formats/CapnProtoSchema.cpp rename src/Formats/{CapnProtoUtils.h => CapnProtoSchema.h} (59%) create mode 100644 src/Formats/CapnProtoSerializer.cpp create mode 100644 src/Formats/CapnProtoSerializer.h delete mode 100644 src/Formats/CapnProtoUtils.cpp create mode 100644 src/Formats/StructureToCapnProtoSchema.cpp create mode 100644 src/Formats/StructureToCapnProtoSchema.h create mode 100644 src/Formats/StructureToFormatSchemaUtils.cpp create mode 100644 src/Formats/StructureToFormatSchemaUtils.h create mode 100644 src/Formats/StructureToProtobufSchema.cpp create mode 100644 src/Formats/StructureToProtobufSchema.h create mode 100644 src/Functions/generateRandomStructure.cpp create mode 100644 src/Functions/structureToFormatSchema.cpp diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 607be1522db..c4a5fe02855 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -965,7 +965,11 @@ class IColumn; M(Bool, output_format_orc_string_as_string, false, "Use ORC String type instead of Binary for String columns", 0) \ M(ORCCompression, output_format_orc_compression_method, "lz4", "Compression method for ORC output format. Supported codecs: lz4, snappy, zlib, zstd, none (uncompressed)", 0) \ \ - M(EnumComparingMode, format_capn_proto_enum_comparising_mode, FormatSettings::EnumComparingMode::BY_VALUES, "How to map ClickHouse Enum and CapnProto Enum", 0) \ + M(CapnProtoEnumComparingMode, format_capn_proto_enum_comparising_mode, FormatSettings::CapnProtoEnumComparingMode::BY_VALUES, "How to map ClickHouse Enum and CapnProto Enum", 0) \ + \ + M(Bool, format_capn_proto_use_autogenerated_schema, true, "Use autogenerated CapnProto schema when format_schema is not set", 0) \ + M(Bool, format_protobuf_use_autogenerated_schema, true, "Use autogenerated Protobuf when format_schema is not set", 0) \ + M(String, output_format_schema, "", "The path to the file where the automatically generated schema will be saved", 0) \ \ M(String, input_format_mysql_dump_table_name, "", "Name of the table in MySQL dump from which to read data", 0) \ M(Bool, input_format_mysql_dump_map_column_names, true, "Match columns from table in MySQL dump and columns from ClickHouse table by names", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index e0f16ea00db..a291a23c140 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -144,10 +144,10 @@ IMPLEMENT_SETTING_ENUM(TransactionsWaitCSNMode, ErrorCodes::BAD_ARGUMENTS, {"wait", TransactionsWaitCSNMode::WAIT}, {"wait_unknown", TransactionsWaitCSNMode::WAIT_UNKNOWN}}) -IMPLEMENT_SETTING_ENUM(EnumComparingMode, ErrorCodes::BAD_ARGUMENTS, - {{"by_names", FormatSettings::EnumComparingMode::BY_NAMES}, - {"by_values", FormatSettings::EnumComparingMode::BY_VALUES}, - {"by_names_case_insensitive", FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE}}) +IMPLEMENT_SETTING_ENUM(CapnProtoEnumComparingMode, ErrorCodes::BAD_ARGUMENTS, + {{"by_names", FormatSettings::CapnProtoEnumComparingMode::BY_NAMES}, + {"by_values", FormatSettings::CapnProtoEnumComparingMode::BY_VALUES}, + {"by_names_case_insensitive", FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE}}) IMPLEMENT_SETTING_ENUM(EscapingRule, ErrorCodes::BAD_ARGUMENTS, {{"None", FormatSettings::EscapingRule::None}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 3ae7bfaa673..1c5be910ef7 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -188,7 +188,7 @@ enum class TransactionsWaitCSNMode DECLARE_SETTING_ENUM(TransactionsWaitCSNMode) -DECLARE_SETTING_ENUM_WITH_RENAME(EnumComparingMode, FormatSettings::EnumComparingMode) +DECLARE_SETTING_ENUM_WITH_RENAME(CapnProtoEnumComparingMode, FormatSettings::CapnProtoEnumComparingMode) DECLARE_SETTING_ENUM_WITH_RENAME(EscapingRule, FormatSettings::EscapingRule) diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index f029ac6ba27..9ee803c4235 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -71,7 +71,7 @@ std::string extractTableName(const std::string & nested_name) } -Block flatten(const Block & block) +static Block flattenImpl(const Block & block, bool flatten_named_tuple) { Block res; @@ -114,7 +114,7 @@ Block flatten(const Block & block) else res.insert(elem); } - else if (const DataTypeTuple * type_tuple = typeid_cast(elem.type.get())) + else if (const DataTypeTuple * type_tuple = typeid_cast(elem.type.get()); type_tuple && flatten_named_tuple) { if (type_tuple->haveExplicitNames()) { @@ -143,6 +143,17 @@ Block flatten(const Block & block) return res; } +Block flatten(const Block & block) +{ + return flattenImpl(block, true); +} + + +Block flattenArrayOfTuples(const Block & block) +{ + return flattenImpl(block, false); +} + namespace { diff --git a/src/DataTypes/NestedUtils.h b/src/DataTypes/NestedUtils.h index 90fdd683493..e009ceb18fe 100644 --- a/src/DataTypes/NestedUtils.h +++ b/src/DataTypes/NestedUtils.h @@ -23,6 +23,9 @@ namespace Nested /// 2) For an Array with named Tuple element column, a Array(Tuple(x ..., y ..., ...)), replace it with multiple Array Columns, a.x ..., a.y ..., ... Block flatten(const Block & block); + /// Same as flatten but only for Array with named Tuple element column. + Block flattenArrayOfTuples(const Block & block); + /// Collect Array columns in a form of `column_name.element_name` to single Array(Tuple(...)) column. NamesAndTypesList collect(const NamesAndTypesList & names_and_types); diff --git a/src/Formats/CapnProtoSchema.cpp b/src/Formats/CapnProtoSchema.cpp new file mode 100644 index 00000000000..22518d5061a --- /dev/null +++ b/src/Formats/CapnProtoSchema.cpp @@ -0,0 +1,298 @@ +#include + +#if USE_CAPNP + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_CAPN_PROTO_SCHEMA; + extern const int BAD_TYPE_OF_FIELD; + extern const int FILE_DOESNT_EXIST; + extern const int UNKNOWN_EXCEPTION; + extern const int CAPN_PROTO_BAD_TYPE; + extern const int BAD_ARGUMENTS; +} + +capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaInfo & schema_info) +{ + capnp::ParsedSchema schema; + try + { + int fd; + KJ_SYSCALL(fd = open(schema_info.schemaDirectory().data(), O_RDONLY)); // NOLINT(bugprone-suspicious-semicolon) + auto schema_dir = kj::newDiskDirectory(kj::OsFileHandle(fd)); + schema = impl.parseFromDirectory(*schema_dir, kj::Path::parse(schema_info.schemaPath()), {}); + } + catch (const kj::Exception & e) + { + /// That's not good to determine the type of error by its description, but + /// this is the only way to do it here, because kj doesn't specify the type of error. + auto description = std::string_view(e.getDescription().cStr()); + if (description.find("No such file or directory") != String::npos || description.find("no such directory") != String::npos) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot open CapnProto schema, file {} doesn't exists", schema_info.absoluteSchemaPath()); + + if (description.find("Parse error") != String::npos) + throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, "Cannot parse CapnProto schema {}:{}", schema_info.schemaPath(), e.getLine()); + + throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, + "Unknown exception while parsing CapnProto schema: {}, schema dir and file: {}, {}", + description, schema_info.schemaDirectory(), schema_info.schemaPath()); + } + + auto message_maybe = schema.findNested(schema_info.messageName()); + auto * message_schema = kj::_::readMaybe(message_maybe); + if (!message_schema) + throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, + "CapnProto schema doesn't contain message with name {}", schema_info.messageName()); + return message_schema->asStruct(); +} + +bool checkIfStructContainsUnnamedUnion(const capnp::StructSchema & struct_schema) +{ + return struct_schema.getFields().size() != struct_schema.getNonUnionFields().size(); +} + +bool checkIfStructIsNamedUnion(const capnp::StructSchema & struct_schema) +{ + return struct_schema.getFields().size() == struct_schema.getUnionFields().size(); +} + +/// Get full name of type for better exception messages. +String getCapnProtoFullTypeName(const capnp::Type & type) +{ + static const std::map capnp_simple_type_names = + { + {capnp::schema::Type::Which::BOOL, "Bool"}, + {capnp::schema::Type::Which::VOID, "Void"}, + {capnp::schema::Type::Which::INT8, "Int8"}, + {capnp::schema::Type::Which::INT16, "Int16"}, + {capnp::schema::Type::Which::INT32, "Int32"}, + {capnp::schema::Type::Which::INT64, "Int64"}, + {capnp::schema::Type::Which::UINT8, "UInt8"}, + {capnp::schema::Type::Which::UINT16, "UInt16"}, + {capnp::schema::Type::Which::UINT32, "UInt32"}, + {capnp::schema::Type::Which::UINT64, "UInt64"}, + {capnp::schema::Type::Which::FLOAT32, "Float32"}, + {capnp::schema::Type::Which::FLOAT64, "Float64"}, + {capnp::schema::Type::Which::TEXT, "Text"}, + {capnp::schema::Type::Which::DATA, "Data"}, + {capnp::schema::Type::Which::INTERFACE, "Interface"}, + {capnp::schema::Type::Which::ANY_POINTER, "AnyPointer"}, + }; + + switch (type.which()) + { + case capnp::schema::Type::Which::STRUCT: + { + auto struct_schema = type.asStruct(); + + auto non_union_fields = struct_schema.getNonUnionFields(); + std::vector non_union_field_names; + for (auto nested_field : non_union_fields) + non_union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); + + auto union_fields = struct_schema.getUnionFields(); + std::vector union_field_names; + for (auto nested_field : union_fields) + union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); + + String union_name = "Union(" + boost::algorithm::join(union_field_names, ", ") + ")"; + /// Check if the struct is a named union. + if (non_union_field_names.empty()) + return union_name; + + String type_name = "Struct(" + boost::algorithm::join(non_union_field_names, ", "); + /// Check if the struct contains unnamed union. + if (!union_field_names.empty()) + type_name += ", " + union_name; + type_name += ")"; + return type_name; + } + case capnp::schema::Type::Which::LIST: + return "List(" + getCapnProtoFullTypeName(type.asList().getElementType()) + ")"; + case capnp::schema::Type::Which::ENUM: + { + auto enum_schema = type.asEnum(); + String enum_name = "Enum("; + auto enumerants = enum_schema.getEnumerants(); + for (unsigned i = 0; i != enumerants.size(); ++i) + { + enum_name += String(enumerants[i].getProto().getName()) + " = " + std::to_string(enumerants[i].getOrdinal()); + if (i + 1 != enumerants.size()) + enum_name += ", "; + } + enum_name += ")"; + return enum_name; + } + default: + auto it = capnp_simple_type_names.find(type.which()); + if (it == capnp_simple_type_names.end()) + throw Exception(ErrorCodes::BAD_TYPE_OF_FIELD, "Unknown CapnProto type"); + return it->second; + } +} + +namespace +{ + + template + static DataTypePtr getEnumDataTypeFromEnumerants(const capnp::EnumSchema::EnumerantList & enumerants) + { + std::vector> values; + for (auto enumerant : enumerants) + values.emplace_back(enumerant.getProto().getName(), ValueType(enumerant.getOrdinal())); + return std::make_shared>(std::move(values)); + } + + static DataTypePtr getEnumDataTypeFromEnumSchema(const capnp::EnumSchema & enum_schema) + { + auto enumerants = enum_schema.getEnumerants(); + if (enumerants.size() < 128) + return getEnumDataTypeFromEnumerants(enumerants); + if (enumerants.size() < 32768) + return getEnumDataTypeFromEnumerants(enumerants); + + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "ClickHouse supports only 8 and 16-bit Enums"); + } + + static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type, bool skip_unsupported_fields) + { + switch (capnp_type.which()) + { + case capnp::schema::Type::INT8: + return std::make_shared(); + case capnp::schema::Type::INT16: + return std::make_shared(); + case capnp::schema::Type::INT32: + return std::make_shared(); + case capnp::schema::Type::INT64: + return std::make_shared(); + case capnp::schema::Type::BOOL: [[fallthrough]]; + case capnp::schema::Type::UINT8: + return std::make_shared(); + case capnp::schema::Type::UINT16: + return std::make_shared(); + case capnp::schema::Type::UINT32: + return std::make_shared(); + case capnp::schema::Type::UINT64: + return std::make_shared(); + case capnp::schema::Type::FLOAT32: + return std::make_shared(); + case capnp::schema::Type::FLOAT64: + return std::make_shared(); + case capnp::schema::Type::DATA: [[fallthrough]]; + case capnp::schema::Type::TEXT: + return std::make_shared(); + case capnp::schema::Type::ENUM: + return getEnumDataTypeFromEnumSchema(capnp_type.asEnum()); + case capnp::schema::Type::LIST: + { + auto list_schema = capnp_type.asList(); + auto nested_type = getDataTypeFromCapnProtoType(list_schema.getElementType(), skip_unsupported_fields); + if (!nested_type) + return nullptr; + return std::make_shared(nested_type); + } + case capnp::schema::Type::STRUCT: + { + auto struct_schema = capnp_type.asStruct(); + + + if (struct_schema.getFields().size() == 0) + { + if (skip_unsupported_fields) + return nullptr; + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Empty messages are not supported"); + } + + /// Check if it can be Nullable. + if (checkIfStructIsNamedUnion(struct_schema)) + { + auto fields = struct_schema.getUnionFields(); + if (fields.size() != 2 || (!fields[0].getType().isVoid() && !fields[1].getType().isVoid())) + { + if (skip_unsupported_fields) + return nullptr; + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unions are not supported"); + } + auto value_type = fields[0].getType().isVoid() ? fields[1].getType() : fields[0].getType(); + if (value_type.isStruct() || value_type.isList()) + { + if (skip_unsupported_fields) + return nullptr; + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Tuples and Lists cannot be inside Nullable"); + } + + auto nested_type = getDataTypeFromCapnProtoType(value_type, skip_unsupported_fields); + if (!nested_type) + return nullptr; + return std::make_shared(nested_type); + } + + if (checkIfStructContainsUnnamedUnion(struct_schema)) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported"); + + /// Treat Struct as Tuple. + DataTypes nested_types; + Names nested_names; + for (auto field : struct_schema.getNonUnionFields()) + { + auto nested_type = getDataTypeFromCapnProtoType(field.getType(), skip_unsupported_fields); + if (!nested_type) + continue; + nested_names.push_back(field.getProto().getName()); + nested_types.push_back(nested_type); + } + if (nested_types.empty()) + return nullptr; + return std::make_shared(std::move(nested_types), std::move(nested_names)); + } + default: + { + if (skip_unsupported_fields) + return nullptr; + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unsupported CapnProtoType: {}", getCapnProtoFullTypeName(capnp_type)); + } + } +} + +} + +NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema, bool skip_unsupported_fields) +{ + if (checkIfStructContainsUnnamedUnion(schema)) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported"); + + NamesAndTypesList names_and_types; + for (auto field : schema.getNonUnionFields()) + { + auto name = field.getProto().getName(); + auto type = getDataTypeFromCapnProtoType(field.getType(), skip_unsupported_fields); + if (type) + names_and_types.emplace_back(name, type); + } + if (names_and_types.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Cannot convert CapnProto schema to ClickHouse table schema, all fields have unsupported types"); + + return names_and_types; +} + +} + +#endif diff --git a/src/Formats/CapnProtoUtils.h b/src/Formats/CapnProtoSchema.h similarity index 59% rename from src/Formats/CapnProtoUtils.h rename to src/Formats/CapnProtoSchema.h index 2d8cdb418d7..225f6f56207 100644 --- a/src/Formats/CapnProtoUtils.h +++ b/src/Formats/CapnProtoSchema.h @@ -30,17 +30,14 @@ public: capnp::StructSchema getMessageSchema(const FormatSchemaInfo & schema_info); }; -std::pair splitCapnProtoFieldName(const String & name); +bool checkIfStructContainsUnnamedUnion(const capnp::StructSchema & struct_schema); +bool checkIfStructIsNamedUnion(const capnp::StructSchema & struct_schema); -bool compareEnumNames(const String & first, const String & second, FormatSettings::EnumComparingMode mode); - -std::pair getStructBuilderAndFieldByColumnName(capnp::DynamicStruct::Builder struct_builder, const String & name); - -capnp::DynamicValue::Reader getReaderByColumnName(const capnp::DynamicStruct::Reader & struct_reader, const String & name); - -void checkCapnProtoSchemaStructure(const capnp::StructSchema & schema, const Block & header, FormatSettings::EnumComparingMode mode); +/// Get full name of type for better exception messages. +String getCapnProtoFullTypeName(const capnp::Type & type); NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema, bool skip_unsupported_fields); + } #endif diff --git a/src/Formats/CapnProtoSerializer.cpp b/src/Formats/CapnProtoSerializer.cpp new file mode 100644 index 00000000000..1a9a51756ba --- /dev/null +++ b/src/Formats/CapnProtoSerializer.cpp @@ -0,0 +1,1220 @@ +#include +#include +#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 THERE_IS_NO_COLUMN; + extern const int BAD_TYPE_OF_FIELD; + extern const int CAPN_PROTO_BAD_CAST; + extern const int INCORRECT_DATA; + extern const int ILLEGAL_COLUMN; +} + +namespace +{ + std::pair splitFieldName(const String & name) + { + const auto * begin = name.data(); + const auto * end = name.data() + name.size(); + const auto * it = find_first_symbols<'_', '.'>(begin, end); + String first = String(begin, it); + String second = it == end ? "" : String(it + 1, end); + return {first, second}; + } + + std::optional findFieldByName(const capnp::StructSchema & struct_schema, const String & name) + { + const auto & fields = struct_schema.getFields(); + for (auto field : fields) + { + auto field_name = String(field.getProto().getName()); + if (boost::to_lower_copy(name) == boost::to_lower_copy(field_name)) + return field; + } + return std::nullopt; + } + + [[noreturn]] void throwCannotConvert(const DataTypePtr & type, const String & name, const capnp::Type & capnp_type) + { + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\" with type {} to CapnProto type {}", + name, + type->getName(), + getCapnProtoFullTypeName(capnp_type)); + } + + struct FieldBuilder + { + virtual ~FieldBuilder() = default; + }; + + struct ListBuilder : public FieldBuilder + { + explicit ListBuilder(capnp::DynamicValue::Builder builder) : impl(builder.as()) + { + } + + capnp::DynamicList::Builder impl; + std::vector> nested_builders; + }; + + struct StructBuilder : public FieldBuilder + { + explicit StructBuilder(capnp::DynamicValue::Builder builder, size_t fields_size) : impl(builder.as()), field_builders(fields_size) + { + } + + explicit StructBuilder(capnp::DynamicStruct::Builder struct_builder, size_t fields_size) : impl(std::move(struct_builder)), field_builders(fields_size) + { + } + + capnp::DynamicStruct::Builder impl; + std::vector> field_builders; + }; + + std::unique_ptr initStructFieldBuilderIfNeeded(const ColumnPtr & column, size_t row_num, capnp::DynamicStruct::Builder & struct_builder, const capnp::StructSchema::Field & field, const capnp::Type & capnp_type, size_t nested_fields_size) + { + switch (capnp_type.which()) + { + case capnp::schema::Type::LIST: + { + const auto * array_column = assert_cast(column.get()); + size_t size = array_column->getOffsets()[row_num] - array_column->getOffsets()[row_num - 1]; + return std::make_unique(struct_builder.init(field, static_cast(size))); + } + case capnp::schema::Type::STRUCT: + { + return std::make_unique(struct_builder.init(field), nested_fields_size); + } + default: + return nullptr; + } + } + + class ICapnProtoSerializer + { + public: + virtual std::optional writeRow(const ColumnPtr & column, FieldBuilder * builder, size_t row_num) = 0; + virtual void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) = 0; + + virtual ~ICapnProtoSerializer() = default; + }; + + template + class CapnProtoIntegerSerializer : public ICapnProtoSerializer + { + public: + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::INT) + return capnp::DynamicValue::Reader(column->getInt(row_num)); + if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::UINT) + return capnp::DynamicValue::Reader(column->getUInt(row_num)); + return capnp::DynamicValue::Reader(column->getBool(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + NumericType value; + if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::INT) + value = static_cast(reader.as()); + else if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::UINT) + value = static_cast(reader.as()); + else if constexpr (capnp_dynamic_type == capnp::DynamicValue::Type::BOOL) + value = static_cast(reader.as()); + + if constexpr (is_bool_data_type) + assert_cast(column).insertValue(static_cast(value)); + else + assert_cast &>(column).insertValue(value); + } + }; + + template + static std::unique_ptr createIntegerSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + switch (capnp_type.which()) + { + case capnp::schema::Type::INT8: [[fallthrough]]; + case capnp::schema::Type::INT16: [[fallthrough]]; + case capnp::schema::Type::INT32: [[fallthrough]]; + case capnp::schema::Type::INT64: + return std::make_unique>(); + case capnp::schema::Type::UINT8: [[fallthrough]]; + case capnp::schema::Type::UINT16: [[fallthrough]]; + case capnp::schema::Type::UINT32: [[fallthrough]]; + case capnp::schema::Type::UINT64: + return std::make_unique>(); + case capnp::schema::Type::BOOL: + return std::make_unique>(); + default: + throwCannotConvert(data_type, column_name, capnp_type); + } + } + + template + class CapnProtoBigIntegerSerializer : public ICapnProtoSerializer + { + public: + CapnProtoBigIntegerSerializer(const DataTypePtr & data_type_, const String & column_name, const capnp::Type & capnp_type) : data_type(data_type_) + { + if (!capnp_type.isData()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto value = reader.as(); + if (value.size() != sizeof(NumericType)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", data_type->getName(), value.size()); + + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + + private: + DataTypePtr data_type; + }; + + template + class CapnProtoFloatSerializer : public ICapnProtoSerializer + { + public: + CapnProtoFloatSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isFloat32() && !capnp_type.isFloat64()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getFloat64(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast &>(column).insertValue(reader.as()); + } + }; + + template + class CapnProtoEnumSerializer : public ICapnProtoSerializer + { + public: + CapnProtoEnumSerializer( + const DataTypePtr & data_type_, + const String & column_name, + const capnp::Type & capnp_type, + const FormatSettings::CapnProtoEnumComparingMode enum_comparing_mode_) : data_type(data_type_), enum_schema(capnp_type.asEnum()), enum_comparing_mode(enum_comparing_mode_) + { + if (!capnp_type.isEnum()) + throwCannotConvert(data_type, column_name, capnp_type); + + bool to_lower = enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE; + const auto * enum_type = assert_cast *>(data_type.get()); + const auto & enum_values = dynamic_cast &>(*enum_type); + + auto enumerants = enum_schema.getEnumerants(); + constexpr auto max_value = std::is_same_v ? INT8_MAX : INT16_MAX; + if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) + { + /// In CapnProto Enum fields are numbered sequentially starting from zero. + if (enumerants.size() > max_value) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Enum from CapnProto schema contains values that are out of range for Clickhouse enum type {}", + data_type->getName()); + + auto values = enum_values.getSetOfAllValues(); + std::unordered_set capn_enum_values; + for (auto enumerant : enumerants) + capn_enum_values.insert(EnumType(enumerant.getOrdinal())); + if (values != capn_enum_values) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "The set of values in Enum from CapnProto schema is different from the set of values in ClickHouse Enum"); + } + else + { + auto names = enum_values.getSetOfAllNames(to_lower); + std::unordered_set capn_enum_names; + + for (auto enumerant : enumerants) + { + String name = enumerant.getProto().getName(); + capn_enum_names.insert(to_lower ? boost::algorithm::to_lower_copy(name) : name); + } + + if (names != capn_enum_names) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "The set of names in Enum from CapnProto schema is different from the set of names in ClickHouse Enum"); + } + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + const auto * enum_data_type = assert_cast *>(data_type.get()); + EnumType enum_value = assert_cast &>(*column).getElement(row_num); + if (enum_comparing_mode == FormatSettings::CapnProtoEnumComparingMode::BY_VALUES) + return capnp::DynamicValue::Reader(capnp::DynamicEnum(enum_schema, enum_value)); + + auto enum_name = enum_data_type->getNameForValue(enum_value); + for (const auto enumerant : enum_schema.getEnumerants()) + { + if (compareEnumNames(String(enum_name), enumerant.getProto().getName(), enum_comparing_mode)) + return capnp::DynamicValue::Reader(capnp::DynamicEnum(enumerant)); + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert CLickHouse Enum value to CapnProto Enum"); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto enum_value = reader.as(); + auto enumerant = *kj::_::readMaybe(enum_value.getEnumerant()); + auto enum_type = assert_cast *>(data_type.get()); + DataTypePtr nested_type = std::make_shared>(); + switch (enum_comparing_mode) + { + case FormatSettings::CapnProtoEnumComparingMode::BY_VALUES: + { + assert_cast &>(column).insertValue(static_cast(enumerant.getOrdinal())); + return; + } + case FormatSettings::CapnProtoEnumComparingMode::BY_NAMES: + { + auto value = enum_type->getValue(String(enumerant.getProto().getName())); + assert_cast &>(column).insertValue(value); + return; + } + case FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE: + { + /// Find the same enum name case insensitive. + String enum_name = enumerant.getProto().getName(); + for (auto & name : enum_type->getAllRegisteredNames()) + { + if (compareEnumNames(name, enum_name, enum_comparing_mode)) + { + assert_cast &>(column).insertValue(enum_type->getValue(name)); + break; + } + } + return; + } + } + } + + private: + bool compareEnumNames(const String & first, const String & second, const FormatSettings::CapnProtoEnumComparingMode mode) + { + if (mode == FormatSettings::CapnProtoEnumComparingMode::BY_NAMES_CASE_INSENSITIVE) + return boost::algorithm::to_lower_copy(first) == boost::algorithm::to_lower_copy(second); + return first == second; + } + + DataTypePtr data_type; + capnp::EnumSchema enum_schema; + const FormatSettings::CapnProtoEnumComparingMode enum_comparing_mode; + }; + + class CapnProtoDateSerializer : public ICapnProtoSerializer + { + public: + CapnProtoDateSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isUInt16()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getUInt(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast(column).insertValue(reader.as()); + } + }; + + class CapnProtoDate32Serializer : public ICapnProtoSerializer + { + public: + CapnProtoDate32Serializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isInt32()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getInt(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast(column).insertValue(reader.as()); + } + }; + + class CapnProtoDateTimeSerializer : public ICapnProtoSerializer + { + public: + CapnProtoDateTimeSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isUInt32()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getInt(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast(column).insertValue(reader.as()); + } + }; + + class CapnProtoDateTime64Serializer : public ICapnProtoSerializer + { + public: + CapnProtoDateTime64Serializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isInt64()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getInt(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast(column).insertValue(reader.as()); + } + }; + + template + class CapnProtoDecimalSerializer : public ICapnProtoSerializer + { + public: + CapnProtoDecimalSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + auto which = WhichDataType(data_type); + if ((!capnp_type.isInt32() && which.isDecimal32()) || (!capnp_type.isInt64() && which.isDecimal64())) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(column->getInt(row_num)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast &>(column).insertValue(reader.as()); + } + }; + + template + class CapnProtoBigDecimalSerializer : public ICapnProtoSerializer + { + public: + CapnProtoBigDecimalSerializer(const DataTypePtr & data_type_, const String & column_name, const capnp::Type & capnp_type) : data_type(data_type_) + { + if (!capnp_type.isData()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto value = reader.as(); + if (value.size() != sizeof(DecimalType)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", data_type->getName(), value.size()); + + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + + private: + DataTypePtr data_type; + }; + + template + class CapnProtoStringSerializer : public ICapnProtoSerializer + { + public: + CapnProtoStringSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type_) : capnp_type(capnp_type_) + { + if (!capnp_type.isData() && !capnp_type.isText()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + if constexpr (is_binary) + { + auto data = column->getDataAt(row_num); + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + } + + /// For type TEXT 0 byte must be at the end, so we cannot use getDataAt method, because it cuts last 0 byte. + const auto & string_column = assert_cast(*column); + const auto & chars = string_column.getChars(); + const auto & offsets = string_column.getOffsets(); + size_t start = offsets[ssize_t(row_num) - 1]; + size_t size = offsets[row_num] - start; + return capnp::DynamicValue::Reader(capnp::Text::Reader(reinterpret_cast(&chars[start]), size)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + if constexpr (is_binary) + { + auto value = reader.as(); + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + else + { + auto value = reader.as(); + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + } + + private: + capnp::Type capnp_type; + }; + + template + class CapnProtoFixedStringSerializer : public ICapnProtoSerializer + { + public: + CapnProtoFixedStringSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type_) : capnp_type(capnp_type_) + { + if (!capnp_type.isData() && !capnp_type.isText()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + if constexpr (is_binary) + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + + if (data.data[data.size - 1] == 0) + return capnp::DynamicValue::Reader(capnp::Text::Reader(reinterpret_cast(data.data), data.size)); + + /// In TEXT type data should be null-terminated, but ClickHouse FixedString data could not be. + /// To make data null-terminated we should copy it to temporary String object and use it in capnp::Text::Reader. + /// Note that capnp::Text::Reader works only with pointer to the data and it's size, so we should + /// guarantee that new String object life time is longer than capnp::Text::Reader life time. + tmp_string = data.toString(); + return capnp::DynamicValue::Reader(capnp::Text::Reader(reinterpret_cast(tmp_string.data()), tmp_string.size())); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto & fixed_string_column = assert_cast(column); + if constexpr (is_binary) + { + auto value = reader.as(); + if (value.size() > fixed_string_column.getN()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read data with size {} to FixedString with size {}", value.size(), fixed_string_column.getN()); + + fixed_string_column.insertData(reinterpret_cast(value.begin()), value.size()); + } + else + { + auto value = reader.as(); + if (value.size() > fixed_string_column.getN()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read data with size {} to FixedString with size {}", value.size(), fixed_string_column.getN()); + + fixed_string_column.insertData(reinterpret_cast(value.begin()), value.size()); + } + } + + private: + String tmp_string; + capnp::Type capnp_type; + }; + + class CapnProtoIPv4Serializer : public ICapnProtoSerializer + { + public: + CapnProtoIPv4Serializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isUInt32()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + return capnp::DynamicValue::Reader(assert_cast(*column).getElement(row_num).toUnderType()); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + assert_cast(column).insertValue(IPv4(reader.as())); + } + }; + + class CapnProtoIPv6Serializer : public ICapnProtoSerializer + { + public: + CapnProtoIPv6Serializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isData()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto value = reader.as(); + if (value.size() != sizeof(IPv6)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of IPv6 value: {}", value.size()); + + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + }; + + class CapnProtoUUIDSerializer : public ICapnProtoSerializer + { + public: + CapnProtoUUIDSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type) + { + if (!capnp_type.isData()) + throwCannotConvert(data_type, column_name, capnp_type); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder *, size_t row_num) override + { + auto data = column->getDataAt(row_num); + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto value = reader.as(); + if (value.size() != sizeof(UUID)) + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of UUID value: {}", value.size()); + + column.insertData(reinterpret_cast(value.begin()), value.size()); + } + }; + + std::unique_ptr createSerializer(const DataTypePtr & type, const String & name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings); + + class CapnProtoLowCardinalitySerializer : public ICapnProtoSerializer + { + public: + CapnProtoLowCardinalitySerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + nested_serializer = createSerializer(assert_cast(*data_type).getDictionaryType(), column_name, capnp_type, settings); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + { + const auto & low_cardinality_column = assert_cast(*column); + size_t index = low_cardinality_column.getIndexAt(row_num); + const auto & dict_column = low_cardinality_column.getDictionary().getNestedColumn(); + return nested_serializer->writeRow(dict_column, field_builder, index); + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto & low_cardinality_column = assert_cast(column); + auto tmp_column = low_cardinality_column.getDictionary().getNestedColumn()->cloneEmpty(); + nested_serializer->readRow(*tmp_column, reader); + low_cardinality_column.insertFromFullColumn(*tmp_column, 0); + } + + private: + std::unique_ptr nested_serializer; + }; + + class CapnProtoNullableSerializer : public ICapnProtoSerializer + { + public: + CapnProtoNullableSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + if (!capnp_type.isStruct()) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert column \"{}\": Nullable can be represented only as a named union of type Void and nested type, got CapnProto type {}", + column_name, + getCapnProtoFullTypeName(capnp_type)); + + /// Check that struct is a named union of type VOID and one arbitrary type. + auto struct_schema = capnp_type.asStruct(); + if (!checkIfStructIsNamedUnion(struct_schema)) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert column \"{}\": Nullable can be represented only as a named union of type Void and nested type." + "Given CapnProto struct is not a named union: {}", + column_name, + getCapnProtoFullTypeName(capnp_type)); + + auto union_fields = struct_schema.getUnionFields(); + if (union_fields.size() != 2) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert column \"{}\": Nullable can be represented only as a named union of type Void and nested type." + "Given CapnProto union have more than 2 fields: {}", + column_name, + getCapnProtoFullTypeName(capnp_type)); + + auto first = union_fields[0]; + auto second = union_fields[1]; + auto nested_type = assert_cast(data_type.get())->getNestedType(); + if (first.getType().isVoid()) + { + null_field = first; + nested_field = second; + nested_capnp_type = second.getType(); + if (nested_capnp_type.isStruct()) + nested_fields_size = nested_capnp_type.asStruct().getFields().size(); + nested_serializer = createSerializer(nested_type, column_name, nested_capnp_type, settings); + } + else if (second.getType().isVoid()) + { + null_field = second; + nested_field = first; + nested_capnp_type = first.getType(); + if (nested_capnp_type.isStruct()) + nested_fields_size = nested_capnp_type.asStruct().getFields().size(); + nested_serializer = createSerializer(nested_type, column_name, nested_capnp_type, settings); + } + else + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert column \"{}\": Nullable can be represented only as a named union of type Void and nested type." + "Given CapnProto union doesn't have field with type Void: {}", + column_name, + getCapnProtoFullTypeName(capnp_type)); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + { + assert(field_builder); + auto & struct_builder = assert_cast(*field_builder); + const auto & nullable_column = assert_cast(*column); + if (nullable_column.isNullAt(row_num)) + { + struct_builder.impl.set(null_field, capnp::Void()); + } + else + { + struct_builder.impl.clear(nested_field); + const auto & nested_column = nullable_column.getNestedColumnPtr(); + auto nested_field_builder = initStructFieldBuilderIfNeeded(nested_column, row_num, struct_builder.impl, nested_field, nested_capnp_type, nested_fields_size); + auto value = nested_serializer->writeRow(nested_column, nested_field_builder.get(), row_num); + if (value) + struct_builder.impl.set(nested_field, *value); + } + + return std::nullopt; + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto struct_reader = reader.as(); + auto & nullable_column = assert_cast(column); + auto field = *kj::_::readMaybe(struct_reader.which()); + if (field.getType().isVoid()) + nullable_column.insertDefault(); + else + { + auto & nested_column = nullable_column.getNestedColumn(); + auto nested_reader = struct_reader.get(field); + nested_serializer->readRow(nested_column, nested_reader); + nullable_column.getNullMapData().push_back(0); + } + } + + private: + std::unique_ptr nested_serializer; + capnp::StructSchema::Field null_field; + capnp::StructSchema::Field nested_field; + size_t nested_fields_size = 0; + capnp::Type nested_capnp_type; + }; + + class CapnProtoArraySerializer : public ICapnProtoSerializer + { + public: + CapnProtoArraySerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + if (!capnp_type.isList()) + throwCannotConvert(data_type, column_name, capnp_type); + + auto nested_type = assert_cast(data_type.get())->getNestedType(); + element_type = capnp_type.asList().getElementType(); + if (element_type.isStruct()) + element_struct_fields = element_type.asStruct().getFields().size(); + nested_serializer = createSerializer(nested_type, column_name, capnp_type.asList().getElementType(), settings); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + { + assert(field_builder); + auto & list_builder = assert_cast(*field_builder); + const auto * array_column = assert_cast(column.get()); + const auto & nested_column = array_column->getDataPtr(); + const auto & offsets = array_column->getOffsets(); + auto offset = offsets[row_num - 1]; + size_t size = offsets[row_num] - offset; + bool need_nested_builders = list_builder.nested_builders.empty(); + for (unsigned i = 0; i != static_cast(size); ++i) + { + if (need_nested_builders) + { + /// For nested lists we need to initialize nested list builder. + if (element_type.isList()) + { + const auto & nested_offset = checkAndGetColumn(*nested_column)->getOffsets(); + size_t nested_array_size = nested_offset[offset + i] - nested_offset[offset + i - 1]; + list_builder.nested_builders.emplace_back(std::make_unique(list_builder.impl.init(i, static_cast(nested_array_size)))); + } + else if (element_type.isStruct()) + { + list_builder.nested_builders.emplace_back(std::make_unique(list_builder.impl[i], element_struct_fields)); + } + else + { + list_builder.nested_builders.emplace_back(); + } + } + + auto value = nested_serializer->writeRow(nested_column, list_builder.nested_builders[i].get(), offset + i); + if (value) + list_builder.impl.set(i, *value); + } + + return std::nullopt; + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto list_reader = reader.as(); + auto & column_array = assert_cast(column); + auto & offsets = column_array.getOffsets(); + offsets.push_back(offsets.back() + list_reader.size()); + + auto & nested_column = column_array.getData(); + for (const auto & nested_reader : list_reader) + nested_serializer->readRow(nested_column, nested_reader); + } + + private: + std::unique_ptr nested_serializer; + capnp::Type element_type; + size_t element_struct_fields; + }; + + class CapnProtoMapSerializer : public ICapnProtoSerializer + { + public: + CapnProtoMapSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + /// We output/input Map type as follow CapnProto schema + /// + /// struct Map { + /// struct Entry { + /// key @0: Key; + /// value @1: Value; + /// } + /// entries @0 :List(Entry); + /// } + + if (!capnp_type.isStruct()) + throwCannotConvert(data_type, column_name, capnp_type); + + auto struct_schema = capnp_type.asStruct(); + + if (checkIfStructContainsUnnamedUnion(struct_schema)) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\" with type {} to CapnProto Struct with unnamed union {}", + column_name, + data_type->getName(), + getCapnProtoFullTypeName(capnp_type)); + + if (struct_schema.getFields().size() != 1) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\": Map type can be represented as a Struct with one list field, got struct: {}", + column_name, + getCapnProtoFullTypeName(capnp_type)); + + const auto & field_type = struct_schema.getFields()[0].getType(); + if (!field_type.isList()) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\": Map type can be represented as a Struct with one list field, got field: {}", + column_name, + getCapnProtoFullTypeName(field_type)); + + auto list_element_type = field_type.asList().getElementType(); + if (!list_element_type.isStruct()) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\": Field of struct that represents Map should be a list of structs, got list of {}", + column_name, + getCapnProtoFullTypeName(list_element_type)); + + auto key_value_struct = list_element_type.asStruct(); + if (checkIfStructContainsUnnamedUnion(key_value_struct)) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\": struct that represents Map entries is unnamed union: {}", + column_name, + getCapnProtoFullTypeName(list_element_type)); + + if (key_value_struct.getFields().size() != 2) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\": struct that represents Map entries should contain only 2 fields, got struct {}", + column_name, + getCapnProtoFullTypeName(list_element_type)); + + const auto & map_type = assert_cast(*data_type); + DataTypes types = {map_type.getKeyType(), map_type.getValueType()}; + Names names = {"key", "value"}; + auto entries_type = std::make_shared(std::make_shared(types, names)); + entries_field = struct_schema.getFields()[0]; + entries_capnp_type = entries_field.getType(); + nested_serializer = createSerializer(entries_type, column_name, field_type, settings); + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder * field_builder, size_t row_num) override + { + assert(field_builder); + auto & struct_builder = assert_cast(*field_builder); + const auto & entries_column = assert_cast(column.get())->getNestedColumnPtr(); + auto entries_builder = initStructFieldBuilderIfNeeded(entries_column, row_num, struct_builder.impl, entries_field, entries_capnp_type, 0); + nested_serializer->writeRow(entries_column, entries_builder.get(), row_num); + return std::nullopt; + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto struct_reader = reader.as(); + auto & entries_column = assert_cast(column).getNestedColumn(); + nested_serializer->readRow(entries_column, struct_reader.get(entries_field)); + } + + private: + std::unique_ptr nested_serializer; + capnp::StructSchema::Field entries_field; + capnp::Type entries_capnp_type; + }; + + class CapnProtoStructureSerializer : public ICapnProtoSerializer + { + public: + CapnProtoStructureSerializer(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings) + { + initialize(data_types, names, schema, settings); + } + + CapnProtoStructureSerializer(const DataTypePtr & data_type, const String & column_name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + if (!capnp_type.isStruct()) + throwCannotConvert(data_type, column_name, capnp_type); + + auto struct_schema = capnp_type.asStruct(); + + if (checkIfStructIsNamedUnion(struct_schema) || checkIfStructContainsUnnamedUnion(struct_schema)) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\" with type {} to CapnProto named union/struct with unnamed union {}", + column_name, + data_type->getName(), + getCapnProtoFullTypeName(capnp_type)); + + const auto * tuple_data_type = assert_cast(data_type.get()); + auto nested_types = tuple_data_type->getElements(); + Names nested_names; + bool have_explicit_names = tuple_data_type->haveExplicitNames(); + auto structure_fields = struct_schema.getFields(); + if (!have_explicit_names) + { + if (nested_types.size() != structure_fields.size()) + throw Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse column \"{}\" with type {} to CapnProto type {}: Tuple and Struct have different sizes {} != {}", + column_name, + data_type->getName(), + getCapnProtoFullTypeName(capnp_type), + nested_types.size(), + structure_fields.size()); + nested_names.reserve(structure_fields.size()); + for (auto field : structure_fields) + nested_names.push_back(field.getProto().getName()); + } + else + { + nested_names = tuple_data_type->getElementNames(); + } + + try + { + initialize(nested_types, nested_names, struct_schema, settings); + } + catch (Exception & e) + { + e.addMessage("(while converting column {})", column_name); + throw e; + } + } + + std::optional writeRow(const ColumnPtr & column, FieldBuilder * builder, size_t row_num) override + { + assert(builder); + auto & struct_builder = assert_cast(*builder); + if (auto tuple_column = typeid_cast(column.get())) + writeRow(tuple_column->getColumnsCopy(), struct_builder, row_num); + else + writeRow(Columns{column}, struct_builder, row_num); + return std::nullopt; + } + + void writeRow(const Columns & columns, StructBuilder & struct_builder, size_t row_num) + { + for (size_t i = 0; i != columns.size(); ++i) + { + const auto & field = fields[i]; + size_t field_index = field.getIndex(); + if (likely(!struct_builder.field_builders[field_index])) + struct_builder.field_builders[field_index] = initStructFieldBuilderIfNeeded( + columns[i], row_num, struct_builder.impl, field, fields_types[i], nested_field_sizes[i]); + + auto value = field_serializers[i]->writeRow(columns[i], struct_builder.field_builders[field_index].get(), row_num); + if (value) + struct_builder.impl.set(field, *value); + } + } + + void readRow(IColumn & column, const capnp::DynamicValue::Reader & reader) override + { + auto struct_reader = reader.as(); + if (auto tuple_column = typeid_cast(&column)) + { + for (size_t i = 0; i != tuple_column->tupleSize(); ++i) + field_serializers[i]->readRow(tuple_column->getColumn(i), struct_reader.get(fields[i])); + } + else + field_serializers[0]->readRow(column, struct_reader.get(fields[0])); + } + + void readRow(MutableColumns & columns, const capnp::DynamicStruct::Reader & reader) + { + for (size_t i = 0; i != columns.size(); ++i) + field_serializers[i]->readRow(*columns[i], reader.get(fields[i])); + } + + private: + void initialize(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings) + { + field_serializers.reserve(data_types.size()); + fields.reserve(data_types.size()); + fields_types.reserve(data_types.size()); + nested_field_sizes.reserve(data_types.size()); + for (size_t i = 0; i != data_types.size(); ++i) + { + auto [field_name, _] = splitFieldName(names[i]); + auto field = findFieldByName(schema, field_name); + if (!field) + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto schema doesn't contain field with name {}", field_name); + + fields.push_back(*field); + auto capnp_type = field->getType(); + fields_types.push_back(capnp_type); + nested_field_sizes.push_back(capnp_type.isStruct() ? capnp_type.asStruct().getFields().size() : 0); + field_serializers.push_back(createSerializer(data_types[i], names[i], capnp_type, settings)); + } + } + + std::vector> field_serializers; + std::vector fields; + std::vector nested_field_sizes; + std::vector fields_types; + }; + + std::unique_ptr createSerializer(const DataTypePtr & type, const String & name, const capnp::Type & capnp_type, const FormatSettings::CapnProto & settings) + { + auto [field_name, nested_name] = splitFieldName(name); + if (!nested_name.empty() && !capnp_type.isList()) + { + if (!capnp_type.isStruct()) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Field {} is not a struct", field_name); + + return std::make_unique(DataTypes{type}, Names{nested_name}, capnp_type.asStruct(), settings); + } + + switch (type->getTypeId()) + { + case TypeIndex::Int8: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::UInt8: + if (isBool(type)) + return createIntegerSerializer(type, name, capnp_type); + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::Int16: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::UInt16: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::Int32: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::UInt32: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::Int64: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::UInt64: + return createIntegerSerializer(type, name, capnp_type); + case TypeIndex::Int128: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::UInt128: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Int256: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::UInt256: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Float32: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Float64: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Date: + return std::make_unique(type, name, capnp_type); + case TypeIndex::Date32: + return std::make_unique(type, name, capnp_type); + case TypeIndex::DateTime: + return std::make_unique(type, name, capnp_type); + case TypeIndex::DateTime64: + return std::make_unique(type, name, capnp_type); + case TypeIndex::Decimal32: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Decimal64: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Decimal128: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::Decimal256: + return std::make_unique>(type, name, capnp_type); + case TypeIndex::IPv4: + return std::make_unique(type, name, capnp_type); + case TypeIndex::IPv6: + return std::make_unique(type, name, capnp_type); + case TypeIndex::UUID: + return std::make_unique(type, name, capnp_type); + case TypeIndex::Enum8: + return std::make_unique>(type, name, capnp_type, settings.enum_comparing_mode); + case TypeIndex::Enum16: + return std::make_unique>(type, name, capnp_type, settings.enum_comparing_mode); + case TypeIndex::String: + if (capnp_type.isData()) + return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); + case TypeIndex::FixedString: + if (capnp_type.isData()) + return std::make_unique>(type, name, capnp_type); + return std::make_unique>(type, name, capnp_type); + case TypeIndex::LowCardinality: + return std::make_unique(type, name, capnp_type, settings); + case TypeIndex::Nullable: + return std::make_unique(type, name, capnp_type, settings); + case TypeIndex::Array: + return std::make_unique(type, name, capnp_type, settings); + case TypeIndex::Map: + return std::make_unique(type, name, capnp_type, settings); + case TypeIndex::Tuple: + return std::make_unique(type, name, capnp_type, settings); + default: + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Type {} is not supported in CapnProto format", type->getName()); + } + } +} + +class CapnProtoSerializer::Impl +{ +public: + Impl(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings) + : struct_serializer(std::make_unique(data_types, names, schema, settings)) + , fields_size(schema.getFields().size()) + { + } + + void writeRow(const Columns & columns, capnp::DynamicStruct::Builder builder, size_t row_num) + { + StructBuilder struct_builder(std::move(builder), fields_size); + struct_serializer->writeRow(columns, struct_builder, row_num); + } + + void readRow(MutableColumns & columns, capnp::DynamicStruct::Reader & reader) + { + struct_serializer->readRow(columns, reader); + } + +private: + std::unique_ptr struct_serializer; + size_t fields_size; +}; + +CapnProtoSerializer::CapnProtoSerializer(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings) + : serializer_impl(std::make_unique(data_types, names, schema, settings)) +{ +} + +void CapnProtoSerializer::writeRow(const Columns & columns, capnp::DynamicStruct::Builder builder, size_t row_num) +{ + serializer_impl->writeRow(columns, std::move(builder), row_num); +} + +void CapnProtoSerializer::readRow(MutableColumns & columns, capnp::DynamicStruct::Reader & reader) +{ + serializer_impl->readRow(columns, reader); +} + +CapnProtoSerializer::~CapnProtoSerializer() = default; + +} diff --git a/src/Formats/CapnProtoSerializer.h b/src/Formats/CapnProtoSerializer.h new file mode 100644 index 00000000000..efae797875b --- /dev/null +++ b/src/Formats/CapnProtoSerializer.h @@ -0,0 +1,25 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class CapnProtoSerializer +{ +public: + CapnProtoSerializer(const DataTypes & data_types, const Names & names, const capnp::StructSchema & schema, const FormatSettings::CapnProto & settings); + + void writeRow(const Columns & columns, capnp::DynamicStruct::Builder builder, size_t row_num); + + void readRow(MutableColumns & columns, capnp::DynamicStruct::Reader & reader); + + ~CapnProtoSerializer(); + +private: + class Impl; + std::unique_ptr serializer_impl; +}; + +} diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp deleted file mode 100644 index d6c032408bb..00000000000 --- a/src/Formats/CapnProtoUtils.cpp +++ /dev/null @@ -1,734 +0,0 @@ -#include - -#if USE_CAPNP - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_PARSE_CAPN_PROTO_SCHEMA; - extern const int THERE_IS_NO_COLUMN; - extern const int BAD_TYPE_OF_FIELD; - extern const int CAPN_PROTO_BAD_CAST; - extern const int FILE_DOESNT_EXIST; - extern const int UNKNOWN_EXCEPTION; - extern const int INCORRECT_DATA; - extern const int CAPN_PROTO_BAD_TYPE; - extern const int BAD_ARGUMENTS; -} - -std::pair splitCapnProtoFieldName(const String & name) -{ - const auto * begin = name.data(); - const auto * end = name.data() + name.size(); - const auto * it = find_first_symbols<'_', '.'>(begin, end); - String first = String(begin, it); - String second = it == end ? "" : String(it + 1, end); - return {first, second}; -} - -capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaInfo & schema_info) -{ - capnp::ParsedSchema schema; - try - { - int fd; - KJ_SYSCALL(fd = open(schema_info.schemaDirectory().data(), O_RDONLY)); // NOLINT(bugprone-suspicious-semicolon) - auto schema_dir = kj::newDiskDirectory(kj::OsFileHandle(fd)); - schema = impl.parseFromDirectory(*schema_dir, kj::Path::parse(schema_info.schemaPath()), {}); - } - catch (const kj::Exception & e) - { - /// That's not good to determine the type of error by its description, but - /// this is the only way to do it here, because kj doesn't specify the type of error. - auto description = std::string_view(e.getDescription().cStr()); - if (description.find("No such file or directory") != String::npos || description.find("no such directory") != String::npos) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot open CapnProto schema, file {} doesn't exists", schema_info.absoluteSchemaPath()); - - if (description.find("Parse error") != String::npos) - throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, "Cannot parse CapnProto schema {}:{}", schema_info.schemaPath(), e.getLine()); - - throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, - "Unknown exception while parsing CapnProto schema: {}, schema dir and file: {}, {}", - description, schema_info.schemaDirectory(), schema_info.schemaPath()); - } - - auto message_maybe = schema.findNested(schema_info.messageName()); - auto * message_schema = kj::_::readMaybe(message_maybe); - if (!message_schema) - throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, - "CapnProto schema doesn't contain message with name {}", schema_info.messageName()); - return message_schema->asStruct(); -} - -bool compareEnumNames(const String & first, const String & second, FormatSettings::EnumComparingMode mode) -{ - if (mode == FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE) - return boost::algorithm::to_lower_copy(first) == boost::algorithm::to_lower_copy(second); - return first == second; -} - -static const std::map capnp_simple_type_names = -{ - {capnp::schema::Type::Which::BOOL, "Bool"}, - {capnp::schema::Type::Which::VOID, "Void"}, - {capnp::schema::Type::Which::INT8, "Int8"}, - {capnp::schema::Type::Which::INT16, "Int16"}, - {capnp::schema::Type::Which::INT32, "Int32"}, - {capnp::schema::Type::Which::INT64, "Int64"}, - {capnp::schema::Type::Which::UINT8, "UInt8"}, - {capnp::schema::Type::Which::UINT16, "UInt16"}, - {capnp::schema::Type::Which::UINT32, "UInt32"}, - {capnp::schema::Type::Which::UINT64, "UInt64"}, - {capnp::schema::Type::Which::FLOAT32, "Float32"}, - {capnp::schema::Type::Which::FLOAT64, "Float64"}, - {capnp::schema::Type::Which::TEXT, "Text"}, - {capnp::schema::Type::Which::DATA, "Data"}, - {capnp::schema::Type::Which::INTERFACE, "Interface"}, - {capnp::schema::Type::Which::ANY_POINTER, "AnyPointer"}, -}; - -static bool checkIfStructContainsUnnamedUnion(const capnp::StructSchema & struct_schema) -{ - return struct_schema.getFields().size() != struct_schema.getNonUnionFields().size(); -} - -static bool checkIfStructIsNamedUnion(const capnp::StructSchema & struct_schema) -{ - return struct_schema.getFields().size() == struct_schema.getUnionFields().size(); -} - -/// Get full name of type for better exception messages. -static String getCapnProtoFullTypeName(const capnp::Type & type) -{ - switch (type.which()) - { - case capnp::schema::Type::Which::STRUCT: - { - auto struct_schema = type.asStruct(); - - auto non_union_fields = struct_schema.getNonUnionFields(); - std::vector non_union_field_names; - for (auto nested_field : non_union_fields) - non_union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); - - auto union_fields = struct_schema.getUnionFields(); - std::vector union_field_names; - for (auto nested_field : union_fields) - union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); - - String union_name = "Union(" + boost::algorithm::join(union_field_names, ", ") + ")"; - /// Check if the struct is a named union. - if (non_union_field_names.empty()) - return union_name; - - String type_name = "Struct(" + boost::algorithm::join(non_union_field_names, ", "); - /// Check if the struct contains unnamed union. - if (!union_field_names.empty()) - type_name += ", " + union_name; - type_name += ")"; - return type_name; - } - case capnp::schema::Type::Which::LIST: - return "List(" + getCapnProtoFullTypeName(type.asList().getElementType()) + ")"; - case capnp::schema::Type::Which::ENUM: - { - auto enum_schema = type.asEnum(); - String enum_name = "Enum("; - auto enumerants = enum_schema.getEnumerants(); - for (unsigned i = 0; i != enumerants.size(); ++i) - { - enum_name += String(enumerants[i].getProto().getName()) + " = " + std::to_string(enumerants[i].getOrdinal()); - if (i + 1 != enumerants.size()) - enum_name += ", "; - } - enum_name += ")"; - return enum_name; - } - default: - auto it = capnp_simple_type_names.find(type.which()); - if (it == capnp_simple_type_names.end()) - throw Exception(ErrorCodes::BAD_TYPE_OF_FIELD, "Unknown CapnProto type"); - return it->second; - } -} - -template -static bool checkEnums(const capnp::Type & capnp_type, const DataTypePtr column_type, FormatSettings::EnumComparingMode mode, UInt64 max_value, String & error_message) -{ - if (!capnp_type.isEnum()) - return false; - - auto enum_schema = capnp_type.asEnum(); - bool to_lower = mode == FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE; - const auto * enum_type = assert_cast *>(column_type.get()); - const auto & enum_values = dynamic_cast &>(*enum_type); - - auto enumerants = enum_schema.getEnumerants(); - if (mode == FormatSettings::EnumComparingMode::BY_VALUES) - { - /// In CapnProto Enum fields are numbered sequentially starting from zero. - if (enumerants.size() > max_value) - { - error_message += "Enum from CapnProto schema contains values that is out of range for Clickhouse Enum"; - return false; - } - - auto values = enum_values.getSetOfAllValues(); - std::unordered_set capn_enum_values; - for (auto enumerant : enumerants) - capn_enum_values.insert(Type(enumerant.getOrdinal())); - auto result = values == capn_enum_values; - if (!result) - error_message += "The set of values in Enum from CapnProto schema is different from the set of values in ClickHouse Enum"; - return result; - } - - auto names = enum_values.getSetOfAllNames(to_lower); - std::unordered_set capn_enum_names; - - for (auto enumerant : enumerants) - { - String name = enumerant.getProto().getName(); - capn_enum_names.insert(to_lower ? boost::algorithm::to_lower_copy(name) : name); - } - - auto result = names == capn_enum_names; - if (!result) - error_message += "The set of names in Enum from CapnProto schema is different from the set of names in ClickHouse Enum"; - return result; -} - -static bool checkCapnProtoType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message, const String & column_name); - -static bool checkNullableType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message, const String & column_name) -{ - if (!capnp_type.isStruct()) - return false; - - /// Check that struct is a named union of type VOID and one arbitrary type. - auto struct_schema = capnp_type.asStruct(); - if (!checkIfStructIsNamedUnion(struct_schema)) - return false; - - auto union_fields = struct_schema.getUnionFields(); - if (union_fields.size() != 2) - return false; - - auto first = union_fields[0]; - auto second = union_fields[1]; - - auto nested_type = assert_cast(data_type.get())->getNestedType(); - if (first.getType().isVoid()) - return checkCapnProtoType(second.getType(), nested_type, mode, error_message, column_name); - if (second.getType().isVoid()) - return checkCapnProtoType(first.getType(), nested_type, mode, error_message, column_name); - return false; -} - -static bool checkTupleType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message) -{ - if (!capnp_type.isStruct()) - return false; - auto struct_schema = capnp_type.asStruct(); - - if (checkIfStructIsNamedUnion(struct_schema)) - return false; - - if (checkIfStructContainsUnnamedUnion(struct_schema)) - { - error_message += "CapnProto struct contains unnamed union"; - return false; - } - - const auto * tuple_data_type = assert_cast(data_type.get()); - auto nested_types = tuple_data_type->getElements(); - if (nested_types.size() != struct_schema.getFields().size()) - { - error_message += "Tuple and Struct types have different sizes"; - return false; - } - - bool have_explicit_names = tuple_data_type->haveExplicitNames(); - const auto & nested_names = tuple_data_type->getElementNames(); - for (uint32_t i = 0; i != nested_names.size(); ++i) - { - if (have_explicit_names) - { - KJ_IF_MAYBE (field, struct_schema.findFieldByName(nested_names[i])) - { - if (!checkCapnProtoType(field->getType(), nested_types[tuple_data_type->getPositionByName(nested_names[i])], mode, error_message, nested_names[i])) - return false; - } - else - { - error_message += "CapnProto struct doesn't contain a field with name " + nested_names[i]; - return false; - } - } - else if (!checkCapnProtoType(struct_schema.getFields()[i].getType(), nested_types[tuple_data_type->getPositionByName(nested_names[i])], mode, error_message, nested_names[i])) - return false; - } - - return true; -} - -static bool checkArrayType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message, const String & column_name) -{ - if (!capnp_type.isList()) - return false; - auto list_schema = capnp_type.asList(); - auto nested_type = assert_cast(data_type.get())->getNestedType(); - - auto [field_name, nested_name] = splitCapnProtoFieldName(column_name); - if (!nested_name.empty() && list_schema.getElementType().isStruct()) - { - auto struct_schema = list_schema.getElementType().asStruct(); - KJ_IF_MAYBE(field, struct_schema.findFieldByName(nested_name)) - return checkCapnProtoType(field->getType(), nested_type, mode, error_message, nested_name); - - error_message += "Element type of List {} doesn't contain field with name " + nested_name; - return false; - } - - return checkCapnProtoType(list_schema.getElementType(), nested_type, mode, error_message, column_name); -} - -static bool checkMapType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message) -{ - /// We output/input Map type as follow CapnProto schema - /// - /// struct Map { - /// struct Entry { - /// key @0: Key; - /// value @1: Value; - /// } - /// entries @0 :List(Entry); - /// } - - if (!capnp_type.isStruct()) - return false; - auto struct_schema = capnp_type.asStruct(); - - if (checkIfStructContainsUnnamedUnion(struct_schema)) - { - error_message += "CapnProto struct contains unnamed union"; - return false; - } - - if (struct_schema.getFields().size() != 1) - { - error_message += "CapnProto struct that represents Map type can contain only one field"; - return false; - } - - const auto & field_type = struct_schema.getFields()[0].getType(); - if (!field_type.isList()) - { - error_message += "Field of CapnProto struct that represents Map is not a list"; - return false; - } - - auto list_element_type = field_type.asList().getElementType(); - if (!list_element_type.isStruct()) - { - error_message += "Field of CapnProto struct that represents Map is not a list of structs"; - return false; - } - - auto key_value_struct = list_element_type.asStruct(); - if (checkIfStructContainsUnnamedUnion(key_value_struct)) - { - error_message += "CapnProto struct contains unnamed union"; - return false; - } - - if (key_value_struct.getFields().size() != 2) - { - error_message += "Key-value structure for Map struct should have exactly 2 fields"; - return false; - } - - const auto & map_type = assert_cast(*data_type); - DataTypes types = {map_type.getKeyType(), map_type.getValueType()}; - Names names = {"key", "value"}; - - for (size_t i = 0; i != types.size(); ++i) - { - KJ_IF_MAYBE(field, key_value_struct.findFieldByName(names[i])) - { - if (!checkCapnProtoType(field->getType(), types[i], mode, error_message, names[i])) - return false; - } - else - { - error_message += R"(Key-value structure for Map struct should have exactly 2 fields with names "key" and "value")"; - return false; - } - } - - return true; -} - -static bool isCapnInteger(const capnp::Type & capnp_type) -{ - return capnp_type.isInt8() || capnp_type.isUInt8() || capnp_type.isInt16() || capnp_type.isUInt16() || capnp_type.isInt32() - || capnp_type.isUInt32() || capnp_type.isInt64() || capnp_type.isUInt64(); -} - -static bool checkCapnProtoType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message, const String & column_name) -{ - switch (data_type->getTypeId()) - { - case TypeIndex::UInt8: - return capnp_type.isBool() || isCapnInteger(capnp_type); - case TypeIndex::Int8: [[fallthrough]]; - case TypeIndex::Int16: [[fallthrough]]; - case TypeIndex::UInt16: [[fallthrough]]; - case TypeIndex::Int32: [[fallthrough]]; - case TypeIndex::UInt32: [[fallthrough]]; - case TypeIndex::Int64: [[fallthrough]]; - case TypeIndex::UInt64: - /// Allow integer conversions durin input/output. - return isCapnInteger(capnp_type); - case TypeIndex::Date: - return capnp_type.isUInt16(); - case TypeIndex::DateTime: [[fallthrough]]; - case TypeIndex::IPv4: - return capnp_type.isUInt32(); - case TypeIndex::Date32: [[fallthrough]]; - case TypeIndex::Decimal32: - return capnp_type.isInt32() || capnp_type.isUInt32(); - case TypeIndex::DateTime64: [[fallthrough]]; - case TypeIndex::Decimal64: - return capnp_type.isInt64() || capnp_type.isUInt64(); - case TypeIndex::Float32:[[fallthrough]]; - case TypeIndex::Float64: - /// Allow converting between Float32 and isFloat64 - return capnp_type.isFloat32() || capnp_type.isFloat64(); - case TypeIndex::Enum8: - return checkEnums(capnp_type, data_type, mode, INT8_MAX, error_message); - case TypeIndex::Enum16: - return checkEnums(capnp_type, data_type, mode, INT16_MAX, error_message); - case TypeIndex::Int128: [[fallthrough]]; - case TypeIndex::UInt128: [[fallthrough]]; - case TypeIndex::Int256: [[fallthrough]]; - case TypeIndex::UInt256: [[fallthrough]]; - case TypeIndex::Decimal128: [[fallthrough]]; - case TypeIndex::Decimal256: - return capnp_type.isData(); - case TypeIndex::Tuple: - return checkTupleType(capnp_type, data_type, mode, error_message); - case TypeIndex::Nullable: - { - auto result = checkNullableType(capnp_type, data_type, mode, error_message, column_name); - if (!result) - error_message += "Nullable can be represented only as a named union of type Void and nested type"; - return result; - } - case TypeIndex::Array: - return checkArrayType(capnp_type, data_type, mode, error_message, column_name); - case TypeIndex::LowCardinality: - return checkCapnProtoType(capnp_type, assert_cast(data_type.get())->getDictionaryType(), mode, error_message, column_name); - case TypeIndex::FixedString: [[fallthrough]]; - case TypeIndex::IPv6: [[fallthrough]]; - case TypeIndex::String: - return capnp_type.isText() || capnp_type.isData(); - case TypeIndex::Map: - return checkMapType(capnp_type, data_type, mode, error_message); - default: - return false; - } -} - -capnp::DynamicValue::Reader getReaderByColumnName(const capnp::DynamicStruct::Reader & struct_reader, const String & name) -{ - auto [field_name, nested_name] = splitCapnProtoFieldName(name); - KJ_IF_MAYBE(field, struct_reader.getSchema().findFieldByName(field_name)) - { - capnp::DynamicValue::Reader field_reader; - try - { - field_reader = struct_reader.get(*field); - } - catch (const kj::Exception & e) - { - throw Exception(ErrorCodes::INCORRECT_DATA, - "Cannot extract field value from struct by provided schema, error: " - "{} Perhaps the data was generated by another schema", String(e.getDescription().cStr())); - } - - if (nested_name.empty()) - return field_reader; - - /// Support reading Nested as List of Structs. - if (field_reader.getType() == capnp::DynamicValue::LIST) - { - auto list_schema = field->getType().asList(); - if (!list_schema.getElementType().isStruct()) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Element type of List {} is not a struct", field_name); - - auto struct_schema = list_schema.getElementType().asStruct(); - KJ_IF_MAYBE(nested_field, struct_schema.findFieldByName(nested_name)) - return field_reader; - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Element type of List {} doesn't contain field with name \"{}\"", field_name, nested_name); - } - - if (field_reader.getType() != capnp::DynamicValue::STRUCT) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Field {} is not a struct", field_name); - - return getReaderByColumnName(field_reader.as(), nested_name); - } - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto struct doesn't contain field with name {}", field_name); -} - -std::pair getStructBuilderAndFieldByColumnName(capnp::DynamicStruct::Builder struct_builder, const String & name) -{ - auto [field_name, nested_name] = splitCapnProtoFieldName(name); - KJ_IF_MAYBE(field, struct_builder.getSchema().findFieldByName(field_name)) - { - if (nested_name.empty()) - return {struct_builder, *field}; - - auto field_builder = struct_builder.get(*field); - - /// Support reading Nested as List of Structs. - if (field_builder.getType() == capnp::DynamicValue::LIST) - { - auto list_schema = field->getType().asList(); - if (!list_schema.getElementType().isStruct()) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Element type of List {} is not a struct", field_name); - - auto struct_schema = list_schema.getElementType().asStruct(); - KJ_IF_MAYBE(nested_field, struct_schema.findFieldByName(nested_name)) - return {struct_builder, *field}; - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Element type of List {} doesn't contain field with name \"{}\"", field_name, nested_name); - } - - if (field_builder.getType() != capnp::DynamicValue::STRUCT) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Field {} is not a struct", field_name); - - return getStructBuilderAndFieldByColumnName(field_builder.as(), nested_name); - } - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto struct doesn't contain field with name {}", field_name); -} - -static std::pair getFieldByName(const capnp::StructSchema & schema, const String & name) -{ - auto [field_name, nested_name] = splitCapnProtoFieldName(name); - KJ_IF_MAYBE(field, schema.findFieldByName(field_name)) - { - if (nested_name.empty()) - return {*field, name}; - - /// Support reading Nested as List of Structs. - if (field->getType().isList()) - { - auto list_schema = field->getType().asList(); - if (!list_schema.getElementType().isStruct()) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Element type of List {} is not a struct", field_name); - - auto struct_schema = list_schema.getElementType().asStruct(); - KJ_IF_MAYBE(nested_field, struct_schema.findFieldByName(nested_name)) - return {*field, name}; - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Element type of List {} doesn't contain field with name \"{}\"", field_name, nested_name); - } - - if (!field->getType().isStruct()) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Field {} is not a struct", field_name); - - return getFieldByName(field->getType().asStruct(), nested_name); - } - - throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto schema doesn't contain field with name {}", field_name); -} - -void checkCapnProtoSchemaStructure(const capnp::StructSchema & schema, const Block & header, FormatSettings::EnumComparingMode mode) -{ - /// Firstly check that struct doesn't contain unnamed union, because we don't support it. - if (checkIfStructContainsUnnamedUnion(schema)) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Schema contains unnamed union that is not supported"); - auto names_and_types = header.getNamesAndTypesList(); - String additional_error_message; - for (auto & [name, type] : names_and_types) - { - auto [field, field_name] = getFieldByName(schema, name); - if (!checkCapnProtoType(field.getType(), type, mode, additional_error_message, field_name)) - { - auto e = Exception( - ErrorCodes::CAPN_PROTO_BAD_CAST, - "Cannot convert ClickHouse type {} to CapnProto type {}", - type->getName(), - getCapnProtoFullTypeName(field.getType())); - if (!additional_error_message.empty()) - e.addMessage(additional_error_message); - throw std::move(e); - } - } -} - -template -static DataTypePtr getEnumDataTypeFromEnumerants(const capnp::EnumSchema::EnumerantList & enumerants) -{ - std::vector> values; - for (auto enumerant : enumerants) - values.emplace_back(enumerant.getProto().getName(), ValueType(enumerant.getOrdinal())); - return std::make_shared>(std::move(values)); -} - -static DataTypePtr getEnumDataTypeFromEnumSchema(const capnp::EnumSchema & enum_schema) -{ - auto enumerants = enum_schema.getEnumerants(); - if (enumerants.size() < 128) - return getEnumDataTypeFromEnumerants(enumerants); - if (enumerants.size() < 32768) - return getEnumDataTypeFromEnumerants(enumerants); - - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "ClickHouse supports only 8 and 16-bit Enums"); -} - -static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type, bool skip_unsupported_fields) -{ - switch (capnp_type.which()) - { - case capnp::schema::Type::INT8: - return std::make_shared(); - case capnp::schema::Type::INT16: - return std::make_shared(); - case capnp::schema::Type::INT32: - return std::make_shared(); - case capnp::schema::Type::INT64: - return std::make_shared(); - case capnp::schema::Type::BOOL: [[fallthrough]]; - case capnp::schema::Type::UINT8: - return std::make_shared(); - case capnp::schema::Type::UINT16: - return std::make_shared(); - case capnp::schema::Type::UINT32: - return std::make_shared(); - case capnp::schema::Type::UINT64: - return std::make_shared(); - case capnp::schema::Type::FLOAT32: - return std::make_shared(); - case capnp::schema::Type::FLOAT64: - return std::make_shared(); - case capnp::schema::Type::DATA: [[fallthrough]]; - case capnp::schema::Type::TEXT: - return std::make_shared(); - case capnp::schema::Type::ENUM: - return getEnumDataTypeFromEnumSchema(capnp_type.asEnum()); - case capnp::schema::Type::LIST: - { - auto list_schema = capnp_type.asList(); - auto nested_type = getDataTypeFromCapnProtoType(list_schema.getElementType(), skip_unsupported_fields); - if (!nested_type) - return nullptr; - return std::make_shared(nested_type); - } - case capnp::schema::Type::STRUCT: - { - auto struct_schema = capnp_type.asStruct(); - - - if (struct_schema.getFields().size() == 0) - { - if (skip_unsupported_fields) - return nullptr; - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Empty messages are not supported"); - } - - /// Check if it can be Nullable. - if (checkIfStructIsNamedUnion(struct_schema)) - { - auto fields = struct_schema.getUnionFields(); - if (fields.size() != 2 || (!fields[0].getType().isVoid() && !fields[1].getType().isVoid())) - { - if (skip_unsupported_fields) - return nullptr; - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unions are not supported"); - } - auto value_type = fields[0].getType().isVoid() ? fields[1].getType() : fields[0].getType(); - if (value_type.isStruct() || value_type.isList()) - { - if (skip_unsupported_fields) - return nullptr; - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Tuples and Lists cannot be inside Nullable"); - } - - auto nested_type = getDataTypeFromCapnProtoType(value_type, skip_unsupported_fields); - if (!nested_type) - return nullptr; - return std::make_shared(nested_type); - } - - if (checkIfStructContainsUnnamedUnion(struct_schema)) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported"); - - /// Treat Struct as Tuple. - DataTypes nested_types; - Names nested_names; - for (auto field : struct_schema.getNonUnionFields()) - { - auto nested_type = getDataTypeFromCapnProtoType(field.getType(), skip_unsupported_fields); - if (!nested_type) - continue; - nested_names.push_back(field.getProto().getName()); - nested_types.push_back(nested_type); - } - if (nested_types.empty()) - return nullptr; - return std::make_shared(std::move(nested_types), std::move(nested_names)); - } - default: - { - if (skip_unsupported_fields) - return nullptr; - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unsupported CapnProtoType: {}", getCapnProtoFullTypeName(capnp_type)); - } - } -} - -NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema, bool skip_unsupported_fields) -{ - if (checkIfStructContainsUnnamedUnion(schema)) - throw Exception(ErrorCodes::CAPN_PROTO_BAD_TYPE, "Unnamed union is not supported"); - - NamesAndTypesList names_and_types; - for (auto field : schema.getNonUnionFields()) - { - auto name = field.getProto().getName(); - auto type = getDataTypeFromCapnProtoType(field.getType(), skip_unsupported_fields); - if (type) - names_and_types.emplace_back(name, type); - } - if (names_and_types.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Cannot convert CapnProto schema to ClickHouse table schema, all fields have unsupported types"); - - return names_and_types; -} - -} - -#endif diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index dd6252b96f1..52f9098fbcc 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -132,12 +132,14 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.protobuf.input_flatten_google_wrappers = settings.input_format_protobuf_flatten_google_wrappers; format_settings.protobuf.output_nullables_with_google_wrappers = settings.output_format_protobuf_nullables_with_google_wrappers; format_settings.protobuf.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_protobuf_skip_fields_with_unsupported_types_in_schema_inference; + format_settings.protobuf.use_autogenerated_schema = settings.format_protobuf_use_autogenerated_schema; format_settings.regexp.escaping_rule = settings.format_regexp_escaping_rule; format_settings.regexp.regexp = settings.format_regexp; format_settings.regexp.skip_unmatched = settings.format_regexp_skip_unmatched; format_settings.schema.format_schema = settings.format_schema; format_settings.schema.format_schema_path = context->getFormatSchemaPath(); format_settings.schema.is_server = context->hasGlobalContext() && (context->getGlobalContext()->getApplicationType() == Context::ApplicationType::SERVER); + format_settings.schema.output_format_schema = settings.output_format_schema; format_settings.skip_unknown_fields = settings.input_format_skip_unknown_fields; format_settings.template_settings.resultset_format = settings.format_template_resultset; format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter; @@ -178,6 +180,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.defaults_for_omitted_fields = settings.input_format_defaults_for_omitted_fields; format_settings.capn_proto.enum_comparing_mode = settings.format_capn_proto_enum_comparising_mode; format_settings.capn_proto.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_capn_proto_skip_fields_with_unsupported_types_in_schema_inference; + format_settings.capn_proto.use_autogenerated_schema = settings.format_capn_proto_use_autogenerated_schema; format_settings.seekable_read = settings.input_format_allow_seeks; format_settings.msgpack.number_of_columns = settings.input_format_msgpack_number_of_columns; format_settings.msgpack.output_uuid_representation = settings.output_format_msgpack_uuid_representation; diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index 8e2afe1b2c9..c0f0aec6fd3 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include #include @@ -105,4 +107,84 @@ FormatSchemaInfo::FormatSchemaInfo(const FormatSettings & settings, const String { } +template +MaybeAutogeneratedFormatSchemaInfo::MaybeAutogeneratedFormatSchemaInfo( + const FormatSettings & settings, const String & format, const Block & header, bool use_autogenerated_schema) +{ + if (!use_autogenerated_schema || !settings.schema.format_schema.empty()) + { + schema_info = std::make_unique(settings, format, true); + return; + } + + String schema_path; + fs::path default_schema_directory_path(fs::canonical(settings.schema.format_schema_path) / ""); + fs::path path; + if (!settings.schema.output_format_schema.empty()) + { + schema_path = settings.schema.output_format_schema; + path = schema_path; + if (path.is_absolute()) + { + if (settings.schema.is_server) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Absolute path in the 'output_format_schema' setting is prohibited: {}", path.string()); + } + else if (path.has_parent_path() && !fs::weakly_canonical(default_schema_directory_path / path).string().starts_with(fs::weakly_canonical(default_schema_directory_path).string())) + { + if (settings.schema.is_server) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: {} ({} not in {})", + default_schema_directory_path.string(), + path.string(), + default_schema_directory_path.string()); + path = default_schema_directory_path / path; + } + else + { + path = default_schema_directory_path / path; + } + } + else + { + if (settings.schema.is_server) + { + tmp_file_path = PocoTemporaryFile::tempName(default_schema_directory_path.string()) + '.' + getFormatSchemaDefaultFileExtension(format); + schema_path = fs::path(tmp_file_path).filename(); + } + else + { + tmp_file_path = PocoTemporaryFile::tempName() + '.' + getFormatSchemaDefaultFileExtension(format); + schema_path = tmp_file_path; + } + + path = tmp_file_path; + } + + WriteBufferFromFile buf(path.string()); + SchemaGenerator::writeSchema(buf, "Message", header.getNamesAndTypesList()); + buf.finalize(); + + schema_info = std::make_unique(schema_path + ":Message", format, true, settings.schema.is_server, settings.schema.format_schema_path); +} + +template +MaybeAutogeneratedFormatSchemaInfo::~MaybeAutogeneratedFormatSchemaInfo() +{ + if (!tmp_file_path.empty()) + { + try + { + fs::remove(tmp_file_path); + } + catch (...) + { + tryLogCurrentException("MaybeAutogeneratedFormatSchemaInfo", "Cannot delete temporary schema file"); + } + } +} + +template class MaybeAutogeneratedFormatSchemaInfo; +template class MaybeAutogeneratedFormatSchemaInfo; + } diff --git a/src/Formats/FormatSchemaInfo.h b/src/Formats/FormatSchemaInfo.h index 8c430218af0..e8758c3f761 100644 --- a/src/Formats/FormatSchemaInfo.h +++ b/src/Formats/FormatSchemaInfo.h @@ -2,6 +2,8 @@ #include #include +#include +#include namespace DB { @@ -30,4 +32,23 @@ private: String message_name; }; + +template +class MaybeAutogeneratedFormatSchemaInfo +{ +public: + MaybeAutogeneratedFormatSchemaInfo(const FormatSettings & settings, const String & format, const Block & header, bool use_autogenerated_schema); + + ~MaybeAutogeneratedFormatSchemaInfo(); + + const FormatSchemaInfo & getSchemaInfo() const { return *schema_info; } +private: + + std::unique_ptr schema_info; + String tmp_file_path; +}; + +using CapnProtoSchemaInfo = MaybeAutogeneratedFormatSchemaInfo; +using ProtobufSchemaInfo = MaybeAutogeneratedFormatSchemaInfo; + } diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index c88af650671..a21668f9642 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -254,6 +254,7 @@ struct FormatSettings */ bool allow_multiple_rows_without_delimiter = false; bool skip_fields_with_unsupported_types_in_schema_inference = false; + bool use_autogenerated_schema = true; } protobuf; struct @@ -275,6 +276,7 @@ struct FormatSettings std::string format_schema; std::string format_schema_path; bool is_server = false; + std::string output_format_schema; } schema; struct @@ -325,17 +327,18 @@ struct FormatSettings /// For capnProto format we should determine how to /// compare ClickHouse Enum and Enum from schema. - enum class EnumComparingMode + enum class CapnProtoEnumComparingMode { BY_NAMES, // Names in enums should be the same, values can be different. BY_NAMES_CASE_INSENSITIVE, // Case-insensitive name comparison. BY_VALUES, // Values should be the same, names can be different. }; - struct + struct CapnProto { - EnumComparingMode enum_comparing_mode = EnumComparingMode::BY_VALUES; + CapnProtoEnumComparingMode enum_comparing_mode = CapnProtoEnumComparingMode::BY_VALUES; bool skip_fields_with_unsupported_types_in_schema_inference = false; + bool use_autogenerated_schema = true; } capn_proto; enum class MsgPackUUIDRepresentation diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index f690800d145..dd37c25719c 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -3029,7 +3029,7 @@ namespace if (!message_serializer) { throw Exception(ErrorCodes::NO_COLUMNS_SERIALIZED_TO_PROTOBUF_FIELDS, - "Not found matches between the names of the columns {{}} and the fields {{}} of the message {} in the protobuf schema", + "Not found matches between the names of the columns ({}) and the fields ({}) of the message {} in the protobuf schema", boost::algorithm::join(column_names, ", "), boost::algorithm::join(getFieldNames(message_descriptor), ", "), quoteString(message_descriptor.full_name())); } @@ -3647,7 +3647,7 @@ namespace if (!message_serializer) { throw Exception(ErrorCodes::NO_COLUMNS_SERIALIZED_TO_PROTOBUF_FIELDS, - "Not found matches between the names of the tuple's elements {{}} and the fields {{}} " + "Not found matches between the names of the tuple's elements ({}) and the fields ({}) " "of the message {} in the protobuf schema", boost::algorithm::join(tuple_data_type.getElementNames(), ", "), boost::algorithm::join(getFieldNames(*field_descriptor.message_type()), ", "), diff --git a/src/Formats/StructureToCapnProtoSchema.cpp b/src/Formats/StructureToCapnProtoSchema.cpp new file mode 100644 index 00000000000..56e2155e5bd --- /dev/null +++ b/src/Formats/StructureToCapnProtoSchema.cpp @@ -0,0 +1,236 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +using namespace StructureToFormatSchemaUtils; + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +const std::unordered_map capn_proto_simple_type_names = +{ + {TypeIndex::Int8, "Int8"}, + {TypeIndex::UInt8, "UInt8"}, + {TypeIndex::Int16, "Int16"}, + {TypeIndex::UInt16, "UInt16"}, + {TypeIndex::Int32, "Int32"}, + {TypeIndex::UInt32, "UInt32"}, + {TypeIndex::Int64, "Int64"}, + {TypeIndex::UInt64, "UInt64"}, + {TypeIndex::Int128, "Data"}, + {TypeIndex::UInt128, "Data"}, + {TypeIndex::Int256, "Data"}, + {TypeIndex::UInt256, "Data"}, + {TypeIndex::Float32, "Float32"}, + {TypeIndex::Float64, "Float64"}, + {TypeIndex::Decimal32, "Int32"}, + {TypeIndex::Decimal64, "Int64"}, + {TypeIndex::Decimal128, "Data"}, + {TypeIndex::Decimal256, "Data"}, + {TypeIndex::String, "Data"}, + {TypeIndex::FixedString, "Data"}, + {TypeIndex::UUID, "Data"}, + {TypeIndex::Date, "UInt16"}, + {TypeIndex::Date32, "Int32"}, + {TypeIndex::DateTime, "UInt32"}, + {TypeIndex::DateTime64, "Int64"}, + {TypeIndex::IPv4, "UInt32"}, + {TypeIndex::IPv6, "Data"}, +}; + +void writeCapnProtoHeader(WriteBuffer & buf) +{ + pcg64 rng(randomSeed()); + size_t id = rng() | (1ull << 63); /// First bit should be 1 + writeString(fmt::format("@0x{};\n\n", getHexUIntLowercase(id)), buf); +} + +void writeFieldDefinition(WriteBuffer & buf, const String & type_name, const String & column_name, size_t & field_index, size_t indent) +{ + writeIndent(buf, indent); + writeString(fmt::format("{} @{} : {};\n", getSchemaFieldName(column_name), field_index++, type_name), buf); +} + +void startEnum(WriteBuffer & buf, const String & enum_name, size_t indent) +{ + startNested(buf, enum_name, "enum", indent); +} + +void startUnion(WriteBuffer & buf, size_t indent) +{ + startNested(buf, "", "union", indent); +} + +void startStruct(WriteBuffer & buf, const String & struct_name, size_t indent) +{ + startNested(buf, struct_name, "struct", indent); +} + +String prepareAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent); + +void writeField(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t & field_index, size_t indent) +{ + auto field_type_name = prepareAndGetCapnProtoTypeName(buf, data_type, column_name, indent); + writeFieldDefinition(buf, field_type_name, column_name, field_index, indent); +} + +String prepareArrayAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent) +{ + const auto & nested_type = assert_cast(*data_type).getNestedType(); + auto nested_type_name = prepareAndGetCapnProtoTypeName(buf, nested_type, column_name, indent); + return "List(" + nested_type_name + ")"; +} + +String prepareNullableAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent) +{ + /// Nullable is represented as a struct with union with 2 fields: + /// + /// struct Nullable + /// { + /// union + /// { + /// value @0 : Value; + /// null @1 : Void; + /// } + /// } + auto struct_name = getSchemaMessageName(column_name); + startStruct(buf, struct_name, indent); + auto nested_type_name = prepareAndGetCapnProtoTypeName(buf, assert_cast(*data_type).getNestedType(), column_name, indent); + startUnion(buf, indent + 1); + size_t field_index = 0; + writeFieldDefinition(buf, nested_type_name, "value", field_index, indent + 2); + writeFieldDefinition(buf, "Void", "null", field_index, indent + 2); + endNested(buf, indent + 1); + endNested(buf, indent); + return struct_name; +} + +String prepareTupleAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent) +{ + const auto & tuple_type = assert_cast(*data_type); + auto nested_names_and_types = getCollectedTupleElements(tuple_type); + + String struct_name = getSchemaMessageName(column_name); + startStruct(buf, struct_name, indent); + size_t nested_field_index = 0; + for (const auto & [name, type] : nested_names_and_types) + writeField(buf, type, name, nested_field_index, indent + 1); + endNested(buf, indent); + return struct_name; +} + +String prepareMapAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent) +{ + /// We output/input Map type as follow CapnProto schema + /// + /// struct Map + /// { + /// struct Entry + /// { + /// key @0: Key; + /// value @1: Value; + /// } + /// entries @0 :List(Entry); + /// } + const auto & map_type = assert_cast(*data_type); + const auto & key_type = map_type.getKeyType(); + const auto & value_type = map_type.getValueType(); + + String struct_name = getSchemaMessageName(column_name); + startStruct(buf, struct_name, indent); + startStruct(buf, "Entry", indent + 1); + auto key_type_name = prepareAndGetCapnProtoTypeName(buf, key_type, "key", indent + 2); + auto value_type_name = prepareAndGetCapnProtoTypeName(buf, value_type, "value", indent + 2); + size_t field_index = 0; + writeFieldDefinition(buf, key_type_name, "key", field_index, indent + 2); + writeFieldDefinition(buf, value_type_name, "value", field_index, indent + 2); + endNested(buf, indent + 1); + field_index = 0; + writeFieldDefinition(buf, "List(Entry)", "entries", field_index, indent + 1); + endNested(buf, indent); + return struct_name; +} + +template +String prepareEnumAndGetCapnProtoTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent) +{ + const auto & enum_type = assert_cast &>(*data_type); + String enum_name = getSchemaMessageName(column_name); + startEnum(buf, enum_name, indent); + const auto & names = enum_type.getAllRegisteredNames(); + for (size_t i = 0; i != names.size(); ++i) + { + writeIndent(buf, indent + 1); + writeString(fmt::format("{} @{};\n", names[i], std::to_string(i)), buf); + } + endNested(buf, indent); + return enum_name; +} + +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: + return prepareNullableAndGetCapnProtoTypeName(buf, data_type, column_name, indent); + case TypeIndex::LowCardinality: + return prepareAndGetCapnProtoTypeName(buf, assert_cast(*data_type).getDictionaryType(), column_name, indent); + case TypeIndex::Array: + return prepareArrayAndGetCapnProtoTypeName(buf, data_type, column_name, indent); + case TypeIndex::Tuple: + return prepareTupleAndGetCapnProtoTypeName(buf, data_type, column_name, indent); + case TypeIndex::Map: + return prepareMapAndGetCapnProtoTypeName(buf, data_type, column_name, indent); + case TypeIndex::Enum8: + return prepareEnumAndGetCapnProtoTypeName(buf, data_type, column_name, indent); + case TypeIndex::Enum16: + return prepareEnumAndGetCapnProtoTypeName(buf, data_type, column_name, indent); + default: + { + if (isBool(data_type)) + return "Bool"; + + auto it = capn_proto_simple_type_names.find(type_id); + if (it == capn_proto_simple_type_names.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "CapnProto type name is not found for type {}", data_type->getName()); + return it->second; + } + } +} + +} + +void StructureToCapnProtoSchema::writeSchema(WriteBuffer & buf, const String & message_name, const NamesAndTypesList & names_and_types_) +{ + auto names_and_types = collectNested(names_and_types_); + writeCapnProtoHeader(buf); + startStruct(buf, getSchemaMessageName(message_name), 0); + + size_t field_index = 0; + for (const auto & [column_name, data_type] : names_and_types) + writeField(buf, data_type, column_name, field_index, 1); + + endNested(buf, 0); +} + +} diff --git a/src/Formats/StructureToCapnProtoSchema.h b/src/Formats/StructureToCapnProtoSchema.h new file mode 100644 index 00000000000..b2a0a8a8cf9 --- /dev/null +++ b/src/Formats/StructureToCapnProtoSchema.h @@ -0,0 +1,16 @@ +#pragma once + +#include +#include + +namespace DB +{ + +struct StructureToCapnProtoSchema +{ + static constexpr auto name = "structureToCapnProtoSchema"; + + static void writeSchema(WriteBuffer & buf, const String & message_name, const NamesAndTypesList & names_and_types_); +}; + +} diff --git a/src/Formats/StructureToFormatSchemaUtils.cpp b/src/Formats/StructureToFormatSchemaUtils.cpp new file mode 100644 index 00000000000..02c9af2c65f --- /dev/null +++ b/src/Formats/StructureToFormatSchemaUtils.cpp @@ -0,0 +1,113 @@ +#include + +namespace DB +{ + +namespace StructureToFormatSchemaUtils +{ + +void writeIndent(WriteBuffer & buf, size_t indent) +{ + writeChar(' ', indent * 4, buf); +} + +void startNested(WriteBuffer & buf, const String & nested_name, const String & nested_type, size_t indent) +{ + writeIndent(buf, indent); + writeString(nested_type, buf); + if (!nested_name.empty()) + { + writeChar(' ', buf); + writeString(nested_name, buf); + } + writeChar('\n', buf); + writeIndent(buf, indent); + writeCString("{\n", buf); +} + +void endNested(WriteBuffer & buf, size_t indent) +{ + writeIndent(buf, indent); + writeCString("}\n", buf); +} + +String getSchemaFieldName(const String & column_name) +{ + String result = column_name; + size_t i = 0; + while (i < result.size() && isupper(result[i])) + { + result[i] = tolower(result[i]); + ++i; + } + return result; +} + +String getSchemaMessageName(const String & column_name) +{ + String result = column_name; + if (!column_name.empty() && isalpha(column_name[0])) + result[0] = toupper(column_name[0]); + return result; +} + +namespace +{ + std::pair splitName(const String & name) + { + const auto * begin = name.data(); + const auto * end = name.data() + name.size(); + const auto * it = find_first_symbols<'_', '.'>(begin, end); + String first = String(begin, it); + String second = it == end ? "" : String(it + 1, end); + return {first, second}; + } +} + +NamesAndTypesList collectNested(const NamesAndTypesList & names_and_types) +{ + /// Find all columns with dots '.' or underscores '_' and move them into a tuple. + /// For example if we have columns 'a.b UInt32, a.c UInt32, x_y String' we will + /// change it to 'a Tuple(b UInt32, c UInt32), x Tuple(y String)' + NamesAndTypesList result; + std::unordered_map nested; + for (const auto & [name, type] : names_and_types) + { + auto [field_name, nested_name] = splitName(name); + if (nested_name.empty()) + result.emplace_back(name, type); + else + nested[field_name].emplace_back(nested_name, type); + } + + for (const auto & [field_name, elements]: nested) + result.emplace_back(field_name, std::make_shared(elements.getTypes(), elements.getNames())); + + return result; +} + +NamesAndTypesList getCollectedTupleElements(const DataTypeTuple & tuple_type) +{ + const auto & nested_types = tuple_type.getElements(); + Names nested_names; + if (tuple_type.haveExplicitNames()) + { + nested_names = tuple_type.getElementNames(); + } + else + { + nested_names.reserve(nested_types.size()); + for (size_t i = 0; i != nested_types.size(); ++i) + nested_names.push_back("e" + std::to_string(i + 1)); + } + + NamesAndTypesList result; + for (size_t i = 0; i != nested_names.size(); ++i) + result.emplace_back(nested_names[i], nested_types[i]); + + return collectNested(result); +} + +} + +} diff --git a/src/Formats/StructureToFormatSchemaUtils.h b/src/Formats/StructureToFormatSchemaUtils.h new file mode 100644 index 00000000000..c6b86501ac8 --- /dev/null +++ b/src/Formats/StructureToFormatSchemaUtils.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +namespace StructureToFormatSchemaUtils +{ + void writeIndent(WriteBuffer & buf, size_t indent); + + void startNested(WriteBuffer & buf, const String & nested_name, const String & nested_type, size_t indent); + + void endNested(WriteBuffer & buf, size_t indent); + + String getSchemaFieldName(const String & column_name); + + String getSchemaMessageName(const String & column_name); + + NamesAndTypesList collectNested(const NamesAndTypesList & names_and_types); + + NamesAndTypesList getCollectedTupleElements(const DataTypeTuple & tuple_type); +} + +} diff --git a/src/Formats/StructureToProtobufSchema.cpp b/src/Formats/StructureToProtobufSchema.cpp new file mode 100644 index 00000000000..086a6ff6864 --- /dev/null +++ b/src/Formats/StructureToProtobufSchema.cpp @@ -0,0 +1,214 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +using namespace StructureToFormatSchemaUtils; + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +const std::unordered_map protobuf_simple_type_names = +{ + {TypeIndex::Int8, "int32"}, + {TypeIndex::UInt8, "uint32"}, + {TypeIndex::Int16, "int32"}, + {TypeIndex::UInt16, "uint32"}, + {TypeIndex::Int32, "int32"}, + {TypeIndex::UInt32, "uint32"}, + {TypeIndex::Int64, "int64"}, + {TypeIndex::UInt64, "uint64"}, + {TypeIndex::Int128, "bytes"}, + {TypeIndex::UInt128, "bytes"}, + {TypeIndex::Int256, "bytes"}, + {TypeIndex::UInt256, "bytes"}, + {TypeIndex::Float32, "float"}, + {TypeIndex::Float64, "double"}, + {TypeIndex::Decimal32, "bytes"}, + {TypeIndex::Decimal64, "bytes"}, + {TypeIndex::Decimal128, "bytes"}, + {TypeIndex::Decimal256, "bytes"}, + {TypeIndex::String, "bytes"}, + {TypeIndex::FixedString, "bytes"}, + {TypeIndex::UUID, "bytes"}, + {TypeIndex::Date, "uint32"}, + {TypeIndex::Date32, "int32"}, + {TypeIndex::DateTime, "uint32"}, + {TypeIndex::DateTime64, "uint64"}, + {TypeIndex::IPv4, "uint32"}, + {TypeIndex::IPv6, "bytes"}, +}; + +void writeProtobufHeader(WriteBuffer & buf) +{ + writeCString("syntax = \"proto3\";\n\n", buf); +} + +void startEnum(WriteBuffer & buf, const String & enum_name, size_t indent) +{ + startNested(buf, enum_name, "enum", indent); +} + +void startMessage(WriteBuffer & buf, const String & message_name, size_t indent) +{ + startNested(buf, message_name, "message", indent); +} + +void writeFieldDefinition(WriteBuffer & buf, const String & type_name, const String & column_name, size_t & field_index, size_t indent) +{ + writeIndent(buf, indent); + writeString(fmt::format("{} {} = {};\n", type_name, getSchemaFieldName(column_name), field_index++), buf); +} + +String prepareAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent); + +void writeProtobufField(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t & field_index, size_t indent) +{ + auto field_type_name = prepareAndGetProtobufTypeName(buf, data_type, column_name, indent); + writeFieldDefinition(buf, field_type_name, column_name, field_index, indent); +} + +String prepareArrayAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent) +{ + const auto & nested_type = assert_cast(*data_type).getNestedType(); + /// Simple case when we can just use 'repeated '. + if (!isArray(nested_type) && !isMap(nested_type)) + { + auto nested_type_name = prepareAndGetProtobufTypeName(buf, nested_type, column_name, indent); + return "repeated " + nested_type_name; + } + + /// Protobuf doesn't support multidimensional repeated fields and repeated maps. + /// When we have Array(Array(...)) or Array(Map(...)) we should place nested type into a nested Message with one field. + String message_name = getSchemaMessageName(column_name); + startMessage(buf, message_name, indent); + size_t nested_field_index = 1; + writeProtobufField(buf, nested_type, column_name, nested_field_index, indent + 1); + endNested(buf, indent); + return "repeated " + message_name; +} + +String prepareTupleAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent) +{ + const auto & tuple_type = assert_cast(*data_type); + auto nested_names_and_types = getCollectedTupleElements(tuple_type); + + String message_name = getSchemaMessageName(column_name); + startMessage(buf, message_name, indent); + size_t nested_field_index = 1; + for (const auto & [name, type] : nested_names_and_types) + writeProtobufField(buf, type, name, nested_field_index, indent + 1); + endNested(buf, indent); + return message_name; +} + +String prepareMapAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent) +{ + const auto & map_type = assert_cast(*data_type); + const auto & key_type = map_type.getKeyType(); + const auto & value_type = map_type.getValueType(); + auto it = protobuf_simple_type_names.find(key_type->getTypeId()); + if (it == protobuf_simple_type_names.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type {} is not supported for conversion into Map key in Protobuf schema", data_type->getName()); + auto key_type_name = it->second; + /// Protobuf map type doesn't support "bytes" type as a key. Change it to "string" + if (key_type_name == "bytes") + key_type_name = "string"; + + /// Special cases when value type is Array or Map, because Protobuf + /// doesn't support syntax "map" and "map>" + /// In this case we should place it into a nested Message with one field. + String value_type_name; + if (isArray(value_type) || isMap(value_type)) + { + value_type_name = getSchemaMessageName(column_name) + "Value"; + startMessage(buf, value_type_name, indent); + size_t nested_field_index = 1; + writeProtobufField(buf, value_type, column_name + "Value", nested_field_index, indent + 1); + endNested(buf, indent); + } + else + { + value_type_name = prepareAndGetProtobufTypeName(buf, value_type, column_name + "Value", indent); + } + + return fmt::format("map<{}, {}>", key_type_name, value_type_name); +} + +template +String prepareEnumAndGetProtobufTypeName(WriteBuffer & buf, const DataTypePtr & data_type, const String & column_name, size_t indent) +{ + const auto & enum_type = assert_cast &>(*data_type); + String enum_name = getSchemaMessageName(column_name); + startEnum(buf, enum_name, indent); + const auto & names = enum_type.getAllRegisteredNames(); + for (size_t i = 0; i != names.size(); ++i) + { + writeIndent(buf, indent + 1); + writeString(fmt::format("{} = {};\n", names[i], std::to_string(i)), buf); + } + endNested(buf, indent); + return enum_name; +} + +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: + return prepareAndGetProtobufTypeName(buf, assert_cast(*data_type).getNestedType(), column_name, indent); + case TypeIndex::LowCardinality: + return prepareAndGetProtobufTypeName(buf, assert_cast(*data_type).getDictionaryType(), column_name, indent); + case TypeIndex::Array: + return prepareArrayAndGetProtobufTypeName(buf, data_type, column_name, indent); + case TypeIndex::Tuple: + return prepareTupleAndGetProtobufTypeName(buf, data_type, column_name, indent); + case TypeIndex::Map: + return prepareMapAndGetProtobufTypeName(buf, data_type, column_name, indent); + case TypeIndex::Enum8: + return prepareEnumAndGetProtobufTypeName(buf, data_type, column_name, indent); + case TypeIndex::Enum16: + return prepareEnumAndGetProtobufTypeName(buf, data_type, column_name, indent); + default: + { + if (isBool(data_type)) + return "bool"; + + auto it = protobuf_simple_type_names.find(type_id); + if (it == protobuf_simple_type_names.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type {} is not supported for conversion into Protobuf schema", data_type->getName()); + return it->second; + } + } +} + +} + +void StructureToProtobufSchema::writeSchema(WriteBuffer & buf, const String & message_name, const NamesAndTypesList & names_and_types_) +{ + auto names_and_types = collectNested(names_and_types_); + writeProtobufHeader(buf); + startMessage(buf, getSchemaMessageName(message_name), 0); + size_t field_index = 1; + for (const auto & [column_name, data_type] : names_and_types) + writeProtobufField(buf, data_type, column_name, field_index, 1); + endNested(buf, 0); +} + +} diff --git a/src/Formats/StructureToProtobufSchema.h b/src/Formats/StructureToProtobufSchema.h new file mode 100644 index 00000000000..f4dfb0ae0c2 --- /dev/null +++ b/src/Formats/StructureToProtobufSchema.h @@ -0,0 +1,16 @@ +#pragma once + +#include +#include + +namespace DB +{ + +struct StructureToProtobufSchema +{ + static constexpr auto name = "structureToProtobufSchema"; + + static void writeSchema(WriteBuffer & buf, const String & message_name, const NamesAndTypesList & names_and_types_); +}; + +} diff --git a/src/Functions/generateRandomStructure.cpp b/src/Functions/generateRandomStructure.cpp new file mode 100644 index 00000000000..5f2153ff89f --- /dev/null +++ b/src/Functions/generateRandomStructure.cpp @@ -0,0 +1,507 @@ +#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 new file mode 100644 index 00000000000..332db44e06e --- /dev/null +++ b/src/Functions/structureToFormatSchema.cpp @@ -0,0 +1,137 @@ +#include +#include +#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; + extern const int LOGICAL_ERROR; +} + +template +class FunctionStructureToFormatSchema : public IFunction +{ +public: + + static constexpr auto name = Impl::name; + FunctionStructureToFormatSchema(ContextPtr context_) : context(std::move(context_)) + { + } + + static FunctionPtr create(ContextPtr ctx) + { + return std::make_shared(std::move(ctx)); + } + + 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; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 1}; } + bool useDefaultImplementationForConstants() const override { return false; } + bool useDefaultImplementationForNulls() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.empty() || arguments.size() > 2) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, expected 1 or 2", + getName(), arguments.size()); + + if (!isString(arguments[0])) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the first argument of function {}, expected constant string", + arguments[0]->getName(), + getName()); + } + + if (arguments.size() > 1 && !isString(arguments[1])) + { + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of the second argument of function {}, expected constant string", + arguments[1]->getName(), + getName()); + } + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + if (arguments.empty() || arguments.size() > 2) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, expected 1 or 2", + getName(), arguments.size()); + + String structure = arguments[0].column->getDataAt(0).toString(); + String message_name = arguments.size() == 2 ? arguments[1].column->getDataAt(0).toString() : "Message"; + auto columns_list = parseColumnsListFromString(structure, context); + auto col_res = ColumnString::create(); + auto & data = assert_cast(*col_res).getChars(); + WriteBufferFromVector buf(data); + Impl::writeSchema(buf, message_name, columns_list.getAll()); + buf.finalize(); + auto & offsets = assert_cast(*col_res).getOffsets(); + offsets.push_back(data.size()); + return ColumnConst::create(std::move(col_res), input_rows_count); + } + +private: + ContextPtr context; +}; + + +REGISTER_FUNCTION(StructureToCapnProtoSchema) +{ + factory.registerFunction>( + { + R"( + +)", + Documentation::Examples{ + {"random", "SELECT structureToCapnProtoSchema()"}, + }, + Documentation::Categories{"Other"} + }, + FunctionFactory::CaseSensitive); +} + + +REGISTER_FUNCTION(StructureToProtobufSchema) +{ + factory.registerFunction>( + { + R"( + +)", + Documentation::Examples{ + {"random", "SELECT structureToCapnProtoSchema()"}, + }, + Documentation::Categories{"Other"} + }, + FunctionFactory::CaseSensitive); +} + +} diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index 2f84e9bde3c..ce7c48d1f37 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -9,23 +9,6 @@ #include #include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include - namespace DB { @@ -35,16 +18,14 @@ namespace ErrorCodes extern const int INCORRECT_DATA; } -CapnProtoRowInputFormat::CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params_, const FormatSchemaInfo & info, const FormatSettings & format_settings_) - : IRowInputFormat(std::move(header), in_, std::move(params_)) +CapnProtoRowInputFormat::CapnProtoRowInputFormat(ReadBuffer & in_, Block header_, Params params_, const CapnProtoSchemaInfo & info, const FormatSettings & format_settings) + : IRowInputFormat(std::move(header_), in_, std::move(params_)) , parser(std::make_shared()) - , format_settings(format_settings_) - , column_types(getPort().getHeader().getDataTypes()) - , column_names(getPort().getHeader().getNames()) { // Parse the schema and fetch the root object - root = parser->getMessageSchema(info); - checkCapnProtoSchemaStructure(root, getPort().getHeader(), format_settings.capn_proto.enum_comparing_mode); + schema = parser->getMessageSchema(info.getSchemaInfo()); + const auto & header = getPort().getHeader(); + serializer = std::make_unique(header.getDataTypes(), header.getNames(), schema, format_settings.capn_proto); } kj::Array CapnProtoRowInputFormat::readMessage() @@ -82,213 +63,6 @@ kj::Array CapnProtoRowInputFormat::readMessage() return msg; } -static void insertInteger(IColumn & column, const DataTypePtr & column_type, UInt64 value) -{ - switch (column_type->getTypeId()) - { - case TypeIndex::Int8: - assert_cast(column).insertValue(value); - break; - case TypeIndex::UInt8: - assert_cast(column).insertValue(value); - break; - case TypeIndex::Int16: - assert_cast(column).insertValue(value); - break; - case TypeIndex::Date: [[fallthrough]]; - case TypeIndex::UInt16: - assert_cast(column).insertValue(value); - break; - case TypeIndex::Int32: - assert_cast(column).insertValue(static_cast(value)); - break; - case TypeIndex::DateTime: [[fallthrough]]; - case TypeIndex::UInt32: - assert_cast(column).insertValue(static_cast(value)); - break; - case TypeIndex::IPv4: - assert_cast(column).insertValue(IPv4(static_cast(value))); - break; - case TypeIndex::Int64: - assert_cast(column).insertValue(value); - break; - case TypeIndex::UInt64: - assert_cast(column).insertValue(value); - break; - case TypeIndex::DateTime64: - assert_cast &>(column).insertValue(value); - break; - case TypeIndex::Decimal32: - assert_cast &>(column).insertValue(static_cast(value)); - break; - case TypeIndex::Decimal64: - assert_cast &>(column).insertValue(value); - break; - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Column type {} cannot be parsed from integer", column_type->getName()); - } -} - -static void insertFloat(IColumn & column, const DataTypePtr & column_type, Float64 value) -{ - switch (column_type->getTypeId()) - { - case TypeIndex::Float32: - assert_cast(column).insertValue(static_cast(value)); - break; - case TypeIndex::Float64: - assert_cast(column).insertValue(value); - break; - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Column type is not a float."); - } -} - -template -static void insertData(IColumn & column, const DataTypePtr & column_type, Value value) -{ - if (column_type->haveMaximumSizeOfValue() && value.size() != column_type->getSizeOfValueInMemory()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected size of {} value: {}", column_type->getName(), value.size()); - - column.insertData(reinterpret_cast(value.begin()), value.size()); -} - -template -static void insertEnum(IColumn & column, const DataTypePtr & column_type, const capnp::DynamicEnum & enum_value, FormatSettings::EnumComparingMode enum_comparing_mode) -{ - auto enumerant = *kj::_::readMaybe(enum_value.getEnumerant()); - auto enum_type = assert_cast *>(column_type.get()); - DataTypePtr nested_type = std::make_shared>(); - switch (enum_comparing_mode) - { - case FormatSettings::EnumComparingMode::BY_VALUES: - insertInteger(column, nested_type, Int64(enumerant.getOrdinal())); - return; - case FormatSettings::EnumComparingMode::BY_NAMES: - insertInteger(column, nested_type, Int64(enum_type->getValue(String(enumerant.getProto().getName())))); - return; - case FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE: - { - /// Find the same enum name case insensitive. - String enum_name = enumerant.getProto().getName(); - for (auto & name : enum_type->getAllRegisteredNames()) - { - if (compareEnumNames(name, enum_name, enum_comparing_mode)) - { - insertInteger(column, nested_type, Int64(enum_type->getValue(name))); - break; - } - } - } - } -} - -static void insertValue(IColumn & column, const DataTypePtr & column_type, const String & column_name, const capnp::DynamicValue::Reader & value, FormatSettings::EnumComparingMode enum_comparing_mode) -{ - if (column_type->lowCardinality()) - { - auto & lc_column = assert_cast(column); - auto tmp_column = lc_column.getDictionary().getNestedColumn()->cloneEmpty(); - auto dict_type = assert_cast(column_type.get())->getDictionaryType(); - insertValue(*tmp_column, dict_type, column_name, value, enum_comparing_mode); - lc_column.insertFromFullColumn(*tmp_column, 0); - return; - } - - switch (value.getType()) - { - case capnp::DynamicValue::Type::INT: - insertInteger(column, column_type, value.as()); - break; - case capnp::DynamicValue::Type::UINT: - insertInteger(column, column_type, value.as()); - break; - case capnp::DynamicValue::Type::FLOAT: - insertFloat(column, column_type, value.as()); - break; - case capnp::DynamicValue::Type::BOOL: - insertInteger(column, column_type, UInt64(value.as())); - break; - case capnp::DynamicValue::Type::DATA: - insertData(column, column_type, value.as()); - break; - case capnp::DynamicValue::Type::TEXT: - insertData(column, column_type, value.as()); - break; - case capnp::DynamicValue::Type::ENUM: - if (column_type->getTypeId() == TypeIndex::Enum8) - insertEnum(column, column_type, value.as(), enum_comparing_mode); - else - insertEnum(column, column_type, value.as(), enum_comparing_mode); - break; - case capnp::DynamicValue::LIST: - { - auto list_value = value.as(); - auto & column_array = assert_cast(column); - auto & offsets = column_array.getOffsets(); - offsets.push_back(offsets.back() + list_value.size()); - - auto & nested_column = column_array.getData(); - auto nested_type = assert_cast(column_type.get())->getNestedType(); - for (const auto & nested_value : list_value) - insertValue(nested_column, nested_type, column_name, nested_value, enum_comparing_mode); - break; - } - case capnp::DynamicValue::Type::STRUCT: - { - auto struct_value = value.as(); - if (column_type->isNullable()) - { - auto & nullable_column = assert_cast(column); - auto field = *kj::_::readMaybe(struct_value.which()); - if (field.getType().isVoid()) - nullable_column.insertDefault(); - else - { - auto & nested_column = nullable_column.getNestedColumn(); - auto nested_type = assert_cast(column_type.get())->getNestedType(); - auto nested_value = struct_value.get(field); - insertValue(nested_column, nested_type, column_name, nested_value, enum_comparing_mode); - nullable_column.getNullMapData().push_back(0); - } - } - else if (isTuple(column_type)) - { - auto & tuple_column = assert_cast(column); - const auto * tuple_type = assert_cast(column_type.get()); - bool have_explicit_names = tuple_type->haveExplicitNames(); - auto struct_schema = struct_value.getSchema(); - for (uint32_t i = 0; i != tuple_column.tupleSize(); ++i) - insertValue( - tuple_column.getColumn(i), - tuple_type->getElements()[i], - tuple_type->getElementNames()[i], - struct_value.get(have_explicit_names ? struct_schema.getFieldByName(tuple_type->getElementNames()[i]) : struct_schema.getFields()[i]), - enum_comparing_mode); - } - else if (isMap(column_type)) - { - const auto & map_type = assert_cast(*column_type); - DataTypes key_value_types = {map_type.getKeyType(), map_type.getValueType()}; - Names key_value_names = {"key", "value"}; - auto entries_type = std::make_shared(std::make_shared(key_value_types, key_value_names)); - auto & entries_column = assert_cast(column).getNestedColumn(); - auto entries_field = struct_value.getSchema().getFields()[0]; - insertValue(entries_column, entries_type, column_name, struct_value.get(entries_field), enum_comparing_mode); - } - else - { - /// It can be nested column from Nested type. - auto [field_name, nested_name] = splitCapnProtoFieldName(column_name); - insertValue(column, column_type, nested_name, struct_value.get(nested_name), enum_comparing_mode); - } - break; - } - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CapnProto value type."); - } -} - bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { if (in->eof()) @@ -298,12 +72,8 @@ bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension { auto array = readMessage(); capnp::FlatArrayMessageReader msg(array); - auto root_reader = msg.getRoot(root); - for (size_t i = 0; i != columns.size(); ++i) - { - auto value = getReaderByColumnName(root_reader, column_names[i]); - insertValue(*columns[i], column_types[i], column_names[i], value, format_settings.capn_proto.enum_comparing_mode); - } + auto root_reader = msg.getRoot(schema); + serializer->readRow(columns, root_reader); } catch (const kj::Exception & e) { @@ -337,8 +107,12 @@ void registerInputFormatCapnProto(FormatFactory & factory) "CapnProto", [](ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(buf, sample, std::move(params), - FormatSchemaInfo(settings, "CapnProto", true), settings); + return std::make_shared( + buf, + sample, + std::move(params), + CapnProtoSchemaInfo(settings, "CapnProto", sample, settings.capn_proto.use_autogenerated_schema), + settings); }); factory.markFormatSupportsSubsetOfColumns("CapnProto"); factory.registerFileExtension("capnp", "CapnProto"); diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h index cf23f22b643..88f91b5eb11 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h @@ -4,7 +4,8 @@ #if USE_CAPNP #include -#include +#include +#include #include #include @@ -23,7 +24,7 @@ class ReadBuffer; class CapnProtoRowInputFormat final : public IRowInputFormat { public: - CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params_, const FormatSchemaInfo & info, const FormatSettings & format_settings_); + CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params_, const CapnProtoSchemaInfo & info, const FormatSettings & format_settings); String getName() const override { return "CapnProtoRowInputFormat"; } @@ -33,10 +34,9 @@ private: kj::Array readMessage(); std::shared_ptr parser; - capnp::StructSchema root; - const FormatSettings format_settings; - DataTypes column_types; - Names column_names; + capnp::StructSchema schema; + std::unique_ptr serializer; + }; class CapnProtoSchemaReader : public IExternalSchemaReader diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp index 0225680b396..0919c93ed28 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp @@ -1,28 +1,13 @@ #include #if USE_CAPNP -#include +#include #include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include - namespace DB { @@ -44,252 +29,24 @@ void CapnProtoOutputStream::write(const void * buffer, size_t size) CapnProtoRowOutputFormat::CapnProtoRowOutputFormat( WriteBuffer & out_, const Block & header_, - const FormatSchemaInfo & info, - const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_), column_names(header_.getNames()), column_types(header_.getDataTypes()), output_stream(std::make_unique(out_)), format_settings(format_settings_) + const CapnProtoSchemaInfo & info, + const FormatSettings & format_settings) + : IRowOutputFormat(header_, out_) + , column_names(header_.getNames()) + , column_types(header_.getDataTypes()) + , output_stream(std::make_unique(out_)) { - schema = schema_parser.getMessageSchema(info); - checkCapnProtoSchemaStructure(schema, getPort(PortKind::Main).getHeader(), format_settings.capn_proto.enum_comparing_mode); -} - -template -static capnp::DynamicEnum getDynamicEnum( - const ColumnPtr & column, - const DataTypePtr & data_type, - size_t row_num, - const capnp::EnumSchema & enum_schema, - FormatSettings::EnumComparingMode mode) -{ - const auto * enum_data_type = assert_cast *>(data_type.get()); - EnumValue enum_value = column->getInt(row_num); - if (mode == FormatSettings::EnumComparingMode::BY_VALUES) - return capnp::DynamicEnum(enum_schema, enum_value); - - auto enum_name = enum_data_type->getNameForValue(enum_value); - for (const auto enumerant : enum_schema.getEnumerants()) - { - if (compareEnumNames(String(enum_name), enumerant.getProto().getName(), mode)) - return capnp::DynamicEnum(enumerant); - } - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert CLickHouse Enum value to CapnProto Enum"); -} - -static capnp::DynamicValue::Builder initStructFieldBuilder(const ColumnPtr & column, size_t row_num, capnp::DynamicStruct::Builder & struct_builder, capnp::StructSchema::Field field) -{ - if (const auto * array_column = checkAndGetColumn(*column)) - { - size_t size = array_column->getOffsets()[row_num] - array_column->getOffsets()[row_num - 1]; - return struct_builder.init(field, static_cast(size)); - } - - if (field.getType().isStruct()) - return struct_builder.init(field); - - return struct_builder.get(field); -} - -static std::optional convertToDynamicValue( - const ColumnPtr & column, - const DataTypePtr & data_type, - size_t row_num, - const String & column_name, - capnp::DynamicValue::Builder builder, - FormatSettings::EnumComparingMode enum_comparing_mode, - std::vector> & temporary_text_data_storage) -{ - /// Here we don't do any types validation, because we did it in CapnProtoRowOutputFormat constructor. - - if (data_type->lowCardinality()) - { - const auto * lc_column = assert_cast(column.get()); - const auto & dict_type = assert_cast(data_type.get())->getDictionaryType(); - size_t index = lc_column->getIndexAt(row_num); - return convertToDynamicValue(lc_column->getDictionary().getNestedColumn(), dict_type, index, column_name, builder, enum_comparing_mode, temporary_text_data_storage); - } - - switch (builder.getType()) - { - case capnp::DynamicValue::Type::INT: - return capnp::DynamicValue::Reader(column->getInt(row_num)); - case capnp::DynamicValue::Type::UINT: - { - /// IPv4 column doesn't support getUInt method. - if (isIPv4(data_type)) - return capnp::DynamicValue::Reader(assert_cast(column.get())->getElement(row_num)); - return capnp::DynamicValue::Reader(column->getUInt(row_num)); - } - case capnp::DynamicValue::Type::BOOL: - return capnp::DynamicValue::Reader(column->getBool(row_num)); - case capnp::DynamicValue::Type::FLOAT: - return capnp::DynamicValue::Reader(column->getFloat64(row_num)); - case capnp::DynamicValue::Type::ENUM: - { - auto enum_schema = builder.as().getSchema(); - if (data_type->getTypeId() == TypeIndex::Enum8) - return capnp::DynamicValue::Reader( - getDynamicEnum(column, data_type, row_num, enum_schema, enum_comparing_mode)); - return capnp::DynamicValue::Reader( - getDynamicEnum(column, data_type, row_num, enum_schema, enum_comparing_mode)); - } - case capnp::DynamicValue::Type::DATA: - { - auto data = column->getDataAt(row_num); - return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); - } - case capnp::DynamicValue::Type::TEXT: - { - /// In TEXT type data should be null-terminated, but ClickHouse String data could not be. - /// To make data null-terminated we should copy it to temporary String object, but - /// capnp::Text::Reader works only with pointer to the data and it's size, so we should - /// guarantee that new String object life time is longer than capnp::Text::Reader life time. - /// To do this we store new String object in a temporary storage, passed in this function - /// by reference. We use unique_ptr instead of just String to avoid pointers - /// invalidation on vector reallocation. - temporary_text_data_storage.push_back(std::make_unique(column->getDataAt(row_num))); - auto & data = temporary_text_data_storage.back(); - return capnp::DynamicValue::Reader(capnp::Text::Reader(data->data(), data->size())); - } - case capnp::DynamicValue::Type::STRUCT: - { - auto struct_builder = builder.as(); - auto nested_struct_schema = struct_builder.getSchema(); - /// Struct can represent Tuple, Nullable (named union with two fields) or single column when it contains one nested column. - if (data_type->isNullable()) - { - const auto * nullable_type = assert_cast(data_type.get()); - const auto * nullable_column = assert_cast(column.get()); - auto fields = nested_struct_schema.getUnionFields(); - if (nullable_column->isNullAt(row_num)) - { - auto null_field = fields[0].getType().isVoid() ? fields[0] : fields[1]; - struct_builder.set(null_field, capnp::Void()); - } - else - { - auto value_field = fields[0].getType().isVoid() ? fields[1] : fields[0]; - struct_builder.clear(value_field); - const auto & nested_column = nullable_column->getNestedColumnPtr(); - auto value_builder = initStructFieldBuilder(nested_column, row_num, struct_builder, value_field); - auto value = convertToDynamicValue(nested_column, nullable_type->getNestedType(), row_num, column_name, value_builder, enum_comparing_mode, temporary_text_data_storage); - if (value) - struct_builder.set(value_field, *value); - } - } - else if (isTuple(data_type)) - { - const auto * tuple_data_type = assert_cast(data_type.get()); - const auto & nested_types = tuple_data_type->getElements(); - const auto & nested_names = tuple_data_type->getElementNames(); - const auto & nested_columns = assert_cast(column.get())->getColumns(); - bool have_explicit_names = tuple_data_type->haveExplicitNames(); - for (uint32_t i = 0; i != nested_names.size(); ++i) - { - capnp::StructSchema::Field nested_field = have_explicit_names ? nested_struct_schema.getFieldByName(nested_names[i]) : nested_struct_schema.getFields()[i]; - auto field_builder = initStructFieldBuilder(nested_columns[i], row_num, struct_builder, nested_field); - auto value = convertToDynamicValue(nested_columns[i], nested_types[i], row_num, nested_names[i], field_builder, enum_comparing_mode, temporary_text_data_storage); - if (value) - struct_builder.set(nested_field, *value); - } - } - else if (isMap(data_type)) - { - /// We output Map type as follow CapnProto schema - /// - /// struct Map { - /// struct Entry { - /// key @0: Key; - /// value @1: Value; - /// } - /// entries @0 :List(Entry); - /// } - /// - /// And we don't need to check that struct have this form here because we checked it before. - const auto & map_type = assert_cast(*data_type); - DataTypes key_value_types = {map_type.getKeyType(), map_type.getValueType()}; - Names key_value_names = {"key", "value"}; - auto entries_type = std::make_shared(std::make_shared(key_value_types, key_value_names)); - - /// Nested column in Map is actually Array(Tuple), so we can output it according to "entries" field schema. - const auto & entries_column = assert_cast(column.get())->getNestedColumnPtr(); - - auto entries_field = nested_struct_schema.getFields()[0]; - auto field_builder = initStructFieldBuilder(entries_column, row_num, struct_builder, entries_field); - auto entries_value = convertToDynamicValue(entries_column, entries_type, row_num, column_name, field_builder, enum_comparing_mode, temporary_text_data_storage); - if (entries_value) - struct_builder.set(entries_field, *entries_value); - } - else - { - /// It can be nested column from Nested type. - auto [field_name, nested_name] = splitCapnProtoFieldName(column_name); - auto nested_field = nested_struct_schema.getFieldByName(nested_name); - auto field_builder = initStructFieldBuilder(column, row_num, struct_builder, nested_field); - auto value = convertToDynamicValue(column, data_type, row_num, nested_name, field_builder, enum_comparing_mode, temporary_text_data_storage); - if (value) - struct_builder.set(nested_field, *value); - } - return std::nullopt; - } - case capnp::DynamicValue::Type::LIST: - { - auto list_builder = builder.as(); - const auto * array_column = assert_cast(column.get()); - const auto & nested_column = array_column->getDataPtr(); - const auto & nested_type = assert_cast(data_type.get())->getNestedType(); - const auto & offsets = array_column->getOffsets(); - auto offset = offsets[row_num - 1]; - size_t size = offsets[row_num] - offset; - - const auto * nested_array_column = checkAndGetColumn(*nested_column); - for (unsigned i = 0; i != static_cast(size); ++i) - { - capnp::DynamicValue::Builder value_builder; - /// For nested arrays we need to initialize nested list builder. - if (nested_array_column) - { - const auto & nested_offset = nested_array_column->getOffsets(); - size_t nested_array_size = nested_offset[offset + i] - nested_offset[offset + i - 1]; - value_builder = list_builder.init(i, static_cast(nested_array_size)); - } - else - value_builder = list_builder[i]; - - auto value = convertToDynamicValue(nested_column, nested_type, offset + i, column_name, value_builder, enum_comparing_mode, temporary_text_data_storage); - if (value) - list_builder.set(i, *value); - } - return std::nullopt; - } - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CapnProto type."); - } + schema = schema_parser.getMessageSchema(info.getSchemaInfo()); + const auto & header = getPort(PortKind::Main).getHeader(); + serializer = std::make_unique(header.getDataTypes(), header.getNames(), schema, format_settings.capn_proto); + capnp::MallocMessageBuilder message; } void CapnProtoRowOutputFormat::write(const Columns & columns, size_t row_num) { capnp::MallocMessageBuilder message; - /// Temporary storage for data that will be outputted in fields with CapnProto type TEXT. - /// See comment in convertToDynamicValue() for more details. - std::vector> temporary_text_data_storage; capnp::DynamicStruct::Builder root = message.initRoot(schema); - - /// Some columns can share same field builder. For example when we have - /// column with Nested type that was flattened into several columns. - std::unordered_map field_builders; - for (size_t i = 0; i != columns.size(); ++i) - { - auto [struct_builder, field] = getStructBuilderAndFieldByColumnName(root, column_names[i]); - if (!field_builders.contains(field.getIndex())) - { - auto field_builder = initStructFieldBuilder(columns[i], row_num, struct_builder, field); - field_builders[field.getIndex()] = field_builder; - } - auto value = convertToDynamicValue(columns[i], column_types[i], row_num, column_names[i], field_builders[field.getIndex()], format_settings.capn_proto.enum_comparing_mode, temporary_text_data_storage); - if (value) - struct_builder.set(field, *value); - } - + serializer->writeRow(columns, std::move(root), row_num); capnp::writeMessage(*output_stream, message); } @@ -300,7 +57,11 @@ void registerOutputFormatCapnProto(FormatFactory & factory) const Block & sample, const FormatSettings & format_settings) { - return std::make_shared(buf, sample, FormatSchemaInfo(format_settings, "CapnProto", true), format_settings); + return std::make_shared( + buf, + sample, + CapnProtoSchemaInfo(format_settings, "CapnProto", sample, format_settings.capn_proto.use_autogenerated_schema), + format_settings); }); } diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h index 5cc7099d4c7..c00dceb498e 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h @@ -3,15 +3,17 @@ #include "config.h" #if USE_CAPNP -#include -#include -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include +# include namespace DB { + class CapnProtoOutputStream : public kj::OutputStream { public: @@ -29,8 +31,8 @@ public: CapnProtoRowOutputFormat( WriteBuffer & out_, const Block & header_, - const FormatSchemaInfo & info, - const FormatSettings & format_settings_); + const CapnProtoSchemaInfo & info, + const FormatSettings & format_settings); String getName() const override { return "CapnProtoRowOutputFormat"; } @@ -43,8 +45,9 @@ private: DataTypes column_types; capnp::StructSchema schema; std::unique_ptr output_stream; - const FormatSettings format_settings; CapnProtoSchemaParser schema_parser; + std::unique_ptr serializer; + }; } diff --git a/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp index 9777f2361a2..8dc5e6fd5d1 100644 --- a/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufListInputFormat.cpp @@ -14,7 +14,7 @@ ProtobufListInputFormat::ProtobufListInputFormat( ReadBuffer & in_, const Block & header_, const Params & params_, - const FormatSchemaInfo & schema_info_, + const ProtobufSchemaInfo & schema_info_, bool flatten_google_wrappers_) : IRowInputFormat(header_, in_, params_) , reader(std::make_unique(in_)) @@ -22,7 +22,7 @@ ProtobufListInputFormat::ProtobufListInputFormat( header_.getNames(), header_.getDataTypes(), missing_column_indices, - *ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::Yes), + *ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::Yes), /* with_length_delimiter = */ true, /* with_envelope = */ true, flatten_google_wrappers_, @@ -84,7 +84,7 @@ void registerInputFormatProtobufList(FormatFactory & factory) const FormatSettings & settings) { return std::make_shared(buf, sample, std::move(params), - FormatSchemaInfo(settings, "Protobuf", true), settings.protobuf.input_flatten_google_wrappers); + ProtobufSchemaInfo(settings, "Protobuf", sample, settings.protobuf.use_autogenerated_schema), settings.protobuf.input_flatten_google_wrappers); }); factory.markFormatSupportsSubsetOfColumns("ProtobufList"); factory.registerAdditionalInfoForSchemaCacheGetter( diff --git a/src/Processors/Formats/Impl/ProtobufListInputFormat.h b/src/Processors/Formats/Impl/ProtobufListInputFormat.h index ba2e8014878..09eabb0ee59 100644 --- a/src/Processors/Formats/Impl/ProtobufListInputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufListInputFormat.h @@ -28,7 +28,7 @@ public: ReadBuffer & in_, const Block & header_, const Params & params_, - const FormatSchemaInfo & schema_info_, + const ProtobufSchemaInfo & schema_info_, bool flatten_google_wrappers_); String getName() const override { return "ProtobufListInputFormat"; } diff --git a/src/Processors/Formats/Impl/ProtobufListOutputFormat.cpp b/src/Processors/Formats/Impl/ProtobufListOutputFormat.cpp index 815b9ebb61d..ae0b9db7357 100644 --- a/src/Processors/Formats/Impl/ProtobufListOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufListOutputFormat.cpp @@ -2,7 +2,6 @@ #if USE_PROTOBUF # include -# include # include # include # include @@ -13,14 +12,14 @@ namespace DB ProtobufListOutputFormat::ProtobufListOutputFormat( WriteBuffer & out_, const Block & header_, - const FormatSchemaInfo & schema_info_, + const ProtobufSchemaInfo & schema_info_, bool defaults_for_nullable_google_wrappers_) : IRowOutputFormat(header_, out_) , writer(std::make_unique(out)) , serializer(ProtobufSerializer::create( header_.getNames(), header_.getDataTypes(), - *ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::Yes), + *ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::Yes), /* with_length_delimiter = */ true, /* with_envelope = */ true, defaults_for_nullable_google_wrappers_, @@ -55,7 +54,7 @@ void registerOutputFormatProtobufList(FormatFactory & factory) const FormatSettings & settings) { return std::make_shared( - buf, header, FormatSchemaInfo(settings, "Protobuf", true), + buf, header, ProtobufSchemaInfo(settings, "Protobuf", header, settings.protobuf.use_autogenerated_schema), settings.protobuf.output_nullables_with_google_wrappers); }); } diff --git a/src/Processors/Formats/Impl/ProtobufListOutputFormat.h b/src/Processors/Formats/Impl/ProtobufListOutputFormat.h index d85018c0351..e7765590d51 100644 --- a/src/Processors/Formats/Impl/ProtobufListOutputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufListOutputFormat.h @@ -4,10 +4,10 @@ #if USE_PROTOBUF # include +# include namespace DB { -class FormatSchemaInfo; class ProtobufWriter; class ProtobufSerializer; @@ -26,7 +26,7 @@ public: ProtobufListOutputFormat( WriteBuffer & out_, const Block & header_, - const FormatSchemaInfo & schema_info_, + const ProtobufSchemaInfo & schema_info_, bool defaults_for_nullable_google_wrappers_); String getName() const override { return "ProtobufListOutputFormat"; } diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index ee60501dba5..94d81fc34c8 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -11,9 +11,9 @@ namespace DB { ProtobufRowInputFormat::ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, const Params & params_, - const FormatSchemaInfo & schema_info_, bool with_length_delimiter_, bool flatten_google_wrappers_) + const ProtobufSchemaInfo & schema_info_, bool with_length_delimiter_, bool flatten_google_wrappers_) : IRowInputFormat(header_, in_, params_) - , message_descriptor(ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::No)) + , message_descriptor(ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::No)) , with_length_delimiter(with_length_delimiter_) , flatten_google_wrappers(flatten_google_wrappers_) { @@ -89,7 +89,7 @@ void registerInputFormatProtobuf(FormatFactory & factory) const FormatSettings & settings) { return std::make_shared(buf, sample, std::move(params), - FormatSchemaInfo(settings, "Protobuf", true), + ProtobufSchemaInfo(settings, "Protobuf", sample, settings.protobuf.use_autogenerated_schema), with_length_delimiter, settings.protobuf.input_flatten_google_wrappers); }); diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h index 5c042f7c5ab..9d7cecd7146 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h @@ -33,7 +33,7 @@ public: ReadBuffer & in_, const Block & header_, const Params & params_, - const FormatSchemaInfo & schema_info_, + const ProtobufSchemaInfo & schema_info_, bool with_length_delimiter_, bool flatten_google_wrappers_); diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp index 129c9ca3156..7b4cc1bf0be 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp @@ -3,7 +3,6 @@ #if USE_PROTOBUF # include # include -# include # include # include # include @@ -20,7 +19,7 @@ namespace ErrorCodes ProtobufRowOutputFormat::ProtobufRowOutputFormat( WriteBuffer & out_, const Block & header_, - const FormatSchemaInfo & schema_info_, + const ProtobufSchemaInfo & schema_info_, const FormatSettings & settings_, bool with_length_delimiter_) : IRowOutputFormat(header_, out_) @@ -28,7 +27,7 @@ ProtobufRowOutputFormat::ProtobufRowOutputFormat( , serializer(ProtobufSerializer::create( header_.getNames(), header_.getDataTypes(), - *ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_, ProtobufSchemas::WithEnvelope::No), + *ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info_.getSchemaInfo(), ProtobufSchemas::WithEnvelope::No), with_length_delimiter_, /* with_envelope = */ false, settings_.protobuf.output_nullables_with_google_wrappers, @@ -61,7 +60,7 @@ void registerOutputFormatProtobuf(FormatFactory & factory) const FormatSettings & settings) { return std::make_shared( - buf, header, FormatSchemaInfo(settings, "Protobuf", true), + buf, header, ProtobufSchemaInfo(settings, "Protobuf", header, settings.protobuf.use_autogenerated_schema), settings, with_length_delimiter); }); } diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h index f6ff5bae999..213e1c785fd 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h @@ -4,11 +4,11 @@ #if USE_PROTOBUF # include +# include namespace DB { class DB; -class FormatSchemaInfo; class ProtobufSerializer; class ProtobufWriter; class WriteBuffer; @@ -30,7 +30,7 @@ public: ProtobufRowOutputFormat( WriteBuffer & out_, const Block & header_, - const FormatSchemaInfo & schema_info_, + const ProtobufSchemaInfo & schema_info_, const FormatSettings & settings_, bool with_length_delimiter_); diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index 567c674ded9..e48d3187cb2 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -470,7 +470,7 @@ class GenerateSource : public ISource { public: GenerateSource(UInt64 block_size_, UInt64 max_array_length_, UInt64 max_string_length_, UInt64 random_seed_, Block block_header_, ContextPtr context_) - : ISource(Nested::flatten(prepareBlockToFill(block_header_))) + : ISource(Nested::flattenArrayOfTuples(prepareBlockToFill(block_header_))) , block_size(block_size_), max_array_length(max_array_length_), max_string_length(max_string_length_) , block_to_fill(std::move(block_header_)), rng(random_seed_), context(context_) {} @@ -485,7 +485,7 @@ protected: for (const auto & elem : block_to_fill) columns.emplace_back(fillColumnWithRandomData(elem.type, block_size, max_array_length, max_string_length, rng, context)); - columns = Nested::flatten(block_to_fill.cloneWithColumns(columns)).getColumns(); + columns = Nested::flattenArrayOfTuples(block_to_fill.cloneWithColumns(columns)).getColumns(); return {std::move(columns), block_size}; } diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index 5f1a13d8857..12cbda334a3 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include "registerTableFunctions.h" @@ -28,7 +29,7 @@ namespace ErrorCodes extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } -void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, ContextPtr /*context*/) +void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, ContextPtr context) { ASTs & args_func = ast_function->children; @@ -45,6 +46,9 @@ void TableFunctionGenerateRandom::parseArguments(const ASTPtr & ast_function, Co "Table function '{}' requires at most four arguments: " " structure, [random_seed, max_string_length, max_array_length].", getName()); + /// 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 c8cc0cddd30..5b81aa72914 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -306,6 +306,7 @@ ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) bool TableFunctionS3::supportsReadingSubsetOfColumns() { + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); } From 56c73cab0d4105c41606d1d036cba4b9f89735fe Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 26 Jun 2023 13:30:22 +0200 Subject: [PATCH 078/554] Fix azure test fails and updated to work without create privileges --- src/Storages/StorageAzureBlob.cpp | 68 +++++++++++++++++-------------- 1 file changed, 38 insertions(+), 30 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 4eb60db0b99..6971094556b 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -316,16 +316,19 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co if (configuration.is_connection_string) { result = std::make_unique(BlobContainerClient::CreateFromConnectionString(configuration.connection_url, configuration.container)); - result->CreateIfNotExists(); } else { std::shared_ptr storage_shared_key_credential; - std::unique_ptr blob_service_client; if (configuration.account_name.has_value() && configuration.account_key.has_value()) { storage_shared_key_credential = std::make_shared(*configuration.account_name, *configuration.account_key); + } + + std::unique_ptr blob_service_client; + if (storage_shared_key_credential) + { blob_service_client = std::make_unique(configuration.connection_url, storage_shared_key_credential); } else @@ -333,40 +336,45 @@ 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; - try + 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) { - blob_service_client = std::make_unique(configuration.connection_url); - 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) + if (configuration.container == container.Name) { - 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; + container_exists = true; + break; + } + } - if (storage_shared_key_credential) - result = std::make_unique(final_url, storage_shared_key_credential); - else - result = std::make_unique(final_url); + 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 - { - throw; - } + 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 + result = std::make_unique(final_url); + } + else + { + result = std::make_unique(blob_service_client->CreateBlobContainer(configuration.container).Value); } } From 8dafd66c8dca5892b3dbdd1387b4c3e0b46e572b Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 26 Jun 2023 16:35:16 +0200 Subject: [PATCH 079/554] Updated creating container using connection string --- src/Storages/StorageAzureBlob.cpp | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 6971094556b..8222940ce53 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -315,7 +315,31 @@ AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration co 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)); + + if (!container_exists) + { + result->CreateIfNotExists(); + } } else { From 81b5ad3b26822c1b22a71ee38202ccf07be88f8f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 27 Jun 2023 16:51:17 +0200 Subject: [PATCH 080/554] Fixed else if syntax error --- 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 d9d068a5974..b3a18bce39d 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -1209,7 +1209,7 @@ ColumnsDescription StorageAzureBlob::getTableStructureFromData( file_iterator = std::make_shared(ctx , ctx->getReadTaskCallback()); } - else (configuration.withGlobs()) + else if (configuration.withGlobs()) { file_iterator = std::make_shared( object_storage, configuration.container, configuration.blob_path, nullptr, Block{}, ctx, &read_keys); From 223b95420542fd8b57c820e97ebbe9acc2ccdb8f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 28 Jun 2023 11:09:19 +0200 Subject: [PATCH 081/554] Fixed glob iterator for table function cluster path without regex characters --- src/Storages/StorageAzureBlob.cpp | 5 ++++- src/Storages/StorageAzureBlobCluster.cpp | 6 +++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index b3a18bce39d..7fcc8ea930c 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -832,6 +832,7 @@ StorageAzureBlobSource::GlobIterator::GlobIterator( blobs_with_metadata.emplace_back(blob_path_with_globs, object_metadata); if (outer_blobs) outer_blobs->emplace_back(blobs_with_metadata.back()); + is_finished = true; return; } @@ -850,8 +851,10 @@ RelativePathWithMetadata StorageAzureBlobSource::GlobIterator::next() { std::lock_guard lock(next_mutex); - if (is_finished) + if (is_finished && index >= blobs_with_metadata.size()) + { return {}; + } bool need_new_batch = blobs_with_metadata.empty() || index >= blobs_with_metadata.size(); diff --git a/src/Storages/StorageAzureBlobCluster.cpp b/src/Storages/StorageAzureBlobCluster.cpp index cfdad5c9e59..1a3f48731a7 100644 --- a/src/Storages/StorageAzureBlobCluster.cpp +++ b/src/Storages/StorageAzureBlobCluster.cpp @@ -81,9 +81,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, std::nullopt, - configuration.blob_path, query, virtual_block, context, nullptr); + auto iterator = std::make_shared( + object_storage.get(), configuration.container, 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) }; } From 349083e673770df6dde3c109be73d63cca263188 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 30 Jun 2023 13:54:57 +0200 Subject: [PATCH 082/554] Update ReadBufferFromHDFS.cpp --- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 904c1c89c16..23218e4c1f4 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -64,7 +64,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory Date: Fri, 30 Jun 2023 14:38:59 +0200 Subject: [PATCH 083/554] Add documentation for building in docker --- docs/en/development/build.md | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 83a4550df88..a1508c669a3 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -13,6 +13,20 @@ Supported platforms: - AArch64 - Power9 (experimental) +## Building in docker +We use the docker image `clickhouse/binary-builder` for our CI builds. It contains everything necessary to build the binary and packages. There is a script `docker/packager/packager` to ease the image usage: + +```bash +# define a directory for the output artifacts +output_dir="build_results" +# a simplest build +./docker/packager/packager --package-type=binary --output-dir "$output_dir" +# build debian packages +./docker/packager/packager --package-type=deb --output-dir "$output_dir" +# by default, debian packages use thin LTO, so we can override it to speed up the build +CMAKE_FLAGS='-DENABLE_THINLTO=' ./docker/packager/packager --package-type=deb --output-dir "./$(git rev-parse --show-cdup)/build_results" +``` + ## Building on Ubuntu The following tutorial is based on Ubuntu Linux. From 64d1f32eddae6ae2973b782bf01d58997a7f54b6 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 30 Jun 2023 16:33:07 +0000 Subject: [PATCH 084/554] Not sending any result --- src/Functions/array/arrayIntersect.cpp | 99 ++++++++++++++++++++++---- 1 file changed, 84 insertions(+), 15 deletions(-) diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index d1bbd169513..d25db9fc10e 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -18,6 +18,7 @@ #include #include #include +#include "Common/Exception.h" #include #include #include @@ -564,29 +565,97 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable 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); + // 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); - } - } + // if (all_nullable) + // null_map.push_back(0); + // } + // } result_offsets.getElement(row) = result_offset; } + for (size_t row = 0; row < rows; ++row) + { + for (size_t arg_num = 0; arg_num < 1; ++arg_num) + { + 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]; + + prev_off[arg_num] = off; + if (arg.is_const) + prev_off[arg_num] = 0; + for (size_t res_num = 0; res_num < result_offset; ++res_num) + { + typename Map::LookupResult pair; + + if constexpr (is_numeric_column) + { + pair = map.find(columns[arg_num]->getElement(res_num)); + } + else if constexpr (std::is_same_v || std::is_same_v) + pair = map.find(columns[arg_num]->getDataAt(res_num)); + else + { + const char * data = nullptr; + pair = map.find(columns[arg_num]->serializeValueIntoArena(res_num, arena, data)); + } + + if (pair->getMapped() == args)//for (const auto & pair : map) + { + if constexpr (is_numeric_column) + { + if (pair->getKey() == columns[arg_num]->getElement(res_num)) + { + ++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)) + { + ++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)) + { + ++result_offset; + result_data.deserializeAndInsertFromArena(pair->getKey().data); + } + } + if (all_nullable) + null_map.push_back(0); + } + } + + + } + } 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)); + + } From 2aa2f39e368ea9ece3c52623261c735d97a0ecca Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 3 Jul 2023 11:14:56 +0000 Subject: [PATCH 085/554] Not completed 'problem with offsets if the second array is smaller' --- src/Functions/array/arrayIntersect.cpp | 60 +++++++++++++++++--------- 1 file changed, 40 insertions(+), 20 deletions(-) diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index d25db9fc10e..9d0021782ac 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -565,29 +565,31 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable 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); + 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; + // 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; const auto & arg = arrays.args[arg_num]; size_t off; // const array has only one row @@ -599,8 +601,12 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable 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) { + if (arg.null_map && (*arg.null_map)[row]) + current_has_nullable = true; + typename Map::LookupResult pair; if constexpr (is_numeric_column) @@ -615,13 +621,20 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable pair = map.find(columns[arg_num]->serializeValueIntoArena(res_num, 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->getMapped() == args)//for (const auto & pair : map) { if constexpr (is_numeric_column) { if (pair->getKey() == columns[arg_num]->getElement(res_num)) { - ++result_offset; + // ++result_offset; result_data.insertValue(pair->getKey()); } } @@ -629,7 +642,7 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable { if (pair->getKey() == columns[arg_num]->getDataAt(res_num)) { - ++result_offset; + // ++result_offset; result_data.insertData(pair->getKey().data, pair->getKey().size); } } @@ -638,14 +651,21 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable const char * data = nullptr; if (pair->getKey() == columns[arg_num]->serializeValueIntoArena(res_num, arena, data)) { - ++result_offset; + // ++result_offset; result_data.deserializeAndInsertFromArena(pair->getKey().data); } } if (all_nullable) null_map.push_back(0); } - } + } + if (all_has_nullable) + { + ++result_offset; + result_data.insertDefault(); + null_map.push_back(1); + } + result_offsets.getElement(row) = result_offset; } From 469da1831ab6cfbbf79903cdcb07a85fea11c7bd Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 3 Jul 2023 17:28:48 +0200 Subject: [PATCH 086/554] Replace `--build-type=debug` by `--debug-build` --- docker/packager/README.md | 2 +- docker/packager/packager | 16 ++++++++-------- tests/ci/build_check.py | 6 +++--- tests/ci/build_report_check.py | 4 ++-- tests/ci/ci_config.py | 34 +++++++++++++++++----------------- tests/ci/report.py | 6 +++--- 6 files changed, 34 insertions(+), 34 deletions(-) diff --git a/docker/packager/README.md b/docker/packager/README.md index a78feb8d7fc..3a91f9a63f0 100644 --- a/docker/packager/README.md +++ b/docker/packager/README.md @@ -6,7 +6,7 @@ Usage: Build deb package with `clang-14` in `debug` mode: ``` $ mkdir deb/test_output -$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=clang-14 --build-type=debug +$ ./packager --output-dir deb/test_output/ --package-type deb --compiler=clang-14 --debug-build $ ls -l deb/test_output -rw-r--r-- 1 root root 3730 clickhouse-client_22.2.2+debug_all.deb -rw-r--r-- 1 root root 84221888 clickhouse-common-static_22.2.2+debug_amd64.deb diff --git a/docker/packager/packager b/docker/packager/packager index 1b3df858cd2..3c3304165b3 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -112,12 +112,12 @@ def run_docker_image_with_env( subprocess.check_call(cmd, shell=True) -def is_release_build(build_type: str, package_type: str, sanitizer: str) -> bool: - return build_type == "" and package_type == "deb" and sanitizer == "" +def is_release_build(debug_build: bool, package_type: str, sanitizer: str) -> bool: + return not debug_build and package_type == "deb" and sanitizer == "" def parse_env_variables( - build_type: str, + debug_build: bool, compiler: str, sanitizer: str, package_type: str, @@ -233,7 +233,7 @@ def parse_env_variables( build_target = ( f"{build_target} clickhouse-odbc-bridge clickhouse-library-bridge" ) - if is_release_build(build_type, package_type, sanitizer): + if is_release_build(debug_build, package_type, sanitizer): cmake_flags.append("-DSPLIT_DEBUG_SYMBOLS=ON") result.append("WITH_PERFORMANCE=1") if is_cross_arm: @@ -253,8 +253,8 @@ def parse_env_variables( if sanitizer: result.append(f"SANITIZER={sanitizer}") - if build_type: - result.append(f"BUILD_TYPE={build_type.capitalize()}") + if debug_build: + result.append("BUILD_TYPE=DEBUG") else: result.append("BUILD_TYPE=None") @@ -359,7 +359,7 @@ def parse_args() -> argparse.Namespace: help="ClickHouse git repository", ) parser.add_argument("--output-dir", type=dir_name, required=True) - parser.add_argument("--build-type", choices=("debug", ""), default="") + parser.add_argument("--debug-build", action="store_true") parser.add_argument( "--compiler", @@ -464,7 +464,7 @@ def main(): build_image(image_with_version, dockerfile) env_prepared = parse_env_variables( - args.build_type, + args.debug_build, args.compiler, args.sanitizer, args.package_type, diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 35b98a7c3bb..2a636faf967 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -45,7 +45,7 @@ def _can_export_binaries(build_config: BuildConfig) -> bool: return False if build_config["sanitizer"] != "": return True - if build_config["build_type"] != "": + if build_config["debug_build"]: return True return False @@ -66,8 +66,8 @@ def get_packager_cmd( f"--package-type={package_type} --compiler={comp}" ) - if build_config["build_type"]: - cmd += f" --build-type={build_config['build_type']}" + if build_config["debug_build"]: + cmd += " --debug-build" if build_config["sanitizer"]: cmd += f" --sanitizer={build_config['sanitizer']}" if build_config["tidy"] == "enable": diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 1362f3c8934..295b6cf9740 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -70,7 +70,7 @@ def get_failed_report( message = f"{job_name} failed" build_result = BuildResult( compiler="unknown", - build_type="unknown", + debug_build=False, sanitizer="unknown", status=message, elapsed_seconds=0, @@ -85,7 +85,7 @@ def process_report( build_config = build_report["build_config"] build_result = BuildResult( compiler=build_config["compiler"], - build_type=build_config["build_type"], + debug_build=build_config["debug_build"], sanitizer=build_config["sanitizer"], status="success" if build_report["status"] else "failure", elapsed_seconds=build_report["elapsed_seconds"], diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index c680b5810fc..875c5a3c8bd 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -10,7 +10,7 @@ CI_CONFIG = { "build_config": { "package_release": { "compiler": "clang-16", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "deb", "static_binary_name": "amd64", @@ -21,7 +21,7 @@ CI_CONFIG = { }, "coverity": { "compiler": "clang-16", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "coverity", "tidy": "disable", @@ -31,7 +31,7 @@ CI_CONFIG = { }, "package_aarch64": { "compiler": "clang-16-aarch64", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "deb", "static_binary_name": "aarch64", @@ -42,7 +42,7 @@ CI_CONFIG = { }, "package_asan": { "compiler": "clang-16", - "build_type": "", + "debug_build": False, "sanitizer": "address", "package_type": "deb", "tidy": "disable", @@ -51,7 +51,7 @@ CI_CONFIG = { }, "package_ubsan": { "compiler": "clang-16", - "build_type": "", + "debug_build": False, "sanitizer": "undefined", "package_type": "deb", "tidy": "disable", @@ -60,7 +60,7 @@ CI_CONFIG = { }, "package_tsan": { "compiler": "clang-16", - "build_type": "", + "debug_build": False, "sanitizer": "thread", "package_type": "deb", "tidy": "disable", @@ -69,7 +69,7 @@ CI_CONFIG = { }, "package_msan": { "compiler": "clang-16", - "build_type": "", + "debug_build": False, "sanitizer": "memory", "package_type": "deb", "tidy": "disable", @@ -78,7 +78,7 @@ CI_CONFIG = { }, "package_debug": { "compiler": "clang-16", - "build_type": "debug", + "debug_build": True, "sanitizer": "", "package_type": "deb", "tidy": "disable", @@ -87,7 +87,7 @@ CI_CONFIG = { }, "binary_release": { "compiler": "clang-16", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "tidy": "disable", @@ -96,7 +96,7 @@ CI_CONFIG = { }, "binary_tidy": { "compiler": "clang-16", - "build_type": "debug", + "debug_build": True, "sanitizer": "", "package_type": "binary", "static_binary_name": "debug-amd64", @@ -106,7 +106,7 @@ CI_CONFIG = { }, "binary_darwin": { "compiler": "clang-16-darwin", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "static_binary_name": "macos", @@ -116,7 +116,7 @@ CI_CONFIG = { }, "binary_aarch64": { "compiler": "clang-16-aarch64", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "tidy": "disable", @@ -125,7 +125,7 @@ CI_CONFIG = { }, "binary_aarch64_v80compat": { "compiler": "clang-16-aarch64-v80compat", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "static_binary_name": "aarch64v80compat", @@ -135,7 +135,7 @@ CI_CONFIG = { }, "binary_freebsd": { "compiler": "clang-16-freebsd", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "static_binary_name": "freebsd", @@ -145,7 +145,7 @@ CI_CONFIG = { }, "binary_darwin_aarch64": { "compiler": "clang-16-darwin-aarch64", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "static_binary_name": "macos-aarch64", @@ -155,7 +155,7 @@ CI_CONFIG = { }, "binary_ppc64le": { "compiler": "clang-16-ppc64le", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "static_binary_name": "powerpc64le", @@ -165,7 +165,7 @@ CI_CONFIG = { }, "binary_amd64_compat": { "compiler": "clang-16-amd64-compat", - "build_type": "", + "debug_build": False, "sanitizer": "", "package_type": "binary", "static_binary_name": "amd64compat", diff --git a/tests/ci/report.py b/tests/ci/report.py index a9014acec12..0f84fbcaeb2 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -239,7 +239,7 @@ def read_test_results(results_path: Path, with_raw_logs: bool = True) -> TestRes @dataclass class BuildResult: compiler: str - build_type: str + debug_build: bool sanitizer: str status: str elapsed_seconds: int @@ -484,8 +484,8 @@ def create_build_html_report( ): row = "" row += f"{build_result.compiler}" - if build_result.build_type: - row += f"{build_result.build_type}" + if build_result.debug_build: + row += "debug" else: row += "relwithdebuginfo" if build_result.sanitizer: From 3a4cb8cb67e0feb523e52adf2d9aff389afe2419 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 4 Jul 2023 00:21:53 +0300 Subject: [PATCH 087/554] Update ThreadStatus.cpp --- src/Common/ThreadStatus.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index a4d1004e44a..2e37885646a 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -126,7 +126,6 @@ ThreadStatus::ThreadStatus() ThreadGroupPtr ThreadStatus::getThreadGroup() const { chassert(current_thread == this); - memory_tracker.adjustWithUntrackedMemory(untracked_memory); return thread_group; } From 3277e4b5bd95c0495a48f54663411fe0320330b7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 4 Jul 2023 11:22:19 +0000 Subject: [PATCH 088/554] Fix build type in packager --- docker/packager/packager | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/packager b/docker/packager/packager index 3c3304165b3..c908094f0f0 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -254,7 +254,7 @@ def parse_env_variables( if sanitizer: result.append(f"SANITIZER={sanitizer}") if debug_build: - result.append("BUILD_TYPE=DEBUG") + result.append("BUILD_TYPE=Debug") else: result.append("BUILD_TYPE=None") From 98aa6b317f6324c58d814025d543d7616a1bf7ee Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 4 Jul 2023 16:50:31 +0000 Subject: [PATCH 089/554] Support reading subcolumns from file/s3/hdfs/url/azureBlobStorage table functions --- src/Core/Settings.h | 8 +- src/Core/SettingsQuirks.cpp | 6 + src/Formats/FormatFactory.cpp | 18 --- src/Formats/FormatFactory.h | 2 - src/Formats/FormatSettings.h | 3 - .../Formats/Impl/ArrowBlockInputFormat.cpp | 2 - .../Formats/Impl/ArrowColumnToCHColumn.cpp | 60 ++++--- .../Formats/Impl/ArrowColumnToCHColumn.h | 2 - .../Formats/Impl/AvroRowInputFormat.cpp | 2 + .../Formats/Impl/ORCBlockInputFormat.cpp | 7 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 2 - .../Impl/ParquetMetadataInputFormat.cpp | 1 - .../Transforms/ExtractColumnsTransform.cpp | 35 +++++ .../Transforms/ExtractColumnsTransform.h | 26 +++ .../DataLakes/DeltaLakeMetadataParser.cpp | 1 - src/Storages/HDFS/StorageHDFS.cpp | 71 +++------ src/Storages/HDFS/StorageHDFS.h | 14 +- src/Storages/HDFS/StorageHDFSCluster.h | 2 + src/Storages/StorageAzureBlob.cpp | 73 ++------- src/Storages/StorageAzureBlob.h | 12 +- src/Storages/StorageFile.cpp | 148 +++++++----------- src/Storages/StorageFile.h | 2 + src/Storages/StorageS3.cpp | 70 ++------- src/Storages/StorageS3.h | 12 +- src/Storages/StorageS3Cluster.h | 2 + src/Storages/StorageURL.cpp | 78 ++++----- src/Storages/StorageURL.h | 14 +- src/Storages/StorageURLCluster.h | 2 + src/Storages/prepareReadingFromFormat.cpp | 69 ++++++++ src/Storages/prepareReadingFromFormat.h | 26 +++ .../test_storage_azure_blob_storage/test.py | 37 +++++ tests/integration/test_storage_hdfs/test.py | 48 ++++++ tests/integration/test_storage_s3/test.py | 63 ++++++++ ...02797_read_subcolumns_from_files.reference | 4 + .../02797_read_subcolumns_from_files.sh | 18 +++ 35 files changed, 541 insertions(+), 399 deletions(-) create mode 100644 src/Processors/Transforms/ExtractColumnsTransform.cpp create mode 100644 src/Processors/Transforms/ExtractColumnsTransform.h create mode 100644 src/Storages/prepareReadingFromFormat.cpp create mode 100644 src/Storages/prepareReadingFromFormat.h create mode 100644 tests/queries/0_stateless/02797_read_subcolumns_from_files.reference create mode 100755 tests/queries/0_stateless/02797_read_subcolumns_from_files.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3d42bd582ed..710a6e4f135 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -825,8 +825,11 @@ 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, input_format_arrow_import_nested, false) \ + MAKE_OBSOLETE(M, Bool, input_format_parquet_import_nested, false) \ + MAKE_OBSOLETE(M, Bool, input_format_orc_import_nested, false) \ - /** The section above is for obsolete settings. Do not add anything there. */ +/** The section above is for obsolete settings. Do not add anything there. */ #define FORMAT_FACTORY_SETTINGS(M, ALIAS) \ @@ -845,12 +848,9 @@ class IColumn; 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) \ - M(Bool, input_format_arrow_import_nested, false, "Allow to insert array of structs into Nested table in Arrow input format.", 0) \ M(Bool, input_format_arrow_case_insensitive_column_matching, false, "Ignore case when matching Arrow columns with CH columns.", 0) \ - M(Bool, input_format_orc_import_nested, false, "Allow to insert array of structs into Nested table in ORC input format.", 0) \ M(Int64, input_format_orc_row_batch_size, 100'000, "Batch size when reading ORC stripes.", 0) \ M(Bool, input_format_orc_case_insensitive_column_matching, false, "Ignore case when matching ORC columns with CH columns.", 0) \ - M(Bool, input_format_parquet_import_nested, false, "Allow to insert array of structs into Nested table in Parquet input format.", 0) \ M(Bool, input_format_parquet_case_insensitive_column_matching, false, "Ignore case when matching Parquet columns with CH columns.", 0) \ M(Bool, input_format_parquet_preserve_order, false, "Avoid reordering rows when reading from Parquet files. Usually makes it much slower.", 0) \ M(Bool, input_format_allow_seeks, true, "Allow seeks while reading in ORC/Parquet/Arrow input formats", 0) \ diff --git a/src/Core/SettingsQuirks.cpp b/src/Core/SettingsQuirks.cpp index 3326f42adf5..cee9c3d497c 100644 --- a/src/Core/SettingsQuirks.cpp +++ b/src/Core/SettingsQuirks.cpp @@ -71,6 +71,12 @@ void applySettingsQuirks(Settings & settings, Poco::Logger * log) } } +//#if defined(THREAD_SANITIZER) + settings.use_hedged_requests.value = false; + if (log) + LOG_WARNING(log, "use_hedged_requests has been disabled for the build with Thread Sanitizer, because they are using fibers, leading to a failed assertion inside TSan"); +//#endif + if (!queryProfilerWorks()) { if (settings.query_profiler_real_time_period_ns) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index ad991aa0335..6e1e574e687 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -117,7 +117,6 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.parquet.row_group_rows = settings.output_format_parquet_row_group_size; format_settings.parquet.row_group_bytes = settings.output_format_parquet_row_group_size_bytes; format_settings.parquet.output_version = settings.output_format_parquet_version; - format_settings.parquet.import_nested = settings.input_format_parquet_import_nested; format_settings.parquet.case_insensitive_column_matching = settings.input_format_parquet_case_insensitive_column_matching; format_settings.parquet.preserve_order = settings.input_format_parquet_preserve_order; format_settings.parquet.allow_missing_columns = settings.input_format_parquet_allow_missing_columns; @@ -161,7 +160,6 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.with_types_use_header = settings.input_format_with_types_use_header; format_settings.write_statistics = settings.output_format_write_statistics; format_settings.arrow.low_cardinality_as_dictionary = settings.output_format_arrow_low_cardinality_as_dictionary; - format_settings.arrow.import_nested = settings.input_format_arrow_import_nested; format_settings.arrow.allow_missing_columns = settings.input_format_arrow_allow_missing_columns; format_settings.arrow.skip_columns_with_unsupported_types_in_schema_inference = settings.input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference; format_settings.arrow.skip_columns_with_unsupported_types_in_schema_inference = settings.input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference; @@ -169,11 +167,9 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.arrow.output_string_as_string = settings.output_format_arrow_string_as_string; format_settings.arrow.output_fixed_string_as_fixed_byte_array = settings.output_format_arrow_fixed_string_as_fixed_byte_array; format_settings.arrow.output_compression_method = settings.output_format_arrow_compression_method; - format_settings.orc.import_nested = settings.input_format_orc_import_nested; format_settings.orc.allow_missing_columns = settings.input_format_orc_allow_missing_columns; format_settings.orc.row_batch_size = settings.input_format_orc_row_batch_size; format_settings.orc.skip_columns_with_unsupported_types_in_schema_inference = settings.input_format_orc_skip_columns_with_unsupported_types_in_schema_inference; - format_settings.orc.import_nested = settings.input_format_orc_import_nested; format_settings.orc.allow_missing_columns = settings.input_format_orc_allow_missing_columns; format_settings.orc.row_batch_size = settings.input_format_orc_row_batch_size; format_settings.orc.skip_columns_with_unsupported_types_in_schema_inference = settings.input_format_orc_skip_columns_with_unsupported_types_in_schema_inference; @@ -676,14 +672,6 @@ void FormatFactory::markFormatSupportsSubsetOfColumns(const String & name) target = true; } -void FormatFactory::markFormatSupportsSubcolumns(const String & name) -{ - auto & target = dict[name].supports_subcolumns; - if (target) - throw Exception(ErrorCodes::LOGICAL_ERROR, "FormatFactory: Format {} is already marked as supporting subcolumns", name); - target = true; -} - void FormatFactory::markOutputFormatPrefersLargeBlocks(const String & name) { auto & target = dict[name].prefers_large_blocks; @@ -692,12 +680,6 @@ void FormatFactory::markOutputFormatPrefersLargeBlocks(const String & name) target = true; } -bool FormatFactory::checkIfFormatSupportsSubcolumns(const String & name) const -{ - const auto & target = getCreators(name); - return target.supports_subcolumns; -} - bool FormatFactory::checkIfFormatSupportsSubsetOfColumns(const String & name) const { const auto & target = getCreators(name); diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 489db944ee6..fc4ab6d4893 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -228,10 +228,8 @@ public: void markOutputFormatSupportsParallelFormatting(const String & name); void markOutputFormatPrefersLargeBlocks(const String & name); - void markFormatSupportsSubcolumns(const String & name); void markFormatSupportsSubsetOfColumns(const String & name); - bool checkIfFormatSupportsSubcolumns(const String & name) const; bool checkIfFormatSupportsSubsetOfColumns(const String & name) const; bool checkIfFormatHasSchemaReader(const String & name) const; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 07d4a7ede4a..6402df33bd9 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -100,7 +100,6 @@ struct FormatSettings { UInt64 row_group_size = 1000000; bool low_cardinality_as_dictionary = false; - bool import_nested = false; bool allow_missing_columns = false; bool skip_columns_with_unsupported_types_in_schema_inference = false; bool case_insensitive_column_matching = false; @@ -212,7 +211,6 @@ struct FormatSettings { UInt64 row_group_rows = 1000000; UInt64 row_group_bytes = 512 * 1024 * 1024; - bool import_nested = false; bool allow_missing_columns = false; bool skip_columns_with_unsupported_types_in_schema_inference = false; bool case_insensitive_column_matching = false; @@ -317,7 +315,6 @@ struct FormatSettings struct { - bool import_nested = false; bool allow_missing_columns = false; int64_t row_batch_size = 100'000; bool skip_columns_with_unsupported_types_in_schema_inference = false; diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index 2fadc09e80f..4293407379e 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -143,7 +143,6 @@ void ArrowBlockInputFormat::prepareReader() arrow_column_to_ch_column = std::make_unique( getPort().getHeader(), "Arrow", - format_settings.arrow.import_nested, format_settings.arrow.allow_missing_columns, format_settings.null_as_default, format_settings.arrow.case_insensitive_column_matching); @@ -190,7 +189,6 @@ void registerInputFormatArrow(FormatFactory & factory) { return std::make_shared(buf, sample, false, format_settings); }); - factory.markFormatSupportsSubcolumns("Arrow"); factory.markFormatSupportsSubsetOfColumns("Arrow"); factory.registerInputFormat( "ArrowStream", diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 5a7306111a5..74d4553a58b 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -1032,13 +1032,11 @@ Block ArrowColumnToCHColumn::arrowSchemaToCHHeader( ArrowColumnToCHColumn::ArrowColumnToCHColumn( const Block & header_, const std::string & format_name_, - bool import_nested_, bool allow_missing_columns_, bool null_as_default_, bool case_insensitive_matching_) : header(header_) , format_name(format_name_) - , import_nested(import_nested_) , allow_missing_columns(allow_missing_columns_) , null_as_default(null_as_default_) , case_insensitive_matching(case_insensitive_matching_) @@ -1080,42 +1078,40 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & if (!name_to_column_ptr.contains(search_column_name)) { bool read_from_nested = false; - /// Check if it's a column from nested table. - if (import_nested) + /// Check if it's a subcolumn from some struct. + String nested_table_name = Nested::extractTableName(header_column.name); + String search_nested_table_name = nested_table_name; + if (case_insensitive_matching) + boost::to_lower(search_nested_table_name); + if (name_to_column_ptr.contains(search_nested_table_name)) { - String nested_table_name = Nested::extractTableName(header_column.name); - String search_nested_table_name = nested_table_name; - if (case_insensitive_matching) - boost::to_lower(search_nested_table_name); - if (name_to_column_ptr.contains(search_nested_table_name)) + if (!nested_tables.contains(search_nested_table_name)) { - if (!nested_tables.contains(search_nested_table_name)) + NamesAndTypesList nested_columns; + for (const auto & name_and_type : header.getNamesAndTypesList()) { - NamesAndTypesList nested_columns; - for (const auto & name_and_type : header.getNamesAndTypesList()) - { - if (name_and_type.name.starts_with(nested_table_name + ".")) - nested_columns.push_back(name_and_type); - } - auto nested_table_type = Nested::collect(nested_columns).front().type; + if (name_and_type.name.starts_with(nested_table_name + ".")) + nested_columns.push_back(name_and_type); + } + auto nested_table_type = Nested::collect(nested_columns).front().type; - std::shared_ptr arrow_column = name_to_column_ptr[search_nested_table_name]; - ColumnsWithTypeAndName cols = {readColumnFromArrowColumn( - arrow_column, nested_table_name, format_name, false, dictionary_infos, true, false, skipped, nested_table_type)}; - BlockPtr block_ptr = std::make_shared(cols); - auto column_extractor = std::make_shared(*block_ptr, case_insensitive_matching); - nested_tables[search_nested_table_name] = {block_ptr, column_extractor}; - } - auto nested_column = nested_tables[search_nested_table_name].second->extractColumn(search_column_name); - if (nested_column) - { - column = *nested_column; - if (case_insensitive_matching) - column.name = header_column.name; - read_from_nested = true; - } + std::shared_ptr arrow_column = name_to_column_ptr[search_nested_table_name]; + ColumnsWithTypeAndName cols = {readColumnFromArrowColumn( + arrow_column, nested_table_name, format_name, false, dictionary_infos, true, false, skipped, nested_table_type)}; + BlockPtr block_ptr = std::make_shared(cols); + auto column_extractor = std::make_shared(*block_ptr, case_insensitive_matching); + nested_tables[search_nested_table_name] = {block_ptr, column_extractor}; + } + auto nested_column = nested_tables[search_nested_table_name].second->extractColumn(search_column_name); + if (nested_column) + { + column = *nested_column; + if (case_insensitive_matching) + column.name = header_column.name; + read_from_nested = true; } } + if (!read_from_nested) { if (!allow_missing_columns) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 64ff99c70ac..57f33069e0e 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -24,7 +24,6 @@ public: ArrowColumnToCHColumn( const Block & header_, const std::string & format_name_, - bool import_nested_, bool allow_missing_columns_, bool null_as_default_, bool case_insensitive_matching_ = false); @@ -53,7 +52,6 @@ public: private: const Block & header; const std::string format_name; - bool import_nested; /// If false, throw exception if some columns in header not exists in arrow table. bool allow_missing_columns; bool null_as_default; diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 1ec7491658e..74108beb370 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -1223,6 +1223,8 @@ void registerInputFormatAvro(FormatFactory & factory) { return std::make_shared(sample, buf, params, settings); }); + + factory.markFormatSupportsSubsetOfColumns("AvroConfluent"); } void registerAvroSchemaReader(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 016f07731d5..ab4e07376f3 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -125,16 +125,12 @@ void ORCBlockInputFormat::prepareReader() arrow_column_to_ch_column = std::make_unique( getPort().getHeader(), "ORC", - format_settings.orc.import_nested, format_settings.orc.allow_missing_columns, format_settings.null_as_default, format_settings.orc.case_insensitive_column_matching); const bool ignore_case = format_settings.orc.case_insensitive_column_matching; - std::unordered_set nested_table_names; - if (format_settings.orc.import_nested) - nested_table_names = Nested::getAllTableNames(getPort().getHeader(), ignore_case); - + std::unordered_set nested_table_names = Nested::getAllTableNames(getPort().getHeader(), ignore_case); for (int i = 0; i < schema->num_fields(); ++i) { const auto & name = schema->field(i)->name(); @@ -171,7 +167,6 @@ void registerInputFormatORC(FormatFactory & factory) { return std::make_shared(buf, sample, settings); }); - factory.markFormatSupportsSubcolumns("ORC"); factory.markFormatSupportsSubsetOfColumns("ORC"); } diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 3dde8ad6a6c..d2df5e5dc8e 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -143,7 +143,6 @@ void ParquetBlockInputFormat::initializeRowGroupReader(size_t row_group_idx) row_group.arrow_column_to_ch_column = std::make_unique( getPort().getHeader(), "Parquet", - format_settings.parquet.import_nested, format_settings.parquet.allow_missing_columns, format_settings.null_as_default, format_settings.parquet.case_insensitive_column_matching); @@ -415,7 +414,6 @@ void registerInputFormatParquet(FormatFactory & factory) max_parsing_threads, min_bytes_for_seek); }); - factory.markFormatSupportsSubcolumns("Parquet"); factory.markFormatSupportsSubsetOfColumns("Parquet"); } diff --git a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp index 229a0630328..6f189816eff 100644 --- a/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetMetadataInputFormat.cpp @@ -504,7 +504,6 @@ void registerInputFormatParquetMetadata(FormatFactory & factory) { return std::make_shared(buf, sample, settings); }); - factory.markFormatSupportsSubcolumns("ParquetMetadata"); factory.markFormatSupportsSubsetOfColumns("ParquetMetadata"); } diff --git a/src/Processors/Transforms/ExtractColumnsTransform.cpp b/src/Processors/Transforms/ExtractColumnsTransform.cpp new file mode 100644 index 00000000000..44bf5582290 --- /dev/null +++ b/src/Processors/Transforms/ExtractColumnsTransform.cpp @@ -0,0 +1,35 @@ +#include +#include + +namespace DB +{ + +ExtractColumnsTransform::ExtractColumnsTransform(const Block & header_, const NamesAndTypesList & requested_columns_) + : ISimpleTransform(header_, transformHeader(header_, requested_columns_), false), requested_columns(requested_columns_) +{ + +} + +Block ExtractColumnsTransform::transformHeader(Block header, const NamesAndTypesList & requested_columns_) +{ + ColumnsWithTypeAndName columns; + columns.reserve(requested_columns_.size()); + for (const auto & required_column : requested_columns_) + columns.emplace_back(getColumnFromBlock(header, required_column), required_column.type, required_column.name); + + return Block(std::move(columns)); +} + +void ExtractColumnsTransform::transform(Chunk & chunk) +{ + size_t num_rows = chunk.getNumRows(); + auto block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()); + Columns columns; + columns.reserve(requested_columns.size()); + for (const auto & required_column : requested_columns) + columns.emplace_back(getColumnFromBlock(block, required_column)); + + chunk.setColumns(std::move(columns), num_rows); +} + +} diff --git a/src/Processors/Transforms/ExtractColumnsTransform.h b/src/Processors/Transforms/ExtractColumnsTransform.h new file mode 100644 index 00000000000..f8b3d803736 --- /dev/null +++ b/src/Processors/Transforms/ExtractColumnsTransform.h @@ -0,0 +1,26 @@ +#pragma once +#include + +namespace DB +{ + +/// Extracts required columns and subcolumns from the block. +class ExtractColumnsTransform final : public ISimpleTransform +{ +public: + ExtractColumnsTransform( + const Block & header_, + const NamesAndTypesList & requested_columns_); + + String getName() const override { return "ExtractColumnsTransform"; } + + static Block transformHeader(Block header, const NamesAndTypesList & requested_columns_); + +protected: + void transform(Chunk & chunk) override; + +private: + const NamesAndTypesList requested_columns; +}; + +} diff --git a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp b/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp index 309aa54909a..1172a40627d 100644 --- a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp +++ b/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp @@ -281,7 +281,6 @@ struct DeltaLakeMetadataParser::Impl ArrowColumnToCHColumn column_reader( header, "Parquet", - format_settings.parquet.import_nested, format_settings.parquet.allow_missing_columns, /* null_as_default */true, /* case_insensitive_column_matching */false); diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index a41c65cdb2e..c662c21e60f 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -332,30 +333,21 @@ StorageHDFS::PathWithInfo HDFSSource::URISIterator::next() return pimpl->next(); } -Block HDFSSource::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; -} - HDFSSource::HDFSSource( + const ReadFromFormatInfo & info, StorageHDFSPtr storage_, - const Block & block_for_format_, - const std::vector & requested_virtual_columns_, ContextPtr context_, UInt64 max_block_size_, - std::shared_ptr file_iterator_, - ColumnsDescription columns_description_) - : ISource(getHeader(block_for_format_, requested_virtual_columns_)) + std::shared_ptr file_iterator_) + : ISource(info.source_header) , WithContext(context_) , storage(std::move(storage_)) - , block_for_format(block_for_format_) - , requested_virtual_columns(requested_virtual_columns_) + , block_for_format(info.format_header) + , requested_columns(info.requested_columns) + , requested_virtual_columns(info.requested_virtual_columns) , max_block_size(max_block_size_) , file_iterator(file_iterator_) - , columns_description(std::move(columns_description_)) + , columns_description(info.columns_description) { initialize(); } @@ -408,6 +400,14 @@ bool HDFSSource::initialize() return std::make_shared(header, columns_description, *input_format, getContext()); }); } + + /// Add ExtractColumnsTransform to extract requested columns/subcolumns + /// from chunk read by IInputFormat. + builder.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, requested_columns); + }); + pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); reader = std::make_unique(*pipeline); return true; @@ -640,50 +640,17 @@ Pipe StorageHDFS::read( }); } - 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); - } - - 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, format_name, getVirtuals()); Pipes pipes; auto this_ptr = std::static_pointer_cast(shared_from_this()); for (size_t i = 0; i < num_streams; ++i) { pipes.emplace_back(std::make_shared( + read_from_format_info, this_ptr, - block_for_format, - requested_virtual_columns, context_, max_block_size, - iterator_wrapper, - columns_description)); + iterator_wrapper)); } return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 74801b68f73..239da19620d 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -7,6 +7,7 @@ #include #include #include +#include #include namespace DB @@ -74,6 +75,8 @@ public: /// format to read only them. Note: this hack cannot be done with ordinary formats like TSV. bool supportsSubsetOfColumns() const override; + bool supportsSubcolumns() const override { return true; } + static ColumnsDescription getTableStructureFromData( const String & format, const String & uri, @@ -140,16 +143,12 @@ public: using IteratorWrapper = std::function; using StorageHDFSPtr = std::shared_ptr; - static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); - HDFSSource( + const ReadFromFormatInfo & info, StorageHDFSPtr storage_, - const Block & block_for_format_, - const std::vector & requested_virtual_columns_, ContextPtr context_, UInt64 max_block_size_, - std::shared_ptr file_iterator_, - ColumnsDescription columns_description_); + std::shared_ptr file_iterator_); String getName() const override; @@ -158,7 +157,8 @@ public: private: StorageHDFSPtr storage; Block block_for_format; - std::vector requested_virtual_columns; + NamesAndTypesList requested_columns; + NamesAndTypesList requested_virtual_columns; UInt64 max_block_size; std::shared_ptr file_iterator; ColumnsDescription columns_description; diff --git a/src/Storages/HDFS/StorageHDFSCluster.h b/src/Storages/HDFS/StorageHDFSCluster.h index 350051ab089..1dbf2f3c4e2 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.h +++ b/src/Storages/HDFS/StorageHDFSCluster.h @@ -37,6 +37,8 @@ public: RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override; + bool supportsSubcolumns() const override { return true; } + private: void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override; diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 133dbb6740f..6d61778a2f1 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include @@ -603,7 +604,7 @@ private: Pipe StorageAzureBlob::read( const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, + const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, ContextPtr local_context, QueryProcessingStage::Enum /*processed_stage*/, @@ -615,15 +616,6 @@ Pipe StorageAzureBlob::read( 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; if (configuration.withGlobs()) { @@ -639,39 +631,15 @@ Pipe StorageAzureBlob::read( std::nullopt, query_info.query, virtual_block, local_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, configuration.format, getVirtuals()); for (size_t i = 0; i < num_streams; ++i) { pipes.emplace_back(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, configuration.compression_method, object_storage.get(), @@ -762,11 +730,6 @@ bool StorageAzureBlob::supportsPartitionBy() const return true; } -bool StorageAzureBlob::supportsSubcolumns() const -{ - return FormatFactory::instance().checkIfFormatSupportsSubcolumns(configuration.format); -} - bool StorageAzureBlob::supportsSubsetOfColumns() const { return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); @@ -1075,35 +1038,26 @@ Chunk StorageAzureBlobSource::generate() return {}; } -Block StorageAzureBlobSource::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; -} - StorageAzureBlobSource::StorageAzureBlobSource( - 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_, String compression_hint_, AzureObjectStorage * object_storage_, const String & container_, std::shared_ptr file_iterator_) - :ISource(getHeader(sample_block_, requested_virtual_columns_)) + :ISource(info.source_header) , WithContext(context_) - , requested_virtual_columns(requested_virtual_columns_) + , requested_columns(info.requested_columns) + , requested_virtual_columns(info.requested_virtual_columns) , format(format_) , name(std::move(name_)) - , sample_block(sample_block_) + , sample_block(info.format_header) , format_settings(format_settings_) - , columns_desc(columns_) + , columns_desc(info.columns_description) , max_block_size(max_block_size_) , compression_hint(compression_hint_) , object_storage(std::move(object_storage_)) @@ -1159,6 +1113,13 @@ StorageAzureBlobSource::ReaderHolder StorageAzureBlobSource::createReader() { return std::make_shared(header, columns_desc, *input_format, getContext()); }); } + /// Add ExtractColumnsTransform to extract requested columns/subcolumns + /// from chunk read by IInputFormat. + builder.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, requested_columns); + }); + auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); auto current_reader = std::make_unique(*pipeline); diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 25c791f1700..ce02324ed13 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB { @@ -93,7 +94,7 @@ public: bool supportsPartitionBy() const override; - bool supportsSubcolumns() const override; + bool supportsSubcolumns() const override { return true; } bool supportsSubsetOfColumns() const override; @@ -185,13 +186,11 @@ public: }; StorageAzureBlobSource( - 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_, String compression_hint_, AzureObjectStorage * object_storage_, @@ -204,10 +203,9 @@ public: String getName() const override; - static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); - private: - std::vector requested_virtual_columns; + NamesAndTypesList requested_columns; + NamesAndTypesList requested_virtual_columns; String format; String name; Block sample_block; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index ff67272e542..2009016d405 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -28,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -564,56 +566,46 @@ using StorageFilePtr = std::shared_ptr; class StorageFileSource : public ISource { public: - struct FilesInfo + class FilesIterator { + public: + explicit FilesIterator(const Strings & files_) : files(files_) + { + } + + String next() + { + auto current_index = index.fetch_add(1, std::memory_order_relaxed); + if (current_index >= files.size()) + return ""; + + return files[current_index]; + } + + private: std::vector files; - - std::atomic next_file_to_read = 0; - - bool need_path_column = false; - bool need_file_column = false; - - size_t total_bytes_to_read = 0; + std::atomic index = 0; }; - using FilesInfoPtr = std::shared_ptr; - - static Block getBlockForSource(const Block & block_for_format, const FilesInfoPtr & files_info) - { - auto res = block_for_format; - if (files_info->need_path_column) - { - res.insert( - {DataTypeLowCardinality{std::make_shared()}.createColumn(), - std::make_shared(std::make_shared()), - "_path"}); - } - if (files_info->need_file_column) - { - res.insert( - {DataTypeLowCardinality{std::make_shared()}.createColumn(), - std::make_shared(std::make_shared()), - "_file"}); - } - return res; - } + using FilesIteratorPtr = std::shared_ptr; StorageFileSource( + const ReadFromFormatInfo & info, std::shared_ptr storage_, const StorageSnapshotPtr & storage_snapshot_, ContextPtr context_, UInt64 max_block_size_, - FilesInfoPtr files_info_, - ColumnsDescription columns_description_, - const Block & block_for_format_, + FilesIteratorPtr files_iterator_, std::unique_ptr read_buf_) - : ISource(getBlockForSource(block_for_format_, files_info_)) + : ISource(info.source_header) , storage(std::move(storage_)) , storage_snapshot(storage_snapshot_) - , files_info(std::move(files_info_)) + , files_iterator(std::move(files_iterator_)) , read_buf(std::move(read_buf_)) - , columns_description(std::move(columns_description_)) - , block_for_format(block_for_format_) + , columns_description(info.columns_description) + , requested_columns(info.requested_columns) + , requested_virtual_columns(info.requested_virtual_columns) + , block_for_format(info.format_header) , context(context_) , max_block_size(max_block_size_) { @@ -699,12 +691,10 @@ 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()) + current_path = files_iterator->next(); + if (current_path.empty()) return {}; - current_path = files_info->files[current_file]; - /// Special case for distributed format. Defaults are not needed here. if (storage->format_name == "Distributed") { @@ -744,6 +734,13 @@ public: }); } + /// Add ExtractColumnsTransform to extract requested columns/subcolumns + /// from chunk read by IInputFormat. + builder.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, requested_columns); + }); + pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); reader = std::make_unique(*pipeline); @@ -755,19 +752,19 @@ public: UInt64 num_rows = chunk.getNumRows(); /// Enrich with virtual columns. - if (files_info->need_path_column) - { - auto column = DataTypeLowCardinality{std::make_shared()}.createColumnConst(num_rows, current_path); - chunk.addColumn(column->convertToFullColumnIfConst()); - } - if (files_info->need_file_column) + for (const auto & virtual_column : requested_virtual_columns) { - 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)); - chunk.addColumn(column->convertToFullColumnIfConst()); + if (virtual_column.name == "_path") + { + chunk.addColumn(virtual_column.type->createColumnConst(num_rows, current_path)->convertToFullColumnIfConst()); + } + else if (virtual_column.name == "_file") + { + size_t last_slash_pos = current_path.find_last_of('/'); + auto file_name = current_path.substr(last_slash_pos + 1); + chunk.addColumn(virtual_column.type->createColumnConst(num_rows, file_name)->convertToFullColumnIfConst()); + } } if (num_rows && total_files_size) @@ -799,7 +796,7 @@ public: private: std::shared_ptr storage; StorageSnapshotPtr storage_snapshot; - FilesInfoPtr files_info; + FilesIteratorPtr files_iterator; String current_path; Block sample_block; std::unique_ptr read_buf; @@ -808,6 +805,8 @@ private: std::unique_ptr reader; ColumnsDescription columns_description; + NamesAndTypesList requested_columns; + NamesAndTypesList requested_virtual_columns; Block block_for_format; ContextPtr context; /// TODO Untangle potential issues with context lifetime. @@ -849,18 +848,7 @@ Pipe StorageFile::read( } } - auto files_info = std::make_shared(); - files_info->files = paths; - files_info->total_bytes_to_read = total_bytes_to_read; - - for (const auto & column : column_names) - { - if (column == "_path") - files_info->need_path_column = true; - if (column == "_file") - files_info->need_file_column = true; - } - + auto files_iterator = std::make_shared(paths); auto this_ptr = std::static_pointer_cast(shared_from_this()); size_t num_streams = max_num_streams; @@ -876,33 +864,10 @@ 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()); + for (size_t i = 0; i < num_streams; ++i) { - 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); - } - else - { - columns_description = storage_snapshot->metadata->getColumns(); - } - - block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); - /// In case of reading from fd we have to check whether we have already created /// the read buffer from it in Storage constructor (for schema inference) or not. /// If yes, then we should use it in StorageFileSource. Atomic bool flag is needed @@ -912,13 +877,12 @@ Pipe StorageFile::read( read_buffer = std::move(peekable_read_buffer_from_fd); pipes.emplace_back(std::make_shared( + read_from_format_info, this_ptr, storage_snapshot, context, max_block_size, - files_info, - columns_description, - block_for_format, + files_iterator, std::move(read_buffer))); } diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index ed50ae73e51..a7c9beece17 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -75,6 +75,8 @@ public: /// format to read only them. Note: this hack cannot be done with ordinary formats like TSV. 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/StorageS3.cpp b/src/Storages/StorageS3.cpp index 292ae4813dd..6a50d147c67 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -42,6 +42,7 @@ #include #include +#include #include #include @@ -528,22 +529,12 @@ size_t StorageS3Source::KeysIterator::getTotalSize() const return pimpl->getTotalSize(); } -Block StorageS3Source::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; -} - StorageS3Source::StorageS3Source( - 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_, @@ -552,20 +543,21 @@ StorageS3Source::StorageS3Source( const String & version_id_, std::shared_ptr file_iterator_, 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) + , requested_columns(info.requested_columns) , max_block_size(max_block_size_) , request_settings(request_settings_) , compression_hint(std::move(compression_hint_)) , client(client_) - , sample_block(sample_block_) + , sample_block(info.format_header) , format_settings(format_settings_) - , requested_virtual_columns(requested_virtual_columns_) + , requested_virtual_columns(info.requested_virtual_columns) , file_iterator(file_iterator_) , download_thread_num(download_thread_num_) , create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) @@ -611,6 +603,13 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() { return std::make_shared(header, columns_desc, *input_format, getContext()); }); } + /// Add ExtractColumnsTransform to extract requested columns/subcolumns + /// from chunk read by IInputFormat. + builder.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, requested_columns); + }); + auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); auto current_reader = std::make_unique(*pipeline); @@ -1026,11 +1025,6 @@ std::shared_ptr StorageS3::createFileIterator( } } -bool StorageS3::supportsSubcolumns() const -{ - return FormatFactory::instance().checkIfFormatSupportsSubcolumns(configuration.format); -} - bool StorageS3::supportsSubsetOfColumns() const { return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); @@ -1062,52 +1056,20 @@ Pipe StorageS3::read( 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( query_configuration, 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(); - } + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, query_configuration.format, getVirtuals()); const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; for (size_t i = 0; i < num_streams; ++i) { pipes.emplace_back(std::make_shared( - requested_virtual_columns, + read_from_format_info, query_configuration.format, getName(), - block_for_format, local_context, format_settings, - columns_description, max_block_size, query_configuration.request_settings, query_configuration.compression_method, diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 13053833623..267007501e0 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -21,6 +21,7 @@ #include #include #include +#include namespace Aws::S3 { @@ -119,16 +120,12 @@ public: ReadTaskCallback callback; }; - static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); - StorageS3Source( - 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_, @@ -150,6 +147,7 @@ private: String version_id; String format; ColumnsDescription columns_desc; + NamesAndTypesList requested_columns; UInt64 max_block_size; S3Settings::RequestSettings request_settings; String compression_hint; @@ -214,7 +212,7 @@ private: ReaderHolder reader; - std::vector requested_virtual_columns; + NamesAndTypesList requested_virtual_columns; std::shared_ptr file_iterator; size_t download_thread_num = 1; @@ -360,7 +358,7 @@ private: const std::optional & format_settings, ContextPtr ctx); - bool supportsSubcolumns() const override; + bool supportsSubcolumns() const override { return true; } bool supportsSubsetOfColumns() const override; diff --git a/src/Storages/StorageS3Cluster.h b/src/Storages/StorageS3Cluster.h index 5c2229875e5..4b4558f6330 100644 --- a/src/Storages/StorageS3Cluster.h +++ b/src/Storages/StorageS3Cluster.h @@ -36,6 +36,8 @@ public: RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override; + bool supportsSubcolumns() const override { return true; } + protected: void updateConfigurationIfChanged(ContextPtr local_context); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 947881d34c6..5dd5055dc42 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -23,6 +22,8 @@ #include #include #include +#include +#include #include #include @@ -208,25 +209,15 @@ void StorageURLSource::setCredentials(Poco::Net::HTTPBasicCredentials & credenti } } -Block StorageURLSource::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; -} - StorageURLSource::StorageURLSource( - const std::vector & requested_virtual_columns_, + const ReadFromFormatInfo & info, std::shared_ptr uri_iterator_, const std::string & http_method, std::function callback, const String & format, const std::optional & format_settings, String name_, - const Block & sample_block, ContextPtr context, - const ColumnsDescription & columns, UInt64 max_block_size, const ConnectionTimeouts & timeouts, CompressionMethod compression_method, @@ -234,7 +225,13 @@ StorageURLSource::StorageURLSource( const HTTPHeaderEntries & headers_, const URIParams & params, bool glob_url) - : ISource(getHeader(sample_block, requested_virtual_columns_)), name(std::move(name_)), requested_virtual_columns(requested_virtual_columns_), uri_iterator(uri_iterator_) + : ISource(info.source_header) + , name(std::move(name_)) + , columns_description(info.columns_description) + , requested_columns(info.requested_columns) + , requested_virtual_columns(info.requested_virtual_columns) + , block_for_format(info.format_header) + , uri_iterator(uri_iterator_) { auto headers = getHeaders(headers_); @@ -292,7 +289,7 @@ StorageURLSource::StorageURLSource( input_format = FormatFactory::instance().getInput( format, *read_buf, - sample_block, + info.format_header, context, max_block_size, format_settings, @@ -304,8 +301,20 @@ StorageURLSource::StorageURLSource( QueryPipelineBuilder builder; builder.init(Pipe(input_format)); - builder.addSimpleTransform([&](const Block & cur_header) - { return std::make_shared(cur_header, columns, *input_format, context); }); + if (columns_description.hasDefaults()) + { + builder.addSimpleTransform([&](const Block & cur_header) + { + return std::make_shared(cur_header, columns_description, *input_format, context); + }); + } + + /// Add ExtractColumnsTransform to extract requested columns/subcolumns + /// from chunk read by IInputFormat. + builder.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, requested_columns); + }); pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); reader = std::make_unique(*pipeline); @@ -712,27 +721,6 @@ Pipe IStorageURLBase::read( { auto params = getReadURIParams(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size); - 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(); - } - - 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); - } - size_t max_download_threads = local_context->getSettingsRef().max_download_threads; std::shared_ptr iterator_wrapper{nullptr}; @@ -776,6 +764,8 @@ Pipe IStorageURLBase::read( num_streams = 1; } + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, format_name, getVirtuals()); + Pipes pipes; pipes.reserve(num_streams); @@ -783,16 +773,14 @@ Pipe IStorageURLBase::read( for (size_t i = 0; i < num_streams; ++i) { pipes.emplace_back(std::make_shared( - requested_virtual_columns, + read_from_format_info, iterator_wrapper, getReadMethod(), - getReadPOSTDataCallback(column_names, columns_description, query_info, local_context, processed_stage, max_block_size), + getReadPOSTDataCallback(column_names, read_from_format_info.columns_description, query_info, local_context, processed_stage, max_block_size), format_name, format_settings, getName(), - block_for_format, local_context, - columns_description, max_block_size, getHTTPTimeouts(local_context), compression_method, @@ -838,17 +826,17 @@ Pipe StorageURLWithFailover::read( return uri_options; }); + auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, format_name, getVirtuals()); + auto pipe = Pipe(std::make_shared( - std::vector{}, + read_from_format_info, iterator_wrapper, getReadMethod(), - getReadPOSTDataCallback(column_names, columns_description, query_info, local_context, processed_stage, max_block_size), + getReadPOSTDataCallback(column_names, read_from_format_info.columns_description, query_info, local_context, processed_stage, max_block_size), format_name, format_settings, getName(), - block_for_format, local_context, - columns_description, max_block_size, getHTTPTimeouts(local_context), compression_method, diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 345f813dd7c..a6fb6840a22 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB @@ -46,6 +47,8 @@ public: bool supportsPartitionBy() const override { return true; } + bool supportsSubcolumns() const override { return true; } + NamesAndTypesList getVirtuals() const override; static ColumnsDescription getTableStructureFromData( @@ -158,16 +161,14 @@ public: using IteratorWrapper = std::function; StorageURLSource( - const std::vector & requested_virtual_columns_, + const ReadFromFormatInfo & info, std::shared_ptr uri_iterator_, const std::string & http_method, std::function callback, const String & format, const std::optional & format_settings, String name_, - const Block & sample_block, ContextPtr context, - const ColumnsDescription & columns, UInt64 max_block_size, const ConnectionTimeouts & timeouts, CompressionMethod compression_method, @@ -182,8 +183,6 @@ public: static void setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri); - static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); - static std::pair> getFirstAvailableURIAndReadBuffer( std::vector::const_iterator & option, const std::vector::const_iterator & end, @@ -202,7 +201,10 @@ private: InitializeFunc initialize; String name; - std::vector requested_virtual_columns; + ColumnsDescription columns_description; + NamesAndTypesList requested_columns; + NamesAndTypesList requested_virtual_columns; + Block block_for_format; std::shared_ptr uri_iterator; Poco::URI curr_uri; diff --git a/src/Storages/StorageURLCluster.h b/src/Storages/StorageURLCluster.h index 67771416771..bd475d78f65 100644 --- a/src/Storages/StorageURLCluster.h +++ b/src/Storages/StorageURLCluster.h @@ -36,6 +36,8 @@ public: RemoteQueryExecutor::Extension getTaskIteratorExtension(ASTPtr query, const ContextPtr & context) const override; + bool supportsSubcolumns() const override { return true; } + private: void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override; diff --git a/src/Storages/prepareReadingFromFormat.cpp b/src/Storages/prepareReadingFromFormat.cpp new file mode 100644 index 00000000000..aac5e53c569 --- /dev/null +++ b/src/Storages/prepareReadingFromFormat.cpp @@ -0,0 +1,69 @@ +#include +#include + +namespace DB +{ + +ReadFromFormatInfo prepareReadingFromFormat(const Strings & requested_columns, const StorageSnapshotPtr & storage_snapshot, const String & format_name, const NamesAndTypesList & virtuals) +{ + ReadFromFormatInfo info; + /// Collect requested virtual columns and remove them from requested columns. + Strings columns_to_read; + for (const auto & column_name : requested_columns) + { + bool is_virtual = false; + for (const auto & virtual_column : virtuals) + { + if (column_name == virtual_column.name) + { + info.requested_virtual_columns.push_back(virtual_column); + is_virtual = true; + break; + } + } + + if (!is_virtual) + columns_to_read.push_back(column_name); + } + + /// Create header for Source that will contain all requested columns including virtual columns at the end + /// (because they will be added to the chunk after reading regular columns). + info.source_header = storage_snapshot->getSampleBlockForColumns(columns_to_read); + for (const auto & requested_virtual_column : info.requested_virtual_columns) + 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); + + if (format_name != "Distributed" && FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name)) + { + /// If only virtual columns were requested, just read the smallest column. + if (columns_to_read.empty()) + { + columns_to_read.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical()).name); + } + /// We need to replace all subcolumns with their nested columns (e.g `a.b`, `a.b.c`, `x.y` -> `a`, `x`), + /// because most formats cannot extract subcolumns on their own. + /// All requested subcolumns will be extracted after reading. + else + { + std::unordered_set columns_to_read_set; + 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()); + } + info.columns_description = storage_snapshot->getDescriptionForColumns(columns_to_read); + } + /// If format doesn't support reading subset of columns, read all columns. + /// Requested columns/subcolumns will be extracted after reading. + else + { + info.columns_description = storage_snapshot->metadata->getColumns(); + } + + /// Create header for InputFormat with columns that will be read from the data. + info.format_header = storage_snapshot->getSampleBlockForColumns(info.columns_description.getNamesOfPhysical()); + return info; +} + +} diff --git a/src/Storages/prepareReadingFromFormat.h b/src/Storages/prepareReadingFromFormat.h new file mode 100644 index 00000000000..ac029c215db --- /dev/null +++ b/src/Storages/prepareReadingFromFormat.h @@ -0,0 +1,26 @@ +#pragma once +#include +#include + +namespace DB +{ + struct ReadFromFormatInfo + { + /// Header that will return Source from storage. + /// It contains all requested columns including virtual columns; + Block source_header; + /// Header that will be passed to IInputFormat to read data from file. + /// It can contain more columns than were requested if format doesn't support + /// reading subset of columns. + Block format_header; + /// Description of columns for format_header. Used for inserting defaults. + ColumnsDescription columns_description; + /// The list of requested columns without virtual columns. + NamesAndTypesList requested_columns; + /// The list of requested virtual columns. + NamesAndTypesList requested_virtual_columns; + }; + + /// 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); +} diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f9d337b6d86..20d58573686 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -592,3 +592,40 @@ 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") + + +def test_read_subcolumns(cluster): + node = cluster.instances["node"] + azure_query( + node, + "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.tsv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)", + ) + + azure_query( + node, + "INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)", + ) + + res = node.query( + f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.tsv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + ) + + assert res == "2\tcont/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" + + res = node.query( + f"select a.b.d, _path, a.b, _file, a.e from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + ) + + assert res == "2\tcont/test_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" + + res = node.query( + f"select x.b.d, _path, x.b, _file, x.e from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + ) + + assert res == "0\tcont/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" + + res = node.query( + f"select x.b.d, _path, x.b, _file, x.e from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_subcolumns.jsonl', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', 'auto', 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" + ) + + assert res == "42\tcont/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 5ac1d3bea6f..82ce901a9e6 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -866,6 +866,54 @@ def test_skip_empty_files(started_cluster): assert int(res) == 0 +def test_read_subcolumns(started_cluster): + node = started_cluster.instances["node1"] + + node.query( + f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" + ) + + node.query( + f"insert into function hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" + ) + + res = node.query( + f"select a.b.d, _path, a.b, _file, a.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + ) + + assert ( + res + == "2\thdfs://hdfs1:9000/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" + ) + + res = node.query( + f"select a.b.d, _path, a.b, _file, a.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + ) + + assert ( + res + == "2\thdfs://hdfs1:9000/test_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" + ) + + res = node.query( + f"select x.b.d, _path, x.b, _file, x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + ) + + assert ( + res + == "0\thdfs://hdfs1:9000/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" + ) + + res = node.query( + f"select x.b.d, _path, x.b, _file, x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" + ) + + assert ( + res + == "42\thdfs://hdfs1:9000/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" + ) + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index a7293337a9e..a6c79b743bb 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1780,3 +1780,66 @@ def test_skip_empty_files(started_cluster): ) assert len(res.strip()) == 0 + + +def test_read_subcolumns(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] + + instance.query( + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" + ) + + instance.query( + f"insert into function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)') select ((1, 2), 3)" + ) + + res = instance.query( + f"select a.b.d, _path, a.b, _file, a.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + ) + + assert res == "2\troot/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" + + res = instance.query( + f"select a.b.d, _path, a.b, _file, a.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + ) + + assert res == "2\troot/test_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" + + res = instance.query( + f"select x.b.d, _path, x.b, _file, x.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + ) + + assert res == "0\troot/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" + + res = instance.query( + f"select x.b.d, _path, x.b, _file, x.e from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" + ) + + assert res == "42\troot/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" + + res = instance.query( + f"select a.b.d, _path, a.b, _file, a.e from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + ) + + assert res == "2\t/root/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" + + res = instance.query( + f"select a.b.d, _path, a.b, _file, a.e from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + ) + + assert res == "2\t/root/test_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" + + res = instance.query( + f"select x.b.d, _path, x.b, _file, x.e from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + ) + + assert res == "0\t/root/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" + + res = instance.query( + f"select x.b.d, _path, x.b, _file, x.e from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" + ) + + assert ( + res == "42\t/root/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" + ) diff --git a/tests/queries/0_stateless/02797_read_subcolumns_from_files.reference b/tests/queries/0_stateless/02797_read_subcolumns_from_files.reference new file mode 100644 index 00000000000..45ea5a7a29f --- /dev/null +++ b/tests/queries/0_stateless/02797_read_subcolumns_from_files.reference @@ -0,0 +1,4 @@ +2 (1,2) 3 +2 (1,2) 3 +0 (0,0) 0 +42 (42,42) 42 diff --git a/tests/queries/0_stateless/02797_read_subcolumns_from_files.sh b/tests/queries/0_stateless/02797_read_subcolumns_from_files.sh new file mode 100755 index 00000000000..767acf68553 --- /dev/null +++ b/tests/queries/0_stateless/02797_read_subcolumns_from_files.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +DATA_FILE=$CLICKHOUSE_TEST_UNIQUE_NAME + +$CLICKHOUSE_LOCAL -q "select ((1, 2), 3)::Tuple(b Tuple(c UInt32, d UInt32), e UInt32) as a format TSV" > $DATA_FILE +$CLICKHOUSE_LOCAL -q "select a.b.d, a.b, a.e from file('$DATA_FILE', TSV, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" + +$CLICKHOUSE_LOCAL -q "select ((1, 2), 3)::Tuple(b Tuple(c UInt32, d UInt32), e UInt32) as a format JSONEachRow" > $DATA_FILE +$CLICKHOUSE_LOCAL -q "select a.b.d, a.b, a.e from file('$DATA_FILE', JSONEachRow, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" +$CLICKHOUSE_LOCAL -q "select x.b.d, x.b, x.e from file('$DATA_FILE', JSONEachRow, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" +$CLICKHOUSE_LOCAL -q "select x.b.d, x.b, x.e from file('$DATA_FILE', JSONEachRow, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" + +rm $DATA_FILE + From 3dc4ff17608b42fb5aabe8a9e7cf90ec0f71e60e Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 4 Jul 2023 21:21:22 +0000 Subject: [PATCH 090/554] Remove obsolete settings --- docs/en/interfaces/formats.md | 3 -- .../operations/settings/settings-formats.md | 33 ------------------- docs/ru/interfaces/formats.md | 6 ---- docs/ru/operations/settings/settings.md | 33 ------------------- 4 files changed, 75 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 79baf04d75d..22a5fd0db37 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -2056,7 +2056,6 @@ To exchange data with Hadoop, you can use [HDFS table engine](/docs/en/engines/t - [output_format_parquet_row_group_size](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_row_group_size) - row group size in rows while data output. Default value - `1000000`. - [output_format_parquet_string_as_string](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_string_as_string) - use Parquet String type instead of Binary for String columns. Default value - `false`. -- [input_format_parquet_import_nested](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_import_nested) - allow inserting array of structs into [Nested](/docs/en/sql-reference/data-types/nested-data-structures/index.md) table in Parquet input format. Default value - `false`. - [input_format_parquet_case_insensitive_column_matching](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_case_insensitive_column_matching) - ignore case when matching Parquet columns with ClickHouse columns. Default value - `false`. - [input_format_parquet_allow_missing_columns](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_allow_missing_columns) - allow missing columns while reading Parquet data. Default value - `false`. - [input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Parquet format. Default value - `false`. @@ -2261,7 +2260,6 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT Arrow" > {filenam - [output_format_arrow_low_cardinality_as_dictionary](/docs/en/operations/settings/settings-formats.md/#output_format_arrow_low_cardinality_as_dictionary) - enable output ClickHouse LowCardinality type as Dictionary Arrow type. Default value - `false`. - [output_format_arrow_string_as_string](/docs/en/operations/settings/settings-formats.md/#output_format_arrow_string_as_string) - use Arrow String type instead of Binary for String columns. Default value - `false`. -- [input_format_arrow_import_nested](/docs/en/operations/settings/settings-formats.md/#input_format_arrow_import_nested) - allow inserting array of structs into Nested table in Arrow input format. Default value - `false`. - [input_format_arrow_case_insensitive_column_matching](/docs/en/operations/settings/settings-formats.md/#input_format_arrow_case_insensitive_column_matching) - ignore case when matching Arrow columns with ClickHouse columns. Default value - `false`. - [input_format_arrow_allow_missing_columns](/docs/en/operations/settings/settings-formats.md/#input_format_arrow_allow_missing_columns) - allow missing columns while reading Arrow data. Default value - `false`. - [input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference](/docs/en/operations/settings/settings-formats.md/#input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Arrow format. Default value - `false`. @@ -2327,7 +2325,6 @@ $ clickhouse-client --query="SELECT * FROM {some_table} FORMAT ORC" > {filename. - [output_format_arrow_string_as_string](/docs/en/operations/settings/settings-formats.md/#output_format_arrow_string_as_string) - use Arrow String type instead of Binary for String columns. Default value - `false`. - [output_format_orc_compression_method](/docs/en/operations/settings/settings-formats.md/#output_format_orc_compression_method) - compression method used in output ORC format. Default value - `none`. -- [input_format_arrow_import_nested](/docs/en/operations/settings/settings-formats.md/#input_format_arrow_import_nested) - allow inserting array of structs into Nested table in Arrow input format. Default value - `false`. - [input_format_arrow_case_insensitive_column_matching](/docs/en/operations/settings/settings-formats.md/#input_format_arrow_case_insensitive_column_matching) - ignore case when matching Arrow columns with ClickHouse columns. Default value - `false`. - [input_format_arrow_allow_missing_columns](/docs/en/operations/settings/settings-formats.md/#input_format_arrow_allow_missing_columns) - allow missing columns while reading Arrow data. Default value - `false`. - [input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference](/docs/en/operations/settings/settings-formats.md/#input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Arrow format. Default value - `false`. diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index 1b22a6d1223..3c7c66fabdb 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1061,17 +1061,6 @@ Default value: 1. ## Arrow format settings {#arrow-format-settings} -### input_format_arrow_import_nested {#input_format_arrow_import_nested} - -Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/index.md) columns as an array of structs in [Arrow](../../interfaces/formats.md/#data_types-matching-arrow) input format. - -Possible values: - -- 0 — Data can not be inserted into `Nested` columns as an array of structs. -- 1 — Data can be inserted into `Nested` columns as an array of structs. - -Default value: `0`. - ### input_format_arrow_case_insensitive_column_matching {#input_format_arrow_case_insensitive_column_matching} Ignore case when matching Arrow column names with ClickHouse column names. @@ -1121,17 +1110,6 @@ Default value: `none`. ## ORC format settings {#orc-format-settings} -### input_format_orc_import_nested {#input_format_orc_import_nested} - -Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/index.md) columns as an array of structs in [ORC](../../interfaces/formats.md/#data-format-orc) input format. - -Possible values: - -- 0 — Data can not be inserted into `Nested` columns as an array of structs. -- 1 — Data can be inserted into `Nested` columns as an array of structs. - -Default value: `0`. - ### input_format_orc_row_batch_size {#input_format_orc_row_batch_size} Batch size when reading ORC stripes. @@ -1170,17 +1148,6 @@ Default value: `none`. ## Parquet format settings {#parquet-format-settings} -### input_format_parquet_import_nested {#input_format_parquet_import_nested} - -Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/index.md) columns as an array of structs in [Parquet](../../interfaces/formats.md/#data-format-parquet) input format. - -Possible values: - -- 0 — Data can not be inserted into `Nested` columns as an array of structs. -- 1 — Data can be inserted into `Nested` columns as an array of structs. - -Default value: `0`. - ### input_format_parquet_case_insensitive_column_matching {#input_format_parquet_case_insensitive_column_matching} Ignore case when matching Parquet column names with ClickHouse column names. diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 48a6132170a..c71b2cfb75e 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -1353,8 +1353,6 @@ ClickHouse поддерживает настраиваемую точность $ cat {filename} | clickhouse-client --query="INSERT INTO {some_table} FORMAT Parquet" ``` -Чтобы вставить данные в колонки типа [Nested](../sql-reference/data-types/nested-data-structures/nested.md) в виде массива структур, нужно включить настройку [input_format_parquet_import_nested](../operations/settings/settings.md#input_format_parquet_import_nested). - Чтобы получить данные из таблицы ClickHouse и сохранить их в файл формата Parquet, используйте команду следующего вида: ``` bash @@ -1413,8 +1411,6 @@ ClickHouse поддерживает настраиваемую точность $ cat filename.arrow | clickhouse-client --query="INSERT INTO some_table FORMAT Arrow" ``` -Чтобы вставить данные в колонки типа [Nested](../sql-reference/data-types/nested-data-structures/nested.md) в виде массива структур, нужно включить настройку [input_format_arrow_import_nested](../operations/settings/settings.md#input_format_arrow_import_nested). - ### Вывод данных {#selecting-data-arrow} Чтобы получить данные из таблицы ClickHouse и сохранить их в файл формата Arrow, используйте команду следующего вида: @@ -1471,8 +1467,6 @@ ClickHouse поддерживает настраиваемую точность $ cat filename.orc | clickhouse-client --query="INSERT INTO some_table FORMAT ORC" ``` -Чтобы вставить данные в колонки типа [Nested](../sql-reference/data-types/nested-data-structures/nested.md) в виде массива структур, нужно включить настройку [input_format_orc_import_nested](../operations/settings/settings.md#input_format_orc_import_nested). - ### Вывод данных {#selecting-data-2} Чтобы получить данные из таблицы ClickHouse и сохранить их в файл формата ORC, используйте команду следующего вида: diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index f83d05ff710..ae56e973ced 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -238,39 +238,6 @@ ClickHouse применяет настройку в тех случаях, ко В случае превышения `input_format_allow_errors_ratio` ClickHouse генерирует исключение. -## input_format_parquet_import_nested {#input_format_parquet_import_nested} - -Включает или отключает возможность вставки данных в колонки типа [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) в виде массива структур в формате ввода [Parquet](../../interfaces/formats.md#data-format-parquet). - -Возможные значения: - -- 0 — данные не могут быть вставлены в колонки типа `Nested` в виде массива структур. -- 0 — данные могут быть вставлены в колонки типа `Nested` в виде массива структур. - -Значение по умолчанию: `0`. - -## input_format_arrow_import_nested {#input_format_arrow_import_nested} - -Включает или отключает возможность вставки данных в колонки типа [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) в виде массива структур в формате ввода [Arrow](../../interfaces/formats.md#data_types-matching-arrow). - -Возможные значения: - -- 0 — данные не могут быть вставлены в колонки типа `Nested` в виде массива структур. -- 0 — данные могут быть вставлены в колонки типа `Nested` в виде массива структур. - -Значение по умолчанию: `0`. - -## input_format_orc_import_nested {#input_format_orc_import_nested} - -Включает или отключает возможность вставки данных в колонки типа [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) в виде массива структур в формате ввода [ORC](../../interfaces/formats.md#data-format-orc). - -Возможные значения: - -- 0 — данные не могут быть вставлены в колонки типа `Nested` в виде массива структур. -- 0 — данные могут быть вставлены в колонки типа `Nested` в виде массива структур. - -Значение по умолчанию: `0`. - ## input_format_values_interpret_expressions {#settings-input_format_values_interpret_expressions} Включает или отключает парсер SQL, если потоковый парсер не может проанализировать данные. Этот параметр используется только для формата [Values](../../interfaces/formats.md#data-format-values) при вставке данных. Дополнительные сведения о парсерах читайте в разделе [Синтаксис](../../sql-reference/syntax.md). From e204a8fbe90d0ae21ac2085465c88a036f0b0b43 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 4 Jul 2023 21:21:43 +0000 Subject: [PATCH 091/554] Fix indent --- 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 710a6e4f135..1138b6a94b3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -829,7 +829,7 @@ class IColumn; MAKE_OBSOLETE(M, Bool, input_format_parquet_import_nested, false) \ MAKE_OBSOLETE(M, Bool, input_format_orc_import_nested, false) \ -/** The section above is for obsolete settings. Do not add anything there. */ + /** The section above is for obsolete settings. Do not add anything there. */ #define FORMAT_FACTORY_SETTINGS(M, ALIAS) \ From 00292be8f23f97a1f534a7f41e37f8d25b6feccd Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 4 Jul 2023 21:24:56 +0000 Subject: [PATCH 092/554] Remore unrelated chages --- src/Core/SettingsQuirks.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Core/SettingsQuirks.cpp b/src/Core/SettingsQuirks.cpp index cee9c3d497c..3326f42adf5 100644 --- a/src/Core/SettingsQuirks.cpp +++ b/src/Core/SettingsQuirks.cpp @@ -71,12 +71,6 @@ void applySettingsQuirks(Settings & settings, Poco::Logger * log) } } -//#if defined(THREAD_SANITIZER) - settings.use_hedged_requests.value = false; - if (log) - LOG_WARNING(log, "use_hedged_requests has been disabled for the build with Thread Sanitizer, because they are using fibers, leading to a failed assertion inside TSan"); -//#endif - if (!queryProfilerWorks()) { if (settings.query_profiler_real_time_period_ns) From ba5e26aebf242211db3dbc2f7f987b92dd2c76bd Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 5 Jul 2023 10:49:32 +0000 Subject: [PATCH 093/554] 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 966f07bd8eb4b2dfdb89e984d35771be56205cc6 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 5 Jul 2023 15:42:49 +0000 Subject: [PATCH 094/554] 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 095/554] 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 096/554] 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 097/554] 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 33f32684d42034db67c77bcf867165c31c4d4354 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 5 Jul 2023 16:34:42 +0000 Subject: [PATCH 098/554] 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 d11cd0dc30d998324961cef3f649e203a1d8c957 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 5 Jul 2023 17:56:03 +0000 Subject: [PATCH 099/554] 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 2d3a148ffd48d683276859b29cf952a985b6fb5e Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 6 Jul 2023 10:56:07 +0200 Subject: [PATCH 100/554] 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 101/554] 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 102/554] 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 103/554] 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 8d634c992bda74095befdf6b47012cbe17acceae Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 6 Jul 2023 17:47:01 +0000 Subject: [PATCH 104/554] 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 72e8303ee7dd73ff0038cb5cbf28b63c98d1a183 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 16 Jul 2023 22:49:06 +0200 Subject: [PATCH 105/554] 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 106/554] 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 e4384e104b75909f978a61cb5ccc413e220c5811 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 00:06:58 +0200 Subject: [PATCH 107/554] 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 d4459a5904aab19586b17f6d4675ed66dc78a614 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 04:16:02 +0200 Subject: [PATCH 108/554] 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 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 109/554] 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 110/554] 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 4a18ec62ea819b6049457ac8c8fd0391b57ef61f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 17 Jul 2023 14:52:11 +0200 Subject: [PATCH 111/554] 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 112/554] 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 fe53cd3d7464f960c8b824e88ec98183f03b54e8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jul 2023 16:11:58 +0200 Subject: [PATCH 113/554] 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 671f8b34d06364e2c6e26a60ef6489f8c23e4ef7 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 17 Jul 2023 17:18:14 +0200 Subject: [PATCH 114/554] 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 32311bf28f148e32836bfdc168b8f312330f3a5a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 17 Jul 2023 15:30:31 +0000 Subject: [PATCH 115/554] 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 342af6442822386adff831cc6565c088f6f884e4 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 18 Jul 2023 18:20:03 +0200 Subject: [PATCH 116/554] 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 b300781fd8995a4e8feac0e58a9b756c17fe93f7 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 18 Jul 2023 17:48:39 +0000 Subject: [PATCH 117/554] 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 118/554] 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 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 119/554] 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 120/554] 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 bab212f25e91cea8c39b2d025735283b7adcaa71 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 19 Jul 2023 12:47:29 +0000 Subject: [PATCH 121/554] 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 da6a31bb628de3f038fbf786b2102cbb3a7fee2d Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 19 Jul 2023 13:26:09 +0000 Subject: [PATCH 122/554] 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 10cdaac32f64c6346625ab2ee7e8af89fb2e0b22 Mon Sep 17 00:00:00 2001 From: zvonand Date: Wed, 19 Jul 2023 16:32:09 +0200 Subject: [PATCH 123/554] 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 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 124/554] 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 710e9a817de5c5b0f47ea1a0ae0ac29ac8e48cb4 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 20 Jul 2023 12:59:05 +0000 Subject: [PATCH 125/554] 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 d2195cff1101f5035148e9c0f7672bfb83eeb693 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 20 Jul 2023 18:21:37 +0200 Subject: [PATCH 126/554] 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 c8b128aad442c763f1b85c20b8389d240466f97f Mon Sep 17 00:00:00 2001 From: Yury Bogomolov Date: Fri, 21 Jul 2023 01:06:49 +0400 Subject: [PATCH 127/554] 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 6cc8a9af5c60a18a2d5ab636205993701e7fbb9a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 21 Jul 2023 11:19:12 +0200 Subject: [PATCH 128/554] 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 e181e602a7e7704160a30e507f2df19e193680fd Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 21 Jul 2023 12:43:32 +0200 Subject: [PATCH 129/554] 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 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 130/554] 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 f82364d2c9bb9d9484e1f5c1648ef9f1284e3633 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 21 Jul 2023 17:03:30 +0200 Subject: [PATCH 131/554] 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 538d1f7c173cae793c65e48dca039d2460de1e4a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 18 Jul 2023 18:55:34 +0200 Subject: [PATCH 132/554] 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 133/554] 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 134/554] 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 135/554] 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 136/554] 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 8c0113e02bb19a4137871f43bbbe9b00702f8681 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 4 Jul 2023 15:54:52 +0200 Subject: [PATCH 137/554] 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 138/554] 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 139/554] 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 c4a86c148adf5f4830050222950ac2e65b9ac11b Mon Sep 17 00:00:00 2001 From: flynn Date: Sat, 22 Jul 2023 17:47:27 +0000 Subject: [PATCH 140/554] 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 2e61e92c865f1080eb4f246ee8ac0d66e6a68a5d Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 23 Jul 2023 04:07:18 +0000 Subject: [PATCH 141/554] 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 f207e5bf1184e75c2769b687b9f5c08f85104384 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 24 Jul 2023 00:31:37 +0200 Subject: [PATCH 142/554] 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 eb8e7f49f7579016069ea1b4ca37c78eef1fb831 Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Thu, 13 Jul 2023 22:44:31 +0800 Subject: [PATCH 143/554] 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 144/554] 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 9ff409f16ed9ba875904651e353b80d1782d9b1d Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Mon, 24 Jul 2023 13:40:06 +0800 Subject: [PATCH 145/554] 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 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 146/554] 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 06b782d37a8a48f66db88a32f456e6cd9649b49b Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 26 Jul 2023 14:40:50 +0300 Subject: [PATCH 147/554] 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 0b5b0df58f22ca5ff6ef35ff74078856dbad09e2 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 26 Jul 2023 15:24:35 +0300 Subject: [PATCH 148/554] 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 149/554] 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 6205218e2b7a87c348cd8fb49f595cd36520286c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 26 Jul 2023 14:57:32 +0000 Subject: [PATCH 150/554] 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 151/554] 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 10a8a8dd74acfa9d8ea161c60a44ac7c5abff562 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 16 Jun 2023 23:24:10 -0400 Subject: [PATCH 152/554] 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 153/554] 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 154/554] 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 155/554] 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 156/554] 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 157/554] 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 adb46fa17498e02b1f92c71735f5ab175d4ff4bf Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 26 Jul 2023 15:20:54 +0000 Subject: [PATCH 158/554] 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 dc5cc0a5418a9aab424939055e8b67200e1f2996 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 26 Jul 2023 16:23:17 +0000 Subject: [PATCH 159/554] 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 bc86c26e4e75172fa8cb0665c26ff7b1228372a3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 26 Jul 2023 23:37:20 +0300 Subject: [PATCH 160/554] 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 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 161/554] 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 162/554] 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 163/554] 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 164/554] 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 eae86f603c24600df39f38f255a700c4acc7432f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 27 Jul 2023 06:34:09 +0200 Subject: [PATCH 165/554] 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 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 166/554] 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 167/554] 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 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 168/554] 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 169/554] 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 170/554] 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 e749938e92667ed3195c773726037d44c865993b Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 27 Jul 2023 12:07:34 +0000 Subject: [PATCH 171/554] 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 b5fc34b770cdbc555d1f9d45516af530da9ab5e5 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 27 Jul 2023 12:20:33 +0000 Subject: [PATCH 172/554] 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 bae407e9581aab765aa24f3cdcc18f823c665af5 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 27 Jul 2023 12:48:36 +0000 Subject: [PATCH 173/554] 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 96545e30edfcce414ed465de6c56ddc49600ab79 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 27 Jul 2023 17:07:34 +0200 Subject: [PATCH 174/554] 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 175/554] 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 176/554] 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 faca843ac0d2fd1b740d009a6cef1c9060346ff6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 27 Jul 2023 18:24:40 +0200 Subject: [PATCH 177/554] 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 d40dbdee6227cb0548f65abc900b1f7f91fd9959 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 27 Jun 2023 17:10:48 +0200 Subject: [PATCH 178/554] 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 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 179/554] 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 180/554] 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 6687f37329318c6995d795f8069ca5123e7bcf61 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 27 Jul 2023 18:11:52 +0000 Subject: [PATCH 181/554] 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 182/554] 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 183/554] 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 67b0993bdf8a5de1dd3a68db5c5082ba0a3ff759 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 27 Jul 2023 18:54:41 +0000 Subject: [PATCH 184/554] 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 185/554] 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 186/554] 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 187/554] 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 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 188/554] 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 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 189/554] 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 190/554] 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 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 191/554] 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 0e15f098d19577845197cd025a53390c26f001e6 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Jul 2023 02:09:57 -0400 Subject: [PATCH 192/554] 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 193/554] 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 17fa8a87a87791a52a4873a329b522cdc84dbb12 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 28 Jul 2023 08:54:25 +0000 Subject: [PATCH 194/554] Update libarchive --- contrib/libarchive | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libarchive b/contrib/libarchive index 30a8610f4d0..ee457961713 160000 --- a/contrib/libarchive +++ b/contrib/libarchive @@ -1 +1 @@ -Subproject commit 30a8610f4d05141d85bb9b123cdec16906a02c59 +Subproject commit ee45796171324519f0c0bfd012018dd099296336 From 8255e71554d52639075f66fdeba962f8742c1154 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 28 Jul 2023 11:39:06 +0200 Subject: [PATCH 195/554] Fix tests --- tests/queries/0_stateless/02833_url_without_path_encoding.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02833_url_without_path_encoding.sh b/tests/queries/0_stateless/02833_url_without_path_encoding.sh index 699b1c8e99b..b71586099cf 100755 --- a/tests/queries/0_stateless/02833_url_without_path_encoding.sh +++ b/tests/queries/0_stateless/02833_url_without_path_encoding.sh @@ -5,8 +5,8 @@ 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" +$CLICKHOUSE_CLIENT -q "select count() from url('http://localhost:11111/test%2Fa.tsv') settings enable_url_encoding=1" # 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 +$CLICKHOUSE_CLIENT -q "select count() from url('http://localhost:11111/test%2Fa.tsv') settings enable_url_encoding=0" 2>&1 | grep -o "test%2Fa.tsv" -m1 From e83e0ec2cd78a8bc09f34a7cde849531a59eafdb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 28 Jul 2023 11:55:23 +0000 Subject: [PATCH 196/554] 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 1fc1b6aae45800e3879245ea1abc3884c0ce0231 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 28 Jul 2023 13:00:35 +0000 Subject: [PATCH 197/554] 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 198/554] 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 1e77c7c73f3dc7db2e52d618999f909fcf3e9d97 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Jul 2023 10:48:22 -0400 Subject: [PATCH 199/554] 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 e0e6472195f87607a87da7207e20ea8609a1482f Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 28 Jul 2023 15:50:22 +0000 Subject: [PATCH 200/554] 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 201/554] 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 10a5230df4d8c352657327fdd75947acc269377a Mon Sep 17 00:00:00 2001 From: copperybean Date: Sat, 29 Jul 2023 11:20:12 +0800 Subject: [PATCH 202/554] 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 c270393495c1622c1e085c1abf750c18bfeb27d4 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 29 Jul 2023 14:56:43 +0000 Subject: [PATCH 203/554] 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 918571a5eecd08362265fb0bd9543a5920fcd96a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 29 Jul 2023 22:04:21 +0000 Subject: [PATCH 204/554] 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 6bcc421c0262fce9e2eade3cbc436b8241e4ae61 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 01:27:14 +0200 Subject: [PATCH 205/554] 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 9e0fb6b81da2a72c41a40f16dea245d1f6a16f35 Mon Sep 17 00:00:00 2001 From: copperybean Date: Sun, 30 Jul 2023 09:48:05 +0800 Subject: [PATCH 206/554] 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 d050a8fbd90c80dbb6be6ff20dc1e1844fd023ac Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Jul 2023 04:29:11 +0200 Subject: [PATCH 207/554] 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 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 208/554] 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 209/554] 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 210/554] 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 211/554] 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 6e99e5c4537c087abb735fa9312b238ad7389dda Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 30 Jul 2023 13:27:01 +0200 Subject: [PATCH 212/554] 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 6ae6268ad0075b1a99e619df599adc58dbcba848 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 30 Jul 2023 13:39:54 +0200 Subject: [PATCH 213/554] 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 214/554] 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 215/554] 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 216/554] 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 0ff0a4e41f39479530c1ef81388dc058266201b5 Mon Sep 17 00:00:00 2001 From: zvonand Date: Sun, 30 Jul 2023 19:42:10 +0200 Subject: [PATCH 217/554] 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 218/554] 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 219/554] 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 220/554] 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 6d971bc3a8985e8ddbaf243a01d1a68b872d5322 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 30 Jul 2023 22:35:19 +0000 Subject: [PATCH 221/554] 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 222/554] 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 36b5ccfa3ad355af55ee2a6fa1b82bc631af1289 Mon Sep 17 00:00:00 2001 From: zvonand Date: Mon, 31 Jul 2023 10:03:44 +0200 Subject: [PATCH 223/554] 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 224/554] 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 225/554] 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 226/554] 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 df456021651b7d9c4b4458a15060a24bbfee9d73 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 31 Jul 2023 08:50:09 +0000 Subject: [PATCH 227/554] 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 d136f18c3084b4af14c0bbf9ff8ef5028edf6ecf Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 Jul 2023 12:17:30 +0200 Subject: [PATCH 228/554] 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 af60b4bb313bc33771562b3dd830395548952041 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 31 Jul 2023 10:29:15 +0000 Subject: [PATCH 229/554] 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 230/554] 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 231/554] 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 232/554] 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 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 233/554] 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 964e4311c3950aa516b6cf435afc75f3e3245fb7 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 31 Jul 2023 13:03:56 +0000 Subject: [PATCH 234/554] 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 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 235/554] 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 236/554] 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 237/554] 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 e02805d9f777cddbc3d933c781aa6585efe3a6b3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 31 Jul 2023 16:55:51 +0300 Subject: [PATCH 238/554] 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 239/554] 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 240/554] 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 241/554] 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 242/554] 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 243/554] 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 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 244/554] 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 245/554] 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 246/554] 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 247/554] 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 248/554] 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 249/554] 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 250/554] 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 251/554] 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 252/554] 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 253/554] 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 254/554] 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 255/554] 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 e5240f900b2f0c05060ce538e0d07fe233f53883 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 1 Aug 2023 00:09:13 +0200 Subject: [PATCH 256/554] 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 1fcbf5f96d1fb13c118c497fa0a2732629e99c75 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 07:33:14 +0200 Subject: [PATCH 257/554] 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 0374b3de3e80edeac7e0b8580d32f313e2734d30 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 1 Aug 2023 09:04:55 +0200 Subject: [PATCH 258/554] 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 259/554] 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 71e58699b5f3f8e61a8b748e2172bdf9ad00b4ac Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 1 Aug 2023 07:52:50 +0000 Subject: [PATCH 260/554] 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 261/554] 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 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 262/554] 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 263/554] 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 264/554] 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 265/554] 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 266/554] 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 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 267/554] 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 268/554] 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 269/554] 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 270/554] 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 271/554] 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 272/554] 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 273/554] 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 274/554] 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 275/554] 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 276/554] 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 277/554] 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 278/554] 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 279/554] 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 280/554] 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 281/554] 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 282/554] 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 283/554] 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 284/554] 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 09cb5cbe8704f158b743b8c355e87aaf512b88e0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 1 Aug 2023 14:30:41 +0000 Subject: [PATCH 285/554] 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 4e849a8abac2baec0156c9d306607422b31a09c7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 1 Aug 2023 16:44:13 +0200 Subject: [PATCH 286/554] 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 287/554] 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 288/554] 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 289/554] 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 290/554] 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 291/554] 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 292/554] 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 293/554] 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 294/554] 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 295/554] 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 296/554] 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 297/554] 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 298/554] 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 aaba48f0df13aa9c314d604affc1ac491c4db5dd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Jul 2023 17:19:33 +0000 Subject: [PATCH 299/554] 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 300/554] 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 301/554] 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 625b6b2b1c8dc5c1375ab74ae2bb758ecb74d081 Mon Sep 17 00:00:00 2001 From: Hendrik M Date: Tue, 1 Aug 2023 22:33:42 +0200 Subject: [PATCH 302/554] 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 303/554] 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 304/554] 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 305/554] 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 306/554] 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 307/554] 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 308/554] 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 309/554] 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 310/554] 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 311/554] 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 312/554] 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 313/554] empty commit From 0174244fe12666f3792586a641c46dab1f597ebf Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 2 Aug 2023 11:21:47 +0000 Subject: [PATCH 314/554] 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 315/554] 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 316/554] 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 317/554] 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 318/554] 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 319/554] 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 320/554] 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 321/554] 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 322/554] 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 323/554] 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 324/554] 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 325/554] 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 326/554] 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 85ba27a2d4b5aa2fc80c90b7393ab327da8e79dc Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 2 Aug 2023 18:22:48 +0200 Subject: [PATCH 327/554] 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 328/554] 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 329/554] 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 330/554] 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 331/554] 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 332/554] 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 333/554] 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 334/554] 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 6dcdc4cd380c773a9dd135c66cdb79995ba0550a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 2 Aug 2023 15:35:47 +0200 Subject: [PATCH 335/554] 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 336/554] 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 fe09a36f4447e4a2cd141b3cac9fb2b0ef995b16 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 2 Aug 2023 18:07:54 +0000 Subject: [PATCH 337/554] 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 338/554] 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 339/554] 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 340/554] 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 341/554] 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 342/554] 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 343/554] 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 344/554] 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 345/554] 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 346/554] 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 48d736f8fdd7bfac6581de7702527445fd7b05bd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 2 Aug 2023 22:34:18 +0200 Subject: [PATCH 347/554] 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 348/554] 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 bde87997155abcb3014b67d78bfde0a29905a5ce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Aug 2023 00:16:04 +0200 Subject: [PATCH 349/554] 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 350/554] 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 351/554] 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 352/554] 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 353/554] 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 354/554] 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 355/554] 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 356/554] 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 357/554] 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 358/554] 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 359/554] 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 360/554] 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 361/554] 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 362/554] 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 363/554] 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 364/554] 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 365/554] 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 366/554] 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 367/554] 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 368/554] 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 369/554] 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 370/554] 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 371/554] 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 372/554] 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 373/554] 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 374/554] 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 375/554] 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 376/554] 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 377/554] 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 378/554] 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 379/554] 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 380/554] 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 381/554] 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 382/554] 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 383/554] 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 384/554] 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 385/554] 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 386/554] 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 387/554] 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 388/554] 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 389/554] 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 390/554] 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 f0eb22ac5f5cd64dec77550e417db06793fe0d94 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 3 Aug 2023 14:20:19 +0000 Subject: [PATCH 391/554] 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 392/554] 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 393/554] 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 394/554] 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 395/554] 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 49f76b8600371948cf77c8fd59d2ab1c12fcd251 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 3 Aug 2023 18:02:32 +0200 Subject: [PATCH 396/554] 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 397/554] 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 398/554] 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 8a7870f2504b3f3f1eed257d5aa3fc87e868a317 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 3 Aug 2023 20:24:29 +0200 Subject: [PATCH 399/554] 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 400/554] 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 401/554] 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 b53084ce43e1714ae89df29dca692ef53e184838 Mon Sep 17 00:00:00 2001 From: hendrik-m Date: Thu, 3 Aug 2023 22:07:05 +0200 Subject: [PATCH 402/554] 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 403/554] 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 404/554] 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 405/554] 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 406/554] 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 407/554] 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 408/554] 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 409/554] 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 410/554] 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 411/554] 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 412/554] 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 413/554] 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 414/554] 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 415/554] 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 416/554] 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 417/554] 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 418/554] 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 419/554] 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 420/554] 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 421/554] 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 422/554] 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 423/554] 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 424/554] 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 425/554] 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 426/554] 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 427/554] 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 428/554] 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 429/554] 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 430/554] 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 431/554] 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 432/554] 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 433/554] 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 434/554] 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 435/554] 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 436/554] 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 437/554] 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 438/554] 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 439/554] 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 440/554] 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 441/554] 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 442/554] 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 443/554] 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 cc5ec9e6345fbbf154536b3a7a95f3ea9b6452d1 Mon Sep 17 00:00:00 2001 From: Maximilian Roos Date: Fri, 4 Aug 2023 11:34:00 -0700 Subject: [PATCH 444/554] 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 445/554] 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 446/554] 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 9a128fa5544cc0cf411ee367d28a564a5f7f8b98 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 4 Aug 2023 20:42:02 +0000 Subject: [PATCH 447/554] 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 448/554] 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 449/554] 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 450/554] 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 451/554] 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 452/554] 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 453/554] 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 712ac83542ad65b41d8bef73a734c283539c1b68 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Sat, 5 Aug 2023 01:01:23 +0000 Subject: [PATCH 454/554] 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 455/554] 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 1d793e69f25e99ba62a5f31ebcbd90bd27b67ce5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Aug 2023 14:29:49 +0000 Subject: [PATCH 456/554] 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 457/554] 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 458/554] 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 459/554] 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 fd5880920750fe667e4a50a7fc5df13270c16800 Mon Sep 17 00:00:00 2001 From: Tanay Tummalapalli Date: Tue, 20 Jun 2023 02:00:07 +0530 Subject: [PATCH 460/554] Fix description for filtering sets in `full_sorting_merge` join The description of the `ReadHeadBalancedProcessor` and `FilterBySetOnTheFlyTransform` was the same as the `CreatingSetsOnTheFlyTransform` in an `EXPLAIN PIPELINE` when using `full_sorting_merge`. --- src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index c54d32c1385..c5b5dad5aa5 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -141,7 +141,7 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi /// Add balancing transform auto idx = position == JoinTableSide::Left ? PingPongProcessor::First : PingPongProcessor::Second; auto stream_balancer = std::make_shared(input_header, num_ports, max_rows_in_set, idx); - stream_balancer->setDescription(getStepDescription()); + stream_balancer->setDescription("Reads rows from two streams evenly"); /// Regular inputs just bypass data for respective ports connectAllInputs(ports, stream_balancer->getInputs(), num_ports); @@ -163,7 +163,7 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi { auto & port = *output_it++; auto transform = std::make_shared(port.getHeader(), column_names, filtering_set); - transform->setDescription(this->getStepDescription()); + transform->setDescription("Filter rows using other join table side's set"); connect(port, transform->getInputPort()); result_transforms.emplace_back(std::move(transform)); } From 2741300585de0e28b0cfc270379074037606fe00 Mon Sep 17 00:00:00 2001 From: Tanay Tummalapalli Date: Sun, 25 Jun 2023 02:33:57 +0530 Subject: [PATCH 461/554] Add test EXPLAIN PIPELINE desription in full_sorting_merge --- .../02383_join_and_filtering_set.reference | 9 +++++ .../02383_join_and_filtering_set.sh | 39 +++++++++++++++++-- 2 files changed, 45 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02383_join_and_filtering_set.reference b/tests/queries/0_stateless/02383_join_and_filtering_set.reference index 2ad282ca07f..822d76aafc6 100644 --- a/tests/queries/0_stateless/02383_join_and_filtering_set.reference +++ b/tests/queries/0_stateless/02383_join_and_filtering_set.reference @@ -8,3 +8,12 @@ Ok Ok Ok Ok +Ok +Ok +Ok +Ok +Ok +Ok +Ok +Ok +Ok diff --git a/tests/queries/0_stateless/02383_join_and_filtering_set.sh b/tests/queries/0_stateless/02383_join_and_filtering_set.sh index 3356be58ff7..3a6d60811c9 100755 --- a/tests/queries/0_stateless/02383_join_and_filtering_set.sh +++ b/tests/queries/0_stateless/02383_join_and_filtering_set.sh @@ -10,6 +10,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT -mn -q """ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + CREATE TABLE t1 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y AS SELECT sipHash64(number, 't1_x') % 100 AS x, sipHash64(number, 't1_y') % 100 AS y FROM numbers(100); @@ -17,6 +20,26 @@ CREATE TABLE t2 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y AS SELECT sipHash64(number, 't2_x') % 100 AS x, sipHash64(number, 't2_y') % 100 AS y FROM numbers(100); """ +# Arguments: +# - Query result +# - Processor name +# - Expected description +# - Check first occurrence +function match_description() { + +QUERY_RESULT=$1 +PROCESSOR_NAME=$2 +EXPECTED_DESCRIPTION=$3 +CHECK_FIRST_OCCURRENCE=${4:-true} + +SED_EXPR="/$PROCESSOR_NAME/{ n; s/^[ \t]*Description: //; p" +[ $CHECK_FIRST_OCCURRENCE = true ] && SED_EXPR+="; q }" || SED_EXPR+=" }" + +DESC=$(sed -n "$SED_EXPR" <<< "$QUERY_RESULT") +[[ "$DESC" == "$EXPECTED_DESCRIPTION" ]] && echo "Ok" || echo "Fail: ReadHeadBalancedProcessor description '$DESC' != '$EXPECTED_DESCRIPTION' " + +} + # Arguments: # - value of max_rows_in_set_to_optimize_join # - join kind @@ -37,10 +60,20 @@ RES=$( EXPECTED_PIPELINE_STEPS=$4 RES=$( $CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join=${PARAM_VALUE} --join_algorithm='full_sorting_merge' \ - -q "EXPLAIN PIPELINE SELECT count() FROM t1 ${JOIN_KIND} JOIN t2 ON t1.x = t2.x" \ - | grep -o -e ReadHeadBalancedProcessor -e FilterBySetOnTheFlyTransform -e CreatingSetsOnTheFlyTransform | wc -l + -q "EXPLAIN PIPELINE SELECT count() FROM t1 ${JOIN_KIND} JOIN t2 ON t1.x = t2.x" ) -[ "$RES" -eq "$EXPECTED_PIPELINE_STEPS" ] && echo "Ok" || echo "Fail: $RES != $EXPECTED_PIPELINE_STEPS" + +# Count match +COUNT=$(echo "$RES" | grep -o -e ReadHeadBalancedProcessor -e FilterBySetOnTheFlyTransform -e CreatingSetsOnTheFlyTransform | wc -l) +[ "$COUNT" -eq "$EXPECTED_PIPELINE_STEPS" ] && echo "Ok" || echo "Fail: $COUNT != $EXPECTED_PIPELINE_STEPS" + +# Description matchers +if [ "$EXPECTED_PIPELINE_STEPS" -ne 0 ]; then + match_description "$RES" 'ReadHeadBalancedProcessor' 'Reads rows from two streams evenly' + match_description "$RES" 'FilterBySetOnTheFlyTransform' "Filter rows using other join table side\'s set" + match_description "$RES" 'CreatingSetsOnTheFlyTransform' 'Create set and filter Left joined stream +Create set and filter Right joined stream' false +fi } From f1f983a92b09a1ba41e7201f52781d567c013ed6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Aug 2023 13:34:11 +0200 Subject: [PATCH 462/554] Add clusters for running tests locally easily --- programs/server/config.d/clusters.xml | 1 + 1 file changed, 1 insertion(+) create mode 120000 programs/server/config.d/clusters.xml diff --git a/programs/server/config.d/clusters.xml b/programs/server/config.d/clusters.xml new file mode 120000 index 00000000000..97289377736 --- /dev/null +++ b/programs/server/config.d/clusters.xml @@ -0,0 +1 @@ +../../../tests/config/config.d/clusters.xml \ No newline at end of file From ade6e53760000a65facf48ed71c9401b0de3986e Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Sun, 6 Aug 2023 21:46:06 +0200 Subject: [PATCH 463/554] Update README.md --- README.md | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index f0a7dbe2408..4ff9b9caaa1 100644 --- a/README.md +++ b/README.md @@ -23,11 +23,8 @@ curl https://clickhouse.com/ | sh ## Upcoming Events -* [**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 -* [**ClickHouse Meetup in Singapore**](https://www.meetup.com/clickhouse-singapore-meetup-group/events/294428050/) - Jul 27 +* [**v23.8 Community Call**](https://clickhouse.com/company/events/v23-8-community-release-call?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-08) - Aug 31 - 23.8 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. +* [**ClickHouse & AI - A Meetup in San Francisco**](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/294472987) - Aug 8 * [**ClickHouse Meetup in Paris**](https://www.meetup.com/clickhouse-france-user-group/events/294283460) - Sep 12 Also, keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler clickhouse com. From d977e221b42c74dc131e8e741bff86eb1ab3f146 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Aug 2023 03:26:59 +0200 Subject: [PATCH 464/554] Upload build time-trace data to CI database --- .github/workflows/backport_branches.yml | 3 ++ .github/workflows/master.yml | 3 ++ .github/workflows/pull_request.yml | 3 ++ .github/workflows/release_branches.yml | 3 ++ docker/packager/binary/build.sh | 9 ++++-- docker/packager/packager | 6 +++- tests/ci/build_check.py | 32 ++++++++++++++++++- .../prepare-time-trace/prepare-time-trace.sh | 3 +- 8 files changed, 56 insertions(+), 6 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index d69168b01ee..1df8cea4ad5 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -3,6 +3,9 @@ name: BackportPR env: # Force the stdout and stderr streams to be unbuffered PYTHONUNBUFFERED: 1 + # Export system tables to ClickHouse Cloud + CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }} + CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }} on: # yamllint disable-line rule:truthy push: diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index ae1862e327f..f25611e8666 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -3,6 +3,9 @@ name: MasterCI env: # Force the stdout and stderr streams to be unbuffered PYTHONUNBUFFERED: 1 + # Export system tables to ClickHouse Cloud + CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }} + CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }} on: # yamllint disable-line rule:truthy push: diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index d97b9975c3c..25927954809 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -3,6 +3,9 @@ name: PullRequestCI env: # Force the stdout and stderr streams to be unbuffered PYTHONUNBUFFERED: 1 + # Export system tables to ClickHouse Cloud + CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }} + CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }} on: # yamllint disable-line rule:truthy pull_request: diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 21284815583..e5527177aa5 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -3,6 +3,9 @@ name: ReleaseBranchCI env: # Force the stdout and stderr streams to be unbuffered PYTHONUNBUFFERED: 1 + # Export system tables to ClickHouse Cloud + CLICKHOUSE_CI_LOGS_HOST: ${{ secrets.CLICKHOUSE_CI_LOGS_HOST }} + CLICKHOUSE_CI_LOGS_PASSWORD: ${{ secrets.CLICKHOUSE_CI_LOGS_PASSWORD }} on: # yamllint disable-line rule:truthy push: diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 38b2ef75d24..ba284a7a94b 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -74,10 +74,10 @@ then rm -f CMakeCache.txt # Build the rest of binaries - cmake --debug-trycompile -DBUILD_STANDALONE_KEEPER=0 -DCREATE_KEEPER_SYMLINK=0 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. + cmake --debug-trycompile -DBUILD_STANDALONE_KEEPER=0 -DCREATE_KEEPER_SYMLINK=0 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_BUILD_PROFILING=1 "${CMAKE_FLAGS[@]}" .. else # Build everything - cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. + cmake --debug-trycompile -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 -DENABLE_BUILD_PROFILING=1 "${CMAKE_FLAGS[@]}" .. fi # No quotes because I want it to expand to nothing if empty. @@ -181,4 +181,9 @@ then tar -cv -I pixz -f /output/ccache.log.txz "$CCACHE_LOGFILE" fi +# Prepare profile info (time-trace) +mkdir -p /profile-tmp +../utils/prepare-time-trace/prepare-time-trace.sh . /profile-tmp +cat /profile-tmp/* > /profile/profile.json + ls -l /output diff --git a/docker/packager/packager b/docker/packager/packager index b6b9682f418..f70a8f41746 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -78,11 +78,14 @@ def run_docker_image_with_env( image_name: str, as_root: bool, output_dir: Path, + profile_dir: Path, env_variables: List[str], ch_root: Path, ccache_dir: Optional[Path], ): output_dir.mkdir(parents=True, exist_ok=True) + profile_dir.mkdir(parents=True, exist_ok=True) + env_part = " -e ".join(env_variables) if env_part: env_part = " -e " + env_part @@ -103,7 +106,7 @@ def run_docker_image_with_env( cmd = ( f"docker run --network=host --user={user} --rm {ccache_mount}" - f"--volume={output_dir}:/output --volume={ch_root}:/build {env_part} " + f"--volume={output_dir}:/output --volume={ch_root}:/build --volume={profile_dir}:/profile {env_part} " f"{interactive} {image_name}" ) @@ -488,6 +491,7 @@ def main(): image_with_version, args.as_root, args.output_dir, + args.profile_dir, env_prepared, ch_root, args.ccache_dir, diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 592c27c2c68..25a8efa5159 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -6,6 +6,8 @@ import json import os import sys import time +import urllib.parse +import requests from typing import List, Tuple from ci_config import CI_CONFIG, BuildConfig @@ -30,6 +32,7 @@ from version_helper import ( from clickhouse_helper import ( ClickHouseHelper, prepare_tests_results_for_clickhouse, + get_instance_type, ) from stopwatch import Stopwatch @@ -51,6 +54,7 @@ def get_packager_cmd( build_config: BuildConfig, packager_path: str, output_path: str, + profile_path: str, build_version: str, image_version: str, official: bool, @@ -59,7 +63,7 @@ def get_packager_cmd( 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"cd {packager_path} && CMAKE_FLAGS='{cmake_flags}' ./packager --output-dir={output_path} --profile-dir={profile_path}" f"--package-type={package_type} --compiler={comp}" ) @@ -286,10 +290,15 @@ def main(): if not os.path.exists(build_output_path): os.makedirs(build_output_path) + build_profile_path = os.path.join(TEMP_PATH, f"{build_name}_profile") + if not os.path.exists(build_profile_path): + os.makedirs(build_profile_path) + packager_cmd = get_packager_cmd( build_config, os.path.join(REPO_COPY, "docker/packager"), build_output_path, + build_profile_path, version.string, image_version, official_flag, @@ -360,6 +369,27 @@ def main(): upload_master_static_binaries(pr_info, build_config, s3_helper, build_output_path) + # Upload profile data + + instance_type = get_instance_type() + query = urllib.parse.quote(f""" + INSERT INTO build_time_trace (pull_request_number, commit_sha, check_start_time, check_name, instance_type, file, library, time, pid, tid, ph, ts, dur, cat, name, detail, count, avgMs, args_name) + SELECT {pr_info.number}, '{pr_info.sha}', '{stopwatch.start_time_str}', '{build_name}', '{instance_type}', * + FROM input('file String, library String, time DateTime64(6), pid UInt32, tid UInt32, ph String, ts UInt64, dur UInt64, cat String, name String, detail String, count UInt64, avgMs UInt64, args_name String') + FORMAT JSONEachRow + """) + clickhouse_ci_logs_host = os.getenv("CLICKHOUSE_CI_LOGS_HOST") + clickhouse_ci_logs_password = os.getenv("CLICKHOUSE_CI_LOGS_PASSWORD") + url = f"https://ci:{clickhouse_ci_logs_password}@{clickhouse_ci_logs_host}/?query={query}" + file_path = os.path.join(build_profile_path, "profile.json") + + print(f"::notice ::Log Uploading profile data, path: {file_path}, query: {query}") + + with open(file_path, 'rb') as file: + response = requests.post(url, data=file) + + # Upload statistics to CI database + ch_helper = ClickHouseHelper() prepared_events = prepare_tests_results_for_clickhouse( pr_info, diff --git a/utils/prepare-time-trace/prepare-time-trace.sh b/utils/prepare-time-trace/prepare-time-trace.sh index 300a32b0fd1..7cacdec8c94 100755 --- a/utils/prepare-time-trace/prepare-time-trace.sh +++ b/utils/prepare-time-trace/prepare-time-trace.sh @@ -35,7 +35,6 @@ ENGINE = MergeTree ORDER BY (date, file, name, args_name); INPUT_DIR=$1 OUTPUT_DIR=$2 -EXTRA_COLUMN_VALUES=$3 find "$INPUT_DIR" -name '*.json' | grep -P '\.(c|cpp|cc|cxx)\.json$' | xargs -P $(nproc) -I{} bash -c " @@ -43,7 +42,7 @@ find "$INPUT_DIR" -name '*.json' | grep -P '\.(c|cpp|cc|cxx)\.json$' | xargs -P LIBRARY_NAME=\$(echo '{}' | sed -r -e 's!^.*/CMakeFiles/([^/]+)\.dir/.*\$!\1!') START_TIME=\$(jq '.beginningOfTime' '{}') - jq -c '.traceEvents[] | [${EXTRA_COLUMN_VALUES} \"'\"\$ORIGINAL_FILENAME\"'\", \"'\"\$LIBRARY_NAME\"'\", '\$START_TIME', .pid, .tid, .ph, .ts, .dur, .cat, .name, .args.detail, .args.count, .args[\"avg ms\"], .args.name]' '{}' > \"${OUTPUT_DIR}/\$\$\" + jq -c '.traceEvents[] | [\"'\"\$ORIGINAL_FILENAME\"'\", \"'\"\$LIBRARY_NAME\"'\", '\$START_TIME', .pid, .tid, .ph, .ts, .dur, .cat, .name, .args.detail, .args.count, .args[\"avg ms\"], .args.name]' '{}' > \"${OUTPUT_DIR}/\$\$\" " # Now you can upload it as follows: From 19ad81dabd8c3964293acc95ada758c9d03673ee Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 7 Aug 2023 01:50:29 +0000 Subject: [PATCH 465/554] Automatic style fix --- tests/ci/build_check.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 25a8efa5159..515c5a4943b 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -372,12 +372,14 @@ def main(): # Upload profile data instance_type = get_instance_type() - query = urllib.parse.quote(f""" + query = urllib.parse.quote( + f""" INSERT INTO build_time_trace (pull_request_number, commit_sha, check_start_time, check_name, instance_type, file, library, time, pid, tid, ph, ts, dur, cat, name, detail, count, avgMs, args_name) SELECT {pr_info.number}, '{pr_info.sha}', '{stopwatch.start_time_str}', '{build_name}', '{instance_type}', * FROM input('file String, library String, time DateTime64(6), pid UInt32, tid UInt32, ph String, ts UInt64, dur UInt64, cat String, name String, detail String, count UInt64, avgMs UInt64, args_name String') FORMAT JSONEachRow - """) + """ + ) clickhouse_ci_logs_host = os.getenv("CLICKHOUSE_CI_LOGS_HOST") clickhouse_ci_logs_password = os.getenv("CLICKHOUSE_CI_LOGS_PASSWORD") url = f"https://ci:{clickhouse_ci_logs_password}@{clickhouse_ci_logs_host}/?query={query}" @@ -385,7 +387,7 @@ def main(): print(f"::notice ::Log Uploading profile data, path: {file_path}, query: {query}") - with open(file_path, 'rb') as file: + with open(file_path, "rb") as file: response = requests.post(url, data=file) # Upload statistics to CI database From ed2a1d7c9b30b7aea111cffd56e682156ec39af1 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 7 Aug 2023 03:15:20 +0000 Subject: [PATCH 466/554] select required columns when getting join --- src/Interpreters/ExpressionAnalyzer.cpp | 10 +++++++++- src/Planner/PlannerJoins.cpp | 4 +++- src/Storages/StorageJoin.cpp | 6 ++---- src/Storages/StorageJoin.h | 2 +- .../performance/storage_join_direct_join.xml | 19 +++++++++++++++++++ 5 files changed, 34 insertions(+), 7 deletions(-) create mode 100644 tests/performance/storage_join_direct_join.xml diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 9aee61eb8f0..acedede579b 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1130,9 +1130,17 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin( if (auto storage = analyzed_join->getStorageJoin()) { + auto joined_block_actions = analyzed_join->createJoinedBlockActions(getContext()); + NamesWithAliases required_columns_with_aliases = analyzed_join->getRequiredColumns( + Block(joined_block_actions->getResultColumns()), joined_block_actions->getRequiredColumns().getNames()); + + Names original_right_column_names; + for (auto & pr : required_columns_with_aliases) + original_right_column_names.push_back(pr.first); + auto right_columns = storage->getRightSampleBlock().getColumnsWithTypeAndName(); std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, right_columns); - return storage->getJoinLocked(analyzed_join, getContext()); + return storage->getJoinLocked(analyzed_join, getContext(), original_right_column_names); } joined_plan = buildJoinedPlan(getContext(), join_element, *analyzed_join, query_options); diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index e495b0967e9..e87930a4d6b 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -635,6 +635,7 @@ std::shared_ptr chooseJoinAlgorithm(std::shared_ptr & table_jo /// JOIN with JOIN engine. if (auto storage = table_join->getStorageJoin()) { + Names required_column_names; for (const auto & result_column : right_table_expression_header) { const auto * source_column_name = right_table_expression_data.getColumnNameOrNull(result_column.name); @@ -644,8 +645,9 @@ std::shared_ptr chooseJoinAlgorithm(std::shared_ptr & table_jo fmt::join(storage->getKeyNames(), ", "), result_column.name); table_join->setRename(*source_column_name, result_column.name); + required_column_names.push_back(*source_column_name); } - return storage->getJoinLocked(table_join, planner_context->getQueryContext()); + return storage->getJoinLocked(table_join, planner_context->getQueryContext(), required_column_names); } /** JOIN with constant. diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index cc64a1a15fc..121d859a3f2 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -178,7 +178,7 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) } } -HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, ContextPtr context) const +HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, ContextPtr context, const Names & required_columns_names) const { auto metadata_snapshot = getInMemoryMetadataPtr(); if (!analyzed_join->sameStrictnessAndKind(strictness, kind)) @@ -239,10 +239,8 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, analyzed_join->setRightKeys(key_names); analyzed_join->setLeftKeys(left_key_names_resorted); Block right_sample_block; - for (const auto & name : getKeyNames()) + for (const auto & name : required_columns_names) 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); diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 5559b5d1ec8..4626d744a38 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -49,7 +49,7 @@ public: /// Return instance of HashJoin holding lock that protects from insertions to StorageJoin. /// HashJoin relies on structure of hash table that's why we need to return it with locked mutex. - HashJoinPtr getJoinLocked(std::shared_ptr analyzed_join, ContextPtr context) const; + HashJoinPtr getJoinLocked(std::shared_ptr analyzed_join, ContextPtr context, const Names & required_columns_names) const; /// Get result type for function "joinGet(OrNull)" DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const; diff --git a/tests/performance/storage_join_direct_join.xml b/tests/performance/storage_join_direct_join.xml new file mode 100644 index 00000000000..bdcf125d64f --- /dev/null +++ b/tests/performance/storage_join_direct_join.xml @@ -0,0 +1,19 @@ + + + 1 + + + CREATE TABLE keys (key UInt64) ENGINE = MergeTree ORDER BY key; + CREATE TABLE dict (key UInt64, value1 UInt64, value2 Float64, value3 String, + value4 String, value5 String, value6 String, value7 String, value8 String, value9 String, + value10 String) ENGINE = Join(ANY, LEFT, key); + + INSERT INTO keys SELECT rand() FROM numbers(10000000); + INSERT INTO dict SELECT rand(), rand()%1000, rand()*0.0001, toString(number), + toString(number), toString(number), toString(number), toString(number), toString(number), + toString(number), toString(number) FROM numbers(1000000); + + SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key); + SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) SETTINGS + allow_experimental_analyzer=1 + \ No newline at end of file From ad0ac4381427922c7d364298d1a9850cab13947d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 7 Aug 2023 06:25:46 +0000 Subject: [PATCH 467/554] fix performance test --- tests/performance/storage_join_direct_join.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/performance/storage_join_direct_join.xml b/tests/performance/storage_join_direct_join.xml index bdcf125d64f..2fc63c2c926 100644 --- a/tests/performance/storage_join_direct_join.xml +++ b/tests/performance/storage_join_direct_join.xml @@ -13,7 +13,7 @@ toString(number), toString(number), toString(number), toString(number), toString(number), toString(number), toString(number) FROM numbers(1000000); - SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key); - SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) SETTINGS + SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null; + SELECT keys.key, value1 FROM keys ANY LEFT JOIN dict AS d ON (keys.key = d.key) FORMAT Null SETTINGS allow_experimental_analyzer=1 \ No newline at end of file From f423b440d1aea63abfd9af2699b39a9e9b4e8f25 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Elena=20Torr=C3=B3?= Date: Mon, 7 Aug 2023 09:55:16 +0200 Subject: [PATCH 468/554] Add optional parameters to Engine definition --- docs/en/engines/table-engines/special/buffer.md | 2 +- docs/ru/engines/table-engines/special/buffer.md | 2 +- docs/zh/engines/table-engines/special/buffer.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/special/buffer.md b/docs/en/engines/table-engines/special/buffer.md index f7d84b9b452..a3bb11117cd 100644 --- a/docs/en/engines/table-engines/special/buffer.md +++ b/docs/en/engines/table-engines/special/buffer.md @@ -13,7 +13,7 @@ A recommended alternative to the Buffer Table Engine is enabling [asynchronous i ::: ``` sql -Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes) +Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes [,flush_time [,flush_rows [,flush_bytes]]]) ``` ### Engine parameters: diff --git a/docs/ru/engines/table-engines/special/buffer.md b/docs/ru/engines/table-engines/special/buffer.md index 574d9273088..a4e58e66e0c 100644 --- a/docs/ru/engines/table-engines/special/buffer.md +++ b/docs/ru/engines/table-engines/special/buffer.md @@ -9,7 +9,7 @@ sidebar_label: Buffer Буферизует записываемые данные в оперативке, периодически сбрасывая их в другую таблицу. При чтении, производится чтение данных одновременно из буфера и из другой таблицы. ``` sql -Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes) +Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes [,flush_time [,flush_rows [,flush_bytes]]]) ``` Параметры движка: diff --git a/docs/zh/engines/table-engines/special/buffer.md b/docs/zh/engines/table-engines/special/buffer.md index bb95ecdc583..f92a819f3c3 100644 --- a/docs/zh/engines/table-engines/special/buffer.md +++ b/docs/zh/engines/table-engines/special/buffer.md @@ -5,7 +5,7 @@ slug: /zh/engines/table-engines/special/buffer 缓冲数据写入 RAM 中,周期性地将数据刷新到另一个表。在读取操作时,同时从缓冲区和另一个表读取数据。 - Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes) + Buffer(database, table, num_layers, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes [,flush_time [,flush_rows [,flush_bytes]]]) 引擎的参数:database,table - 要刷新数据的表。可以使用返回字符串的常量表达式而不是数据库名称。 num_layers - 并行层数。在物理上,该表将表示为 num_layers 个独立缓冲区。建议值为16。min_time,max_time,min_rows,max_rows,min_bytes,max_bytes - 从缓冲区刷新数据的条件。 From 55c3581242c139daa746276b19561d817f848965 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 7 Aug 2023 10:27:10 +0200 Subject: [PATCH 469/554] Replace Upgrade check (debug) with a less flaky upgrade check --- 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 ed61de05231..979f7c35c2c 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -372,7 +372,7 @@ REQUIRED_CHECKS = [ "Stress test (msan)", "Stress test (tsan)", "Stress test (ubsan)", - "Upgrade check (debug)", + "Upgrade check (asan)", "Style Check", "Unit tests (asan)", "Unit tests (msan)", From 30d1452bbc00a30c003b1d9da61ab3c5fcfceeb7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 7 Aug 2023 10:48:24 +0200 Subject: [PATCH 470/554] Compatibility with clang-17 --- src/Storages/StorageFile.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index e3908c75a58..5c03540de9a 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -396,9 +396,9 @@ std::unique_ptr createReadBuffer( 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 reader->readFile([my_matcher = std::move(matcher)](const std::string & path) { - return re2::RE2::FullMatch(path, *matcher); + return re2::RE2::FullMatch(path, *my_matcher); }); } else From 11b5a3dd2249adf3cbb8fdd871a6713145c979ad Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 7 Aug 2023 11:46:14 +0200 Subject: [PATCH 471/554] Update TableFunctionRemote.cpp --- 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 d00cda2462f..1f42ce4ba30 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -223,7 +223,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr if (arg_num < args.size()) { if (sharding_key) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "user and password should be string literal (in single quotes)"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Arguments `user` and `password` should be string literals (in single quotes)"); sharding_key = args[arg_num]; ++arg_num; } From 80b681022c1715da0672f8c3da287fffa2335425 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 7 Aug 2023 12:20:44 +0200 Subject: [PATCH 472/554] Remove duplicate test --- tests/integration/test_multiple_disks/test.py | 147 ------------------ 1 file changed, 147 deletions(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 17621d09422..ec3af74456b 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -1165,153 +1165,6 @@ def produce_alter_move(node, name): except QueryRuntimeException as ex: pass - -@pytest.mark.parametrize( - "name,engine", - [ - pytest.param("concurrently_altering_mt", "MergeTree()", id="mt"), - pytest.param( - "concurrently_altering_replicated_mt", - "ReplicatedMergeTree('/clickhouse/concurrently_altering_replicated_mt', '1')", - id="replicated", - ), - ], -) -def test_concurrent_alter_move(start_cluster, name, engine): - try: - node1.query_with_retry( - """ - CREATE TABLE IF NOT EXISTS {name} ( - EventDate Date, - number UInt64 - ) ENGINE = {engine} - ORDER BY tuple() - PARTITION BY toYYYYMM(EventDate) - SETTINGS storage_policy='jbods_with_external' - """.format( - name=name, engine=engine - ) - ) - - values = list({random.randint(1, 1000000) for _ in range(0, 1000)}) - - def insert(num): - for i in range(num): - day = random.randint(11, 30) - value = values.pop() - month = "0" + str(random.choice([3, 4])) - node1.query_with_retry( - "INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format( - name, m=month, d=day, v=value - ) - ) - - def alter_move(num): - for i in range(num): - produce_alter_move(node1, name) - - def alter_update(num): - for i in range(num): - node1.query( - "ALTER TABLE {} UPDATE number = number + 1 WHERE 1".format(name) - ) - - def optimize_table(num): - for i in range(num): - node1.query_with_retry("OPTIMIZE TABLE {} FINAL".format(name)) - - p = Pool(15) - tasks = [] - for i in range(5): - tasks.append(p.apply_async(insert, (100,))) - tasks.append(p.apply_async(alter_move, (100,))) - tasks.append(p.apply_async(alter_update, (100,))) - tasks.append(p.apply_async(optimize_table, (100,))) - - for task in tasks: - task.get(timeout=240) - - assert node1.query("SELECT 1") == "1\n" - assert node1.query("SELECT COUNT() FROM {}".format(name)) == "500\n" - finally: - node1.query(f"DROP TABLE IF EXISTS {name} SYNC") - - -@pytest.mark.parametrize( - "name,engine", - [ - pytest.param("concurrently_dropping_mt", "MergeTree()", id="mt"), - pytest.param( - "concurrently_dropping_replicated_mt", - "ReplicatedMergeTree('/clickhouse/concurrently_dropping_replicated_mt', '1')", - id="replicated", - ), - ], -) -def test_concurrent_alter_move_and_drop(start_cluster, name, engine): - try: - node1.query( - """ - CREATE TABLE IF NOT EXISTS {name} ( - EventDate Date, - number UInt64 - ) ENGINE = {engine} - ORDER BY tuple() - PARTITION BY toYYYYMM(EventDate) - SETTINGS storage_policy='jbods_with_external' - """.format( - name=name, engine=engine - ) - ) - - values = list({random.randint(1, 1000000) for _ in range(0, 1000)}) - - def insert(num): - for i in range(num): - day = random.randint(11, 30) - value = values.pop() - month = "0" + str(random.choice([3, 4])) - node1.query_with_retry( - "INSERT INTO {} VALUES(toDate('2019-{m}-{d}'), {v})".format( - name, m=month, d=day, v=value - ) - ) - - def alter_move(num): - for i in range(num): - produce_alter_move(node1, name) - - def alter_drop(num): - for i in range(num): - partition = random.choice([201903, 201904]) - op = random.choice(["drop", "detach"]) - try: - node1.query( - "ALTER TABLE {} {} PARTITION {}".format(name, op, partition) - ) - except QueryRuntimeException as e: - if "Code: 650" in e.stderr: - pass - else: - raise e - - insert(20) - p = Pool(15) - tasks = [] - for i in range(5): - tasks.append(p.apply_async(insert, (20,))) - tasks.append(p.apply_async(alter_move, (20,))) - tasks.append(p.apply_async(alter_drop, (20,))) - - for task in tasks: - task.get(timeout=120) - - assert node1.query("SELECT 1") == "1\n" - - finally: - node1.query_with_retry(f"DROP TABLE IF EXISTS {name} SYNC") - - @pytest.mark.parametrize( "name,engine", [ From 8dc360e4c7828e60d6d7d8b8814e1b1306e09ca5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 7 Aug 2023 10:31:41 +0000 Subject: [PATCH 473/554] Automatic style fix --- tests/integration/test_multiple_disks/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index ec3af74456b..30669feb6b3 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -1165,6 +1165,7 @@ def produce_alter_move(node, name): except QueryRuntimeException as ex: pass + @pytest.mark.parametrize( "name,engine", [ From 1c400432d9c2995450184a11c2d758f3fd18b80b Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 7 Aug 2023 10:52:55 +0000 Subject: [PATCH 474/554] fixed suggestions --- src/Functions/FunctionBinaryArithmetic.h | 6 ++---- .../0_stateless/02812_pointwise_array_operations.sql | 2 +- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index d4e414e1e54..fb9143db333 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -1408,9 +1408,9 @@ public: } - if (isArray(arguments[0]) && isArray(arguments[1])) + 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(), @@ -1418,8 +1418,6 @@ public: }; return std::make_shared(getReturnTypeImplStatic(new_arguments, context)); } - else - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot use this operation on arrays"); } diff --git a/tests/queries/0_stateless/02812_pointwise_array_operations.sql b/tests/queries/0_stateless/02812_pointwise_array_operations.sql index 9bd633be232..e28c4bda347 100644 --- a/tests/queries/0_stateless/02812_pointwise_array_operations.sql +++ b/tests/queries/0_stateless/02812_pointwise_array_operations.sql @@ -11,7 +11,7 @@ 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 ([12,13] % [5,6]); -- { serverError 43 } 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]); From c830ad0a8133a02e0abc984e836bdbb85521df66 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 7 Aug 2023 12:57:41 +0200 Subject: [PATCH 475/554] Fix --- tests/integration/test_merge_tree_s3/test.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index d7c267eed50..a2d71874729 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -855,6 +855,11 @@ def test_s3_engine_heavy_write_check_mem( memory = in_flight_memory[1] node = cluster.instances[node_name] + + # it's bad idea to test something related to memory with sanitizers + if node.is_built_with_sanitizer(): + pytest.skip("Disabled for sanitizers") + node.query("DROP TABLE IF EXISTS s3_test SYNC") node.query( "CREATE TABLE s3_test" From eb43d480f0d397a6880ff9f1d1e30faa3787d51d Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 7 Aug 2023 12:59:25 +0200 Subject: [PATCH 476/554] Relax flaky test --- tests/integration/test_merge_tree_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index a2d71874729..507f25209a4 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -892,7 +892,7 @@ def test_s3_engine_heavy_write_check_mem( assert int(memory_usage) < 1.2 * memory assert int(memory_usage) > 0.8 * memory - assert int(wait_inflight) > 10 * 1000 * 1000 + assert int(wait_inflight) > in_flight * 1000 * 1000 check_no_objects_after_drop(cluster, node_name=node_name) From d35dc460619519075f5fe51fbed09cc74a2745dd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 7 Aug 2023 14:39:58 +0300 Subject: [PATCH 477/554] Update PocoHTTPClient.cpp --- src/IO/S3/PocoHTTPClient.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index fd825720ac9..8ed50416f3f 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -536,7 +536,10 @@ void PocoHTTPClient::makeRequestInternalImpl( } catch (...) { - tryLogCurrentException(log, fmt::format("Failed to make request to: {}", uri)); + auto error_message = getCurrentExceptionMessageAndPattern(/* with_stacktrace */ true); + error_message.text = fmt::format("Failed to make request to: {}: {}", uri, error_message.text); + LOG_INFO(log, error_message); + response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); response->SetClientErrorMessage(getCurrentExceptionMessage(false)); From 46ef77b5cb216b5fb89db41a7d392404159d9762 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 7 Aug 2023 13:41:12 +0200 Subject: [PATCH 478/554] Update FunctionBinaryArithmetic.h --- src/Functions/FunctionBinaryArithmetic.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index fb9143db333..723abc7c308 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -71,7 +71,6 @@ 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 UNSUPPORTED_METHOD; extern const int SIZES_OF_ARRAYS_DONT_MATCH; } From c7996d54536492ebd4f436672e466464e8474ff9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 7 Aug 2023 14:48:52 +0300 Subject: [PATCH 479/554] increase election timeout in integration tests (#53052) --- tests/integration/helpers/keeper_config1.xml | 2 ++ tests/integration/helpers/keeper_config2.xml | 2 ++ tests/integration/helpers/keeper_config3.xml | 2 ++ 3 files changed, 6 insertions(+) diff --git a/tests/integration/helpers/keeper_config1.xml b/tests/integration/helpers/keeper_config1.xml index daacd55887d..f40ed9ac6fa 100644 --- a/tests/integration/helpers/keeper_config1.xml +++ b/tests/integration/helpers/keeper_config1.xml @@ -18,6 +18,8 @@ 15000 trace false + 2000 + 4000 diff --git a/tests/integration/helpers/keeper_config2.xml b/tests/integration/helpers/keeper_config2.xml index 53eb023dba5..d5bdb92a79d 100644 --- a/tests/integration/helpers/keeper_config2.xml +++ b/tests/integration/helpers/keeper_config2.xml @@ -18,6 +18,8 @@ 15000 trace false + 2000 + 4000 diff --git a/tests/integration/helpers/keeper_config3.xml b/tests/integration/helpers/keeper_config3.xml index 1db091c12bc..aa69b554660 100644 --- a/tests/integration/helpers/keeper_config3.xml +++ b/tests/integration/helpers/keeper_config3.xml @@ -18,6 +18,8 @@ 15000 trace false + 2000 + 4000 From f5597b1f8ee9f39291c8e36e2ca6a01cfe8ae07a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 7 Aug 2023 12:21:50 +0000 Subject: [PATCH 480/554] Revert changes from https://github.com/ClickHouse/ClickHouse/pull/53064 --- src/IO/ZstdDeflatingAppendableWriteBuffer.cpp | 115 ++++++++++++------ src/IO/ZstdDeflatingAppendableWriteBuffer.h | 2 - 2 files changed, 76 insertions(+), 41 deletions(-) diff --git a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp index 5c4ee6203b3..81be8d8ce4d 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.cpp @@ -42,13 +42,50 @@ 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; } - flush(ZSTD_e_flush); + 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; + } + } ZstdDeflatingAppendableWriteBuffer::~ZstdDeflatingAppendableWriteBuffer() @@ -66,58 +103,58 @@ void ZstdDeflatingAppendableWriteBuffer::finalizeImpl() } else { - finalizeBefore(); - out->finalize(); - finalizeAfter(); + try + { + finalizeBefore(); + out->finalize(); + finalizeAfter(); + } + catch (...) + { + /// Do not try to flush next time after exception. + out->position() = out->buffer().begin(); + throw; + } } } void ZstdDeflatingAppendableWriteBuffer::finalizeBefore() { - /// 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. - flush(ZSTD_e_end); -} + next(); + + out->nextIfAtEnd(); -void ZstdDeflatingAppendableWriteBuffer::flush(ZSTD_EndDirective mode) -{ input.src = reinterpret_cast(working_buffer.begin()); input.size = offset(); input.pos = 0; - try - { - bool ended = false; - do - { - out->nextIfAtEnd(); + 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) + { + 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()) + { + out->next(); 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 d082178142b..d9c4f32d6da 100644 --- a/src/IO/ZstdDeflatingAppendableWriteBuffer.h +++ b/src/IO/ZstdDeflatingAppendableWriteBuffer.h @@ -52,8 +52,6 @@ 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. From 981da23144259658d79d599eca35c4ed82ce1b1d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 7 Aug 2023 12:32:12 +0000 Subject: [PATCH 481/554] fix reading of empty Nested(Array(...)) --- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/IMergeTreeReader.cpp | 11 +++++++---- src/Storages/MergeTree/IMergeTreeReader.h | 4 ++-- .../MergeTree/MergeTreeBaseSelectProcessor.cpp | 17 +++++++---------- .../MergeTree/MergeTreeBaseSelectProcessor.h | 3 --- .../MergeTree/MergeTreeDataPartCompact.cpp | 4 ++-- .../MergeTree/MergeTreeDataPartCompact.h | 2 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 4 ++-- .../MergeTree/MergeTreeDataPartInMemory.h | 2 +- .../MergeTree/MergeTreeDataPartWide.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeDataPartWide.h | 2 +- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 2 +- .../MergeTree/MergeTreeReaderCompact.cpp | 11 +++++++---- src/Storages/MergeTree/MergeTreeReaderCompact.h | 2 +- .../MergeTree/MergeTreeReaderInMemory.cpp | 4 ++-- .../MergeTree/MergeTreeReaderInMemory.h | 2 +- src/Storages/MergeTree/MergeTreeReaderWide.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeReaderWide.h | 2 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 2 +- .../MergeTree/MergeTreeSequentialSource.cpp | 2 +- .../MergeTreeThreadSelectProcessor.cpp | 4 +--- 21 files changed, 44 insertions(+), 46 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index af6906e004d..9243c91987b 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -89,7 +89,7 @@ public: virtual MergeTreeReaderPtr getReader( const NamesAndTypesList & columns_, - const StorageMetadataPtr & metadata_snapshot, + const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 8acf6413a37..4bb8c400691 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -24,7 +24,7 @@ namespace ErrorCodes IMergeTreeReader::IMergeTreeReader( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, const NamesAndTypesList & columns_, - const StorageMetadataPtr & metadata_snapshot_, + const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, const MarkRanges & all_mark_ranges_, @@ -35,7 +35,7 @@ IMergeTreeReader::IMergeTreeReader( , uncompressed_cache(uncompressed_cache_) , mark_cache(mark_cache_) , settings(settings_) - , metadata_snapshot(metadata_snapshot_) + , storage_snapshot(storage_snapshot_) , all_mark_ranges(all_mark_ranges_) , alter_conversions(data_part_info_for_read->getAlterConversions()) /// For wide parts convert plain arrays of Nested to subcolumns @@ -71,7 +71,7 @@ void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_e res_columns, num_rows, Nested::convertToSubcolumns(requested_columns), Nested::convertToSubcolumns(available_columns), - partially_read_columns, metadata_snapshot); + partially_read_columns, storage_snapshot->metadata); should_evaluate_missing_defaults = std::any_of( res_columns.begin(), res_columns.end(), [](const auto & column) { return column == nullptr; }); @@ -110,7 +110,10 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns } auto dag = DB::evaluateMissingDefaults( - additional_columns, requested_columns, metadata_snapshot->getColumns(), data_part_info_for_read->getContext()); + additional_columns, requested_columns, + storage_snapshot->metadata->getColumns(), + data_part_info_for_read->getContext()); + if (dag) { dag->addMaterializingOutputActions(); diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index f6b6feb3d42..dafe7b2d4ba 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -24,7 +24,7 @@ public: IMergeTreeReader( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, const NamesAndTypesList & columns_, - const StorageMetadataPtr & metadata_snapshot_, + const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, const MarkRanges & all_mark_ranges_, @@ -92,7 +92,7 @@ protected: MergeTreeReaderSettings settings; - StorageMetadataPtr metadata_snapshot; + StorageSnapshotPtr storage_snapshot; MarkRanges all_mark_ranges; /// Position and level (of nesting). diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 3eba9a9de24..d7836ac01b0 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -191,7 +191,6 @@ ChunkAndProgress IMergeTreeSelectAlgorithm::read() } void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForCurrentTask( - const StorageMetadataPtr & metadata_snapshot, const IMergeTreeReader::ValueSizeMap & value_size_map, const ReadBufferFromFileBase::ProfileCallback & profile_callback) { @@ -206,7 +205,7 @@ void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForCurrentTask( else { reader = task->data_part->getReader( - task->task_columns.columns, metadata_snapshot, task->mark_ranges, + task->task_columns.columns, storage_snapshot, task->mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), task->alter_conversions, reader_settings, value_size_map, profile_callback); } @@ -222,8 +221,8 @@ void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForCurrentTask( { initializeMergeTreePreReadersForPart( task->data_part, task->alter_conversions, - task->task_columns, metadata_snapshot, - task->mark_ranges, value_size_map, profile_callback); + task->task_columns, task->mark_ranges, + value_size_map, profile_callback); } } @@ -231,18 +230,17 @@ void IMergeTreeSelectAlgorithm::initializeMergeTreeReadersForPart( const MergeTreeData::DataPartPtr & data_part, const AlterConversionsPtr & alter_conversions, const MergeTreeReadTaskColumns & task_columns, - const StorageMetadataPtr & metadata_snapshot, const MarkRanges & mark_ranges, const IMergeTreeReader::ValueSizeMap & value_size_map, const ReadBufferFromFileBase::ProfileCallback & profile_callback) { reader = data_part->getReader( - task_columns.columns, metadata_snapshot, mark_ranges, + task_columns.columns, storage_snapshot, mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), alter_conversions, reader_settings, value_size_map, profile_callback); initializeMergeTreePreReadersForPart( - data_part, alter_conversions, task_columns, metadata_snapshot, + data_part, alter_conversions, task_columns, mark_ranges, value_size_map, profile_callback); } @@ -250,7 +248,6 @@ void IMergeTreeSelectAlgorithm::initializeMergeTreePreReadersForPart( const MergeTreeData::DataPartPtr & data_part, const AlterConversionsPtr & alter_conversions, const MergeTreeReadTaskColumns & task_columns, - const StorageMetadataPtr & metadata_snapshot, const MarkRanges & mark_ranges, const IMergeTreeReader::ValueSizeMap & value_size_map, const ReadBufferFromFileBase::ProfileCallback & profile_callback) @@ -262,7 +259,7 @@ void IMergeTreeSelectAlgorithm::initializeMergeTreePreReadersForPart( { pre_reader_for_step.push_back( data_part->getReader( - {LightweightDeleteDescription::FILTER_COLUMN}, metadata_snapshot, + {LightweightDeleteDescription::FILTER_COLUMN}, storage_snapshot, mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), alter_conversions, reader_settings, value_size_map, profile_callback)); } @@ -271,7 +268,7 @@ void IMergeTreeSelectAlgorithm::initializeMergeTreePreReadersForPart( { pre_reader_for_step.push_back( data_part->getReader( - pre_columns_per_step, metadata_snapshot, mark_ranges, + pre_columns_per_step, storage_snapshot, mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), alter_conversions, reader_settings, value_size_map, profile_callback)); } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index 7b6dc50060a..42043c03e85 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -120,7 +120,6 @@ protected: /// Sets up data readers for each step of prewhere and where void initializeMergeTreeReadersForCurrentTask( - const StorageMetadataPtr & metadata_snapshot, const IMergeTreeReader::ValueSizeMap & value_size_map, const ReadBufferFromFileBase::ProfileCallback & profile_callback); @@ -128,7 +127,6 @@ protected: const MergeTreeData::DataPartPtr & data_part, const AlterConversionsPtr & alter_conversions, const MergeTreeReadTaskColumns & task_columns, - const StorageMetadataPtr & metadata_snapshot, const MarkRanges & mark_ranges, const IMergeTreeReader::ValueSizeMap & value_size_map, const ReadBufferFromFileBase::ProfileCallback & profile_callback); @@ -207,7 +205,6 @@ private: const MergeTreeData::DataPartPtr & data_part, const AlterConversionsPtr & alter_conversions, const MergeTreeReadTaskColumns & task_columns, - const StorageMetadataPtr & metadata_snapshot, const MarkRanges & mark_ranges, const IMergeTreeReader::ValueSizeMap & value_size_map, const ReadBufferFromFileBase::ProfileCallback & profile_callback); diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 9c47608e364..fc8bfcc925a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -30,7 +30,7 @@ MergeTreeDataPartCompact::MergeTreeDataPartCompact( IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( const NamesAndTypesList & columns_to_read, - const StorageMetadataPtr & metadata_snapshot, + const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, @@ -43,7 +43,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( auto * load_marks_threadpool = reader_settings.read_settings.load_marks_asynchronously ? &read_info->getContext()->getLoadMarksThreadpool() : nullptr; return std::make_unique( - read_info, columns_to_read, metadata_snapshot, uncompressed_cache, + read_info, columns_to_read, storage_snapshot, uncompressed_cache, mark_cache, mark_ranges, reader_settings, load_marks_threadpool, avg_value_size_hints, profile_callback); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 08764eedb43..2bbac766c8e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -30,7 +30,7 @@ public: MergeTreeReaderPtr getReader( const NamesAndTypesList & columns, - const StorageMetadataPtr & metadata_snapshot, + const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 468747a6c36..ba300b110d7 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -32,7 +32,7 @@ MergeTreeDataPartInMemory::MergeTreeDataPartInMemory( IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader( const NamesAndTypesList & columns_to_read, - const StorageMetadataPtr & metadata_snapshot, + const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, UncompressedCache * /* uncompressed_cache */, MarkCache * /* mark_cache */, @@ -45,7 +45,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader( auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( - read_info, ptr, columns_to_read, metadata_snapshot, mark_ranges, reader_settings); + read_info, ptr, columns_to_read, storage_snapshot, mark_ranges, reader_settings); } IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartInMemory::getWriter( diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 2698b69b38e..81549eeed3e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -19,7 +19,7 @@ public: MergeTreeReaderPtr getReader( const NamesAndTypesList & columns, - const StorageMetadataPtr & metadata_snapshot, + const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 2d886e2058b..f62582d48cc 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -29,7 +29,7 @@ MergeTreeDataPartWide::MergeTreeDataPartWide( IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( const NamesAndTypesList & columns_to_read, - const StorageMetadataPtr & metadata_snapshot, + const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, @@ -41,7 +41,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( auto read_info = std::make_shared(shared_from_this(), alter_conversions); return std::make_unique( read_info, columns_to_read, - metadata_snapshot, uncompressed_cache, + storage_snapshot, uncompressed_cache, mark_cache, mark_ranges, reader_settings, avg_value_size_hints, profile_callback); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 0b2ffeb4b18..2076a1ec028 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -25,7 +25,7 @@ public: MergeTreeReaderPtr getReader( const NamesAndTypesList & columns, - const StorageMetadataPtr & metadata_snapshot, + const StorageSnapshotPtr & storage_snapshot, const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index e9e2138d995..901801d8187 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -97,7 +97,7 @@ std::future MergeTreePrefetchedReadPool::createPrefetchedRea Priority priority) const { auto reader = data_part.getReader( - columns, storage_snapshot->metadata, required_ranges, + columns, storage_snapshot, required_ranges, uncompressed_cache, mark_cache, alter_conversions, reader_settings, IMergeTreeReader::ValueSizeMap{}, profile_callback); diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 73b579eee5b..feefca68e7b 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -17,7 +17,7 @@ namespace ErrorCodes MergeTreeReaderCompact::MergeTreeReaderCompact( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, NamesAndTypesList columns_, - const StorageMetadataPtr & metadata_snapshot_, + const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, MarkRanges mark_ranges_, @@ -29,7 +29,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( : IMergeTreeReader( data_part_info_for_read_, columns_, - metadata_snapshot_, + storage_snapshot_, uncompressed_cache_, mark_cache_, mark_ranges_, @@ -166,8 +166,11 @@ void MergeTreeReaderCompact::fillColumnPositions() 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())); + { + auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects(); + auto storage_columns_list = Nested::collect(storage_snapshot->getColumns(options)); + storage_columns_with_collected_nested = ColumnsDescription(std::move(storage_columns_list)); + } column_to_read_with_subcolumns = storage_columns_with_collected_nested ->getColumnOrSubcolumn( diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index 796bc4684a1..cf706526363 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -21,7 +21,7 @@ public: MergeTreeReaderCompact( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, NamesAndTypesList columns_, - const StorageMetadataPtr & metadata_snapshot_, + const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, MarkRanges mark_ranges_, diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index ae1740525f7..bacd86511f5 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -19,13 +19,13 @@ MergeTreeReaderInMemory::MergeTreeReaderInMemory( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, DataPartInMemoryPtr data_part_, NamesAndTypesList columns_, - const StorageMetadataPtr & metadata_snapshot_, + const StorageSnapshotPtr & storage_snapshot_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_) : IMergeTreeReader( data_part_info_for_read_, columns_, - metadata_snapshot_, + storage_snapshot_, nullptr, nullptr, mark_ranges_, diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.h b/src/Storages/MergeTree/MergeTreeReaderInMemory.h index cb67bc46eae..e26a98f0916 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.h +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.h @@ -18,7 +18,7 @@ public: MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, DataPartInMemoryPtr data_part_, NamesAndTypesList columns_, - const StorageMetadataPtr & metadata_snapshot_, + const StorageSnapshotPtr & storage_snapshot_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_); diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 140fb6da5df..a0fe2dc63b4 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -24,7 +24,7 @@ namespace MergeTreeReaderWide::MergeTreeReaderWide( MergeTreeDataPartInfoForReaderPtr data_part_info_, NamesAndTypesList columns_, - const StorageMetadataPtr & metadata_snapshot_, + const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, MarkRanges mark_ranges_, @@ -35,7 +35,7 @@ MergeTreeReaderWide::MergeTreeReaderWide( : IMergeTreeReader( data_part_info_, columns_, - metadata_snapshot_, + storage_snapshot_, uncompressed_cache_, mark_cache_, mark_ranges_, diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index c31b1baf32e..2a850cc2814 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -17,7 +17,7 @@ public: MergeTreeReaderWide( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, NamesAndTypesList columns_, - const StorageMetadataPtr & metadata_snapshot_, + const StorageSnapshotPtr & storage_snapshot_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, MarkRanges mark_ranges_, diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index ce4ba69c08c..e5a45ad5554 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -65,7 +65,7 @@ void MergeTreeSelectAlgorithm::initializeReaders() initializeMergeTreeReadersForPart( data_part, alter_conversions, task_columns, - storage_snapshot->getMetadataForQuery(), all_mark_ranges, {}, {}); + all_mark_ranges, /*value_size_map=*/ {}, /*profile_callback=*/ {}); } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 5a6d59bf0be..88f6eaaf49f 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -150,7 +150,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( mark_ranges.emplace(MarkRanges{MarkRange(0, data_part->getMarksCount())}); reader = data_part->getReader( - columns_for_reader, storage_snapshot->metadata, + columns_for_reader, storage_snapshot, *mark_ranges, /* uncompressed_cache = */ nullptr, mark_cache.get(), alter_conversions, reader_settings, {}, {}); } diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp index 01094d65ac5..892ae9ead87 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp @@ -45,8 +45,6 @@ void MergeTreeThreadSelectAlgorithm::finalizeNewTask() /// Allows pool to reduce number of threads in case of too slow reads. auto profile_callback = [this](ReadBufferFromFileBase::ProfileInfo info_) { pool->profileFeedback(info_); }; - const auto & metadata_snapshot = storage_snapshot->metadata; - IMergeTreeReader::ValueSizeMap value_size_map; if (reader && part_name != last_read_part_name) @@ -57,7 +55,7 @@ void MergeTreeThreadSelectAlgorithm::finalizeNewTask() /// task->reader.valid() means there is a prefetched reader in this test, use it. const bool init_new_readers = !reader || task->reader.valid() || part_name != last_read_part_name; if (init_new_readers) - initializeMergeTreeReadersForCurrentTask(metadata_snapshot, value_size_map, profile_callback); + initializeMergeTreeReadersForCurrentTask(value_size_map, profile_callback); last_read_part_name = part_name; } From 30d7b7999ee0b42300ae0ef6069324044a6fb46d Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 7 Aug 2023 14:46:21 +0200 Subject: [PATCH 482/554] Updated to use azure_query in test & test_cluster --- .../test_storage_azure_blob_storage/test.py | 9 ++-- .../test_cluster.py | 43 ++++++------------- 2 files changed, 18 insertions(+), 34 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 65b7d95b2c0..823c90b05ee 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -35,10 +35,13 @@ def cluster(): cluster.shutdown() -def azure_query(node, query, try_num=10, settings={}): +def azure_query(node, query, expect_error='false', try_num=10, settings={}): for i in range(try_num): try: - return node.query(query, settings=settings) + if expect_error == 'true': + return node.query_and_get_error(query, settings=settings) + else: + 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", @@ -656,7 +659,7 @@ 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) + assert expected_err_msg in azure_query(node, query, expect_error='true') def test_function_signatures(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 48b76061472..2cf9072df5b 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_cluster.py +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -17,6 +17,7 @@ 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 +from test_storage_azure_blob_storage.test import azure_query @pytest.fixture(scope="module") @@ -48,26 +49,6 @@ def cluster(): 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;" @@ -89,7 +70,7 @@ def test_select_all(cluster): ) print(get_azure_file_content("test_cluster_select_all.csv")) - pure_azure = node.query( + pure_azure = azure_query( node, """ SELECT * from azureBlobStorage( 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_select_all.csv', 'devstoreaccount1', @@ -97,7 +78,7 @@ def test_select_all(cluster): 'auto')""" ) print(pure_azure) - distributed_azure = node.query( + distributed_azure = azure_query( node, """ SELECT * from azureBlobStorageCluster( 'simple_cluster', 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_select_all.csv', 'devstoreaccount1', @@ -119,7 +100,7 @@ def test_count(cluster): ) print(get_azure_file_content("test_cluster_count.csv")) - pure_azure = node.query( + pure_azure = azure_query( node, """ SELECT count(*) from azureBlobStorage( 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_count.csv', 'devstoreaccount1', @@ -127,7 +108,7 @@ def test_count(cluster): 'auto', 'key UInt64')""" ) print(pure_azure) - distributed_azure = node.query( + distributed_azure = azure_query( node, """ SELECT count(*) from azureBlobStorageCluster( 'simple_cluster', 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_count.csv', 'devstoreaccount1', @@ -148,7 +129,7 @@ def test_union_all(cluster): "'auto', 'a Int32, b String') VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", ) - pure_azure = node.query( + pure_azure = azure_query( node, """ SELECT * FROM ( @@ -165,7 +146,7 @@ def test_union_all(cluster): ORDER BY (a) """ ) - azure_distributed = node.query( + azure_distributed = azure_query( node, """ SELECT * FROM ( @@ -197,7 +178,7 @@ def test_skip_unavailable_shards(cluster): "'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " "'auto', 'a UInt64') VALUES (1), (2)", ) - result = node.query( + result = azure_query( node, """ SELECT count(*) from azureBlobStorageCluster( 'cluster_non_existent_port', @@ -220,7 +201,7 @@ def test_unset_skip_unavailable_shards(cluster): "'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " "'auto', 'a UInt64') VALUES (1), (2)", ) - result = node.query( + result = azure_query( node, """ SELECT count(*) from azureBlobStorageCluster( 'cluster_non_existent_port', @@ -243,7 +224,7 @@ def test_cluster_with_named_collection(cluster): "'auto', 'a UInt64') VALUES (1), (2)", ) - pure_azure = node.query( + pure_azure = azure_query( node, """ SELECT * from azureBlobStorage( 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_with_named_collection.csv', 'devstoreaccount1', @@ -251,7 +232,7 @@ def test_cluster_with_named_collection(cluster): """ ) - azure_cluster = node.query( + azure_cluster = azure_query( node, """ SELECT * from azureBlobStorageCluster( 'simple_cluster', azure_conf2, container='cont', blob_path='test_cluster_with_named_collection.csv') @@ -277,7 +258,7 @@ def test_partition_parallel_readig_withcluster(cluster): 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( + azure_cluster = azure_query( node, """ SELECT count(*) from azureBlobStorageCluster( 'simple_cluster', From 9cb949b60b16238e75ce8ab549a73c8ad524496b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Aug 2023 14:53:38 +0200 Subject: [PATCH 483/554] Fix style --- tests/ci/build_check.py | 41 ++++++++++++++++++++++++++++++++++++++--- 1 file changed, 38 insertions(+), 3 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 515c5a4943b..6c9cd7e193b 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -374,9 +374,44 @@ def main(): instance_type = get_instance_type() query = urllib.parse.quote( f""" - INSERT INTO build_time_trace (pull_request_number, commit_sha, check_start_time, check_name, instance_type, file, library, time, pid, tid, ph, ts, dur, cat, name, detail, count, avgMs, args_name) + INSERT INTO build_time_trace + ( + pull_request_number, + commit_sha, + check_start_time, + check_name, + instance_type, + file, + library, + time, + pid, + tid, + ph, + ts, + dur, + cat, + name, + detail, + count, + avgMs, + args_name + ) SELECT {pr_info.number}, '{pr_info.sha}', '{stopwatch.start_time_str}', '{build_name}', '{instance_type}', * - FROM input('file String, library String, time DateTime64(6), pid UInt32, tid UInt32, ph String, ts UInt64, dur UInt64, cat String, name String, detail String, count UInt64, avgMs UInt64, args_name String') + FROM input(' + file String, + library String, + time DateTime64(6), + pid UInt32, + tid UInt32, + ph String, + ts UInt64, + dur UInt64, + cat String, + name String, + detail String, + count UInt64, + avgMs UInt64, + args_name String') FORMAT JSONEachRow """ ) @@ -388,7 +423,7 @@ def main(): print(f"::notice ::Log Uploading profile data, path: {file_path}, query: {query}") with open(file_path, "rb") as file: - response = requests.post(url, data=file) + requests.post(url, data=file) # Upload statistics to CI database From 598a48fe22cdb6901d76eb498acc4799920fa7a7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Aug 2023 14:56:50 +0200 Subject: [PATCH 484/554] Fix mypy --- 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..ca27853de2d 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -19,7 +19,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ shellcheck \ yamllint \ locales \ - && pip3 install black==23.1.0 boto3 codespell==2.2.1 mypy==1.3.0 PyGithub unidiff pylint==2.6.2 \ + && pip3 install black==23.1.0 boto3 codespell==2.2.1 mypy==1.3.0 PyGithub unidiff pylint==2.6.2 types-requests=2.31.0.2 types-urllib3=1.26.25.14 \ && apt-get clean \ && rm -rf /root/.cache/pip From 9587589a49ba82e0b1877a262445a9251d913608 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 7 Aug 2023 13:04:51 +0000 Subject: [PATCH 485/554] Automatic style fix --- .../test_storage_azure_blob_storage/test.py | 6 +- .../test_cluster.py | 55 +++++++++++-------- 2 files changed, 36 insertions(+), 25 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 823c90b05ee..37cdc9b609b 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -35,10 +35,10 @@ def cluster(): cluster.shutdown() -def azure_query(node, query, expect_error='false', try_num=10, settings={}): +def azure_query(node, query, expect_error="false", try_num=10, settings={}): for i in range(try_num): try: - if expect_error == 'true': + if expect_error == "true": return node.query_and_get_error(query, settings=settings) else: return node.query(query, settings=settings) @@ -659,7 +659,7 @@ 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 azure_query(node, query, expect_error='true') + assert expected_err_msg in azure_query(node, query, expect_error="true") def test_function_signatures(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 2cf9072df5b..454f1cdf294 100644 --- a/tests/integration/test_storage_azure_blob_storage/test_cluster.py +++ b/tests/integration/test_storage_azure_blob_storage/test_cluster.py @@ -70,20 +70,22 @@ def test_select_all(cluster): ) print(get_azure_file_content("test_cluster_select_all.csv")) - pure_azure = azure_query( node, + pure_azure = azure_query( + node, """ SELECT * from azureBlobStorage( 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_select_all.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', - 'auto')""" + 'auto')""", ) print(pure_azure) - distributed_azure = azure_query( node, + distributed_azure = azure_query( + node, """ SELECT * from azureBlobStorageCluster( 'simple_cluster', 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_select_all.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', - 'auto')""" + 'auto')""", ) print(distributed_azure) assert TSV(pure_azure) == TSV(distributed_azure) @@ -100,20 +102,22 @@ def test_count(cluster): ) print(get_azure_file_content("test_cluster_count.csv")) - pure_azure = azure_query( node, + pure_azure = azure_query( + node, """ SELECT count(*) from azureBlobStorage( 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_count.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', - 'auto', 'key UInt64')""" + 'auto', 'key UInt64')""", ) print(pure_azure) - distributed_azure = azure_query( node, + distributed_azure = azure_query( + node, """ SELECT count(*) from azureBlobStorageCluster( 'simple_cluster', 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_count.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', - 'auto', 'key UInt64')""" + 'auto', 'key UInt64')""", ) print(distributed_azure) assert TSV(pure_azure) == TSV(distributed_azure) @@ -129,7 +133,8 @@ def test_union_all(cluster): "'auto', 'a Int32, b String') VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", ) - pure_azure = azure_query( node, + pure_azure = azure_query( + node, """ SELECT * FROM ( @@ -144,9 +149,10 @@ def test_union_all(cluster): 'auto', 'a Int32, b String') ) ORDER BY (a) - """ + """, ) - azure_distributed = azure_query( node, + azure_distributed = azure_query( + node, """ SELECT * FROM ( @@ -163,7 +169,7 @@ def test_union_all(cluster): 'auto', 'a Int32, b String') ) ORDER BY (a) - """ + """, ) assert TSV(pure_azure) == TSV(azure_distributed) @@ -178,14 +184,15 @@ def test_skip_unavailable_shards(cluster): "'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " "'auto', 'a UInt64') VALUES (1), (2)", ) - result = azure_query( node, + result = azure_query( + node, """ 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" @@ -201,13 +208,14 @@ def test_unset_skip_unavailable_shards(cluster): "'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'auto', " "'auto', 'a UInt64') VALUES (1), (2)", ) - result = azure_query( node, + result = azure_query( + node, """ 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" @@ -224,19 +232,21 @@ def test_cluster_with_named_collection(cluster): "'auto', 'a UInt64') VALUES (1), (2)", ) - pure_azure = azure_query( node, + pure_azure = azure_query( + node, """ SELECT * from azureBlobStorage( 'http://azurite1:10000/devstoreaccount1', 'cont', 'test_cluster_with_named_collection.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==') - """ + """, ) - azure_cluster = azure_query( node, + azure_cluster = azure_query( + node, """ SELECT * from azureBlobStorageCluster( 'simple_cluster', azure_conf2, container='cont', blob_path='test_cluster_with_named_collection.csv') - """ + """, ) assert TSV(pure_azure) == TSV(azure_cluster) @@ -258,12 +268,13 @@ def test_partition_parallel_readig_withcluster(cluster): 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 = azure_query( node, + azure_cluster = azure_query( + node, """ 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" From 97a2988919cd2d64748f114750b1bf48d21e9363 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 7 Aug 2023 16:58:59 +0300 Subject: [PATCH 486/554] Update test.py --- .../integration/test_restore_replica/test.py | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_restore_replica/test.py b/tests/integration/test_restore_replica/test.py index 31c503f6184..3d0baa87c21 100644 --- a/tests/integration/test_restore_replica/test.py +++ b/tests/integration/test_restore_replica/test.py @@ -44,6 +44,18 @@ def fill_table(): check_data(499500, 1000) +# kazoo.delete may throw NotEmptyError on concurrent modifications of the path +def zk_rmr_with_retries(zk, path): + for i in range(1, 10): + try: + zk.delete(path, recursive=True) + return + except Exception as ex: + print(ex) + time.sleep(0.5) + assert False + + @pytest.fixture(scope="module") def start_cluster(): try: @@ -84,7 +96,7 @@ def test_restore_replica_sequential(start_cluster): fill_table() print("Deleting root ZK path metadata") - zk.delete("/clickhouse/tables/test", recursive=True) + zk_rmr_with_retries(zk, "/clickhouse/tables/test") assert zk.exists("/clickhouse/tables/test") is None node_1.query("SYSTEM RESTART REPLICA test") @@ -119,7 +131,7 @@ def test_restore_replica_parallel(start_cluster): fill_table() print("Deleting root ZK path metadata") - zk.delete("/clickhouse/tables/test", recursive=True) + zk_rmr_with_retries(zk, "/clickhouse/tables/test") assert zk.exists("/clickhouse/tables/test") is None node_1.query("SYSTEM RESTART REPLICA test") @@ -147,12 +159,12 @@ def test_restore_replica_alive_replicas(start_cluster): fill_table() print("Deleting replica2 path, trying to restore replica1") - zk.delete("/clickhouse/tables/test/replicas/replica2", recursive=True) + zk_rmr_with_retries(zk, "/clickhouse/tables/test/replicas/replica2") assert zk.exists("/clickhouse/tables/test/replicas/replica2") is None node_1.query_and_get_error("SYSTEM RESTORE REPLICA test") print("Deleting replica1 path, trying to restore replica1") - zk.delete("/clickhouse/tables/test/replicas/replica1", recursive=True) + zk_rmr_with_retries(zk, "/clickhouse/tables/test/replicas/replica1") assert zk.exists("/clickhouse/tables/test/replicas/replica1") is None node_1.query("SYSTEM RESTART REPLICA test") From f9264b5f903e76f74d296a4d55d81dd83c4f5b00 Mon Sep 17 00:00:00 2001 From: ekrasikov Date: Mon, 7 Aug 2023 17:03:48 +0200 Subject: [PATCH 487/554] do not fail if prctl is not allowed --- src/Common/setThreadName.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/setThreadName.cpp b/src/Common/setThreadName.cpp index 65c4b5c6523..f90398825af 100644 --- a/src/Common/setThreadName.cpp +++ b/src/Common/setThreadName.cpp @@ -43,7 +43,7 @@ void setThreadName(const char * name) #else if (0 != prctl(PR_SET_NAME, name, 0, 0, 0)) #endif - if (errno != ENOSYS) /// It's ok if the syscall is unsupported in some environments. + if (errno != ENOSYS && errno != EPERM) /// It's ok if the syscall is unsupported or not allowed in some environments. DB::throwFromErrno("Cannot set thread name with prctl(PR_SET_NAME, ...)", DB::ErrorCodes::PTHREAD_ERROR); memcpy(thread_name, name, std::min(1 + strlen(name), THREAD_NAME_SIZE - 1)); @@ -63,7 +63,7 @@ const char * getThreadName() // throw DB::Exception(DB::ErrorCodes::PTHREAD_ERROR, "Cannot get thread name with pthread_get_name_np()"); #else if (0 != prctl(PR_GET_NAME, thread_name, 0, 0, 0)) - if (errno != ENOSYS) /// It's ok if the syscall is unsupported in some environments. + if (errno != ENOSYS && errno != EPERM) /// It's ok if the syscall is unsupported or not allowed in some environments. DB::throwFromErrno("Cannot get thread name with prctl(PR_GET_NAME)", DB::ErrorCodes::PTHREAD_ERROR); #endif From 53f9bf2ea39ff4fb501be4e02700a1e5ae5cd94b Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 7 Aug 2023 15:22:11 +0000 Subject: [PATCH 488/554] Update list --- tests/analyzer_integration_broken_tests.txt | 44 ------------------- .../test.py | 6 +-- ...roupBitmapAndState_on_distributed_table.py | 6 +-- 3 files changed, 6 insertions(+), 50 deletions(-) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 8075bb6d230..dde6c9f7f1e 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -1,5 +1,4 @@ test_access_for_functions/test.py::test_access_rights_for_function -test_backward_compatibility/test_normalized_count_comparison.py::test_select_aggregate_alias_column test_concurrent_backups_s3/test.py::test_concurrent_backups test_distributed_ddl/test.py::test_default_database[configs] test_distributed_ddl/test.py::test_default_database[configs_secure] @@ -29,16 +28,6 @@ test_distributed_load_balancing/test.py::test_load_balancing_default test_distributed_load_balancing/test.py::test_load_balancing_priority_round_robin[dist_priority] test_distributed_load_balancing/test.py::test_load_balancing_priority_round_robin[dist_priority_negative] test_distributed_load_balancing/test.py::test_load_balancing_round_robin -test_backward_compatibility/test.py::test_backward_compatability1 -test_backward_compatibility/test_aggregate_fixed_key.py::test_two_level_merge -test_backward_compatibility/test_aggregate_function_state.py::test_backward_compatability_for_avg -test_backward_compatibility/test_aggregate_function_state.py::test_backward_compatability_for_uniq_exact[1000] -test_backward_compatibility/test_aggregate_function_state.py::test_backward_compatability_for_uniq_exact[500000] -test_backward_compatibility/test_aggregate_function_state.py::test_backward_compatability_for_uniq_exact_variadic[1000] -test_backward_compatibility/test_aggregate_function_state.py::test_backward_compatability_for_uniq_exact_variadic[500000] -test_backward_compatibility/test_ip_types_binary_compatibility.py::test_ip_types_binary_compatibility -test_backward_compatibility/test_select_aggregate_alias_column.py::test_select_aggregate_alias_column -test_backward_compatibility/test_short_strings_aggregation.py::test_backward_compatability test_mask_sensitive_info/test.py::test_encryption_functions test_merge_table_over_distributed/test.py::test_global_in test_merge_table_over_distributed/test.py::test_select_table_name_from_merge_over_distributed @@ -87,7 +76,6 @@ test_row_policy/test.py::test_users_xml_is_readonly test_row_policy/test.py::test_with_prewhere test_row_policy/test.py::test_with_prewhere test_settings_constraints_distributed/test.py::test_select_clamps_settings -test_backward_compatibility/test_cte_distributed.py::test_cte_distributed test_compression_codec_read/test.py::test_default_codec_read test_dictionaries_update_and_reload/test.py::test_reload_after_fail_in_cache_dictionary test_distributed_type_object/test.py::test_distributed_type_object @@ -98,9 +86,6 @@ test_storage_postgresql/test.py::test_postgres_select_insert test_storage_rabbitmq/test.py::test_rabbitmq_materialized_view test_system_merges/test.py::test_mutation_simple[] test_system_merges/test.py::test_mutation_simple[replicated] -test_backward_compatibility/test_insert_profile_events.py::test_new_client_compatible -test_backward_compatibility/test_insert_profile_events.py::test_old_client_compatible -test_backward_compatibility/test_vertical_merges_from_compact_parts.py::test_vertical_merges_from_compact_parts test_disk_over_web_server/test.py::test_cache[node2] test_disk_over_web_server/test.py::test_incorrect_usage test_disk_over_web_server/test.py::test_replicated_database @@ -108,16 +93,8 @@ test_disk_over_web_server/test.py::test_unavailable_server test_disk_over_web_server/test.py::test_usage[node2] test_distributed_backward_compatability/test.py::test_distributed_in_tuple test_executable_table_function/test.py::test_executable_function_input_python -test_groupBitmapAnd_on_distributed/test_groupBitmapAndState_on_distributed_table.py::test_groupBitmapAndState_on_different_version_nodes -test_groupBitmapAnd_on_distributed/test_groupBitmapAndState_on_distributed_table.py::test_groupBitmapAndState_on_distributed_table 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_backward_compatibility/test_functions.py::test_aggregate_states -test_backward_compatibility/test_functions.py::test_string_functions -test_default_compression_codec/test.py::test_default_codec_for_compact_parts -test_default_compression_codec/test.py::test_default_codec_multiple -test_default_compression_codec/test.py::test_default_codec_single -test_default_compression_codec/test.py::test_default_codec_version_update test_postgresql_protocol/test.py::test_python_client test_quota/test.py::test_add_remove_interval test_quota/test.py::test_add_remove_quota @@ -135,25 +112,8 @@ 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_compatibility -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_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_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_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 @@ -161,7 +121,6 @@ test_sql_user_defined_functions_on_cluster/test.py::test_sql_user_defined_functi 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 @@ -182,7 +141,4 @@ 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 diff --git a/tests/integration/test_groupBitmapAnd_on_distributed/test.py b/tests/integration/test_groupBitmapAnd_on_distributed/test.py index f6e54a110eb..a7d3eab0d4b 100644 --- a/tests/integration/test_groupBitmapAnd_on_distributed/test.py +++ b/tests/integration/test_groupBitmapAnd_on_distributed/test.py @@ -5,13 +5,13 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/clusters.xml"], with_zookeeper=True + "node1", main_configs=["configs/clusters.xml"], with_zookeeper=True, allow_analyzer=False ) node2 = cluster.add_instance( - "node2", main_configs=["configs/clusters.xml"], with_zookeeper=True + "node2", main_configs=["configs/clusters.xml"], with_zookeeper=True, allow_analyzer=False ) node3 = cluster.add_instance( - "node3", main_configs=["configs/clusters.xml"], with_zookeeper=True + "node3", main_configs=["configs/clusters.xml"], with_zookeeper=True, allow_analyzer=False ) node4 = cluster.add_instance( "node4", 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 3cbdf7e473a..920f9cede94 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 @@ -5,13 +5,13 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/clusters.xml"], with_zookeeper=True + "node1", main_configs=["configs/clusters.xml"], with_zookeeper=True, allow_analyzer=False ) node2 = cluster.add_instance( - "node2", main_configs=["configs/clusters.xml"], with_zookeeper=True + "node2", main_configs=["configs/clusters.xml"], with_zookeeper=True, allow_analyzer=False ) node3 = cluster.add_instance( - "node3", main_configs=["configs/clusters.xml"], with_zookeeper=True + "node3", main_configs=["configs/clusters.xml"], with_zookeeper=True, allow_analyzer=False ) node4 = cluster.add_instance( "node4", From a69bbc8a5a432a86b9243e03be7ba77546af3f9d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 7 Aug 2023 15:49:34 +0000 Subject: [PATCH 489/554] Automatic style fix --- .../test_groupBitmapAnd_on_distributed/test.py | 15 ++++++++++++--- ...st_groupBitmapAndState_on_distributed_table.py | 15 ++++++++++++--- 2 files changed, 24 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_groupBitmapAnd_on_distributed/test.py b/tests/integration/test_groupBitmapAnd_on_distributed/test.py index a7d3eab0d4b..8cf7e0fb2c1 100644 --- a/tests/integration/test_groupBitmapAnd_on_distributed/test.py +++ b/tests/integration/test_groupBitmapAnd_on_distributed/test.py @@ -5,13 +5,22 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/clusters.xml"], with_zookeeper=True, allow_analyzer=False + "node1", + main_configs=["configs/clusters.xml"], + with_zookeeper=True, + allow_analyzer=False, ) node2 = cluster.add_instance( - "node2", main_configs=["configs/clusters.xml"], with_zookeeper=True, allow_analyzer=False + "node2", + main_configs=["configs/clusters.xml"], + with_zookeeper=True, + allow_analyzer=False, ) node3 = cluster.add_instance( - "node3", main_configs=["configs/clusters.xml"], with_zookeeper=True, allow_analyzer=False + "node3", + main_configs=["configs/clusters.xml"], + with_zookeeper=True, + allow_analyzer=False, ) node4 = cluster.add_instance( "node4", 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 920f9cede94..115e6009801 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 @@ -5,13 +5,22 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", main_configs=["configs/clusters.xml"], with_zookeeper=True, allow_analyzer=False + "node1", + main_configs=["configs/clusters.xml"], + with_zookeeper=True, + allow_analyzer=False, ) node2 = cluster.add_instance( - "node2", main_configs=["configs/clusters.xml"], with_zookeeper=True, allow_analyzer=False + "node2", + main_configs=["configs/clusters.xml"], + with_zookeeper=True, + allow_analyzer=False, ) node3 = cluster.add_instance( - "node3", main_configs=["configs/clusters.xml"], with_zookeeper=True, allow_analyzer=False + "node3", + main_configs=["configs/clusters.xml"], + with_zookeeper=True, + allow_analyzer=False, ) node4 = cluster.add_instance( "node4", From 7082cbf5c6b0c01e0d5b32f1a433310c9819b890 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 7 Aug 2023 17:37:13 +0200 Subject: [PATCH 490/554] Use more unique name for TemporaryFileOnDisk. --- src/Disks/TemporaryFileOnDisk.cpp | 15 +++------------ 1 file changed, 3 insertions(+), 12 deletions(-) diff --git a/src/Disks/TemporaryFileOnDisk.cpp b/src/Disks/TemporaryFileOnDisk.cpp index b9c9e238468..06d7da4af58 100644 --- a/src/Disks/TemporaryFileOnDisk.cpp +++ b/src/Disks/TemporaryFileOnDisk.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -41,17 +40,9 @@ TemporaryFileOnDisk::TemporaryFileOnDisk(const DiskPtr & disk_, const String & p ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal); - /// Do not use default temporaty root path `/tmp/tmpXXXXXX`. - /// The `dummy_prefix` is used to know what to replace with the real prefix. - String dummy_prefix = "a/"; - relative_path = Poco::TemporaryFile::tempName(dummy_prefix); - dummy_prefix += "tmp"; - /// a/tmpXXXXX -> XXXXX - assert(relative_path.starts_with(dummy_prefix)); - relative_path.replace(0, dummy_prefix.length(), prefix); - - if (relative_path.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file name is empty"); + /// A disk can be remote and shared between multiple replicas. + /// That's why we must not use Poco::TemporaryFile::tempName() here (Poco::TemporaryFile::tempName() can return the same names for different processes on different nodes). + relative_path = prefix + toString(UUIDHelpers::generateV4()); } String TemporaryFileOnDisk::getAbsolutePath() const From 2347f5886b194ce77374309a8400db73449512ec Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 7 Aug 2023 18:06:50 +0200 Subject: [PATCH 491/554] Update `Mergeable Check` at the finishing CI --- tests/ci/finish_check.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index aa8a0cf9553..74392947b82 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -8,6 +8,7 @@ from commit_status_helper import ( get_commit, get_commit_filtered_statuses, post_commit_status, + update_mergeable_check, ) from get_robot_token import get_best_robot_token from pr_info import PRInfo @@ -18,6 +19,8 @@ def main(): pr_info = PRInfo(need_orgs=True) gh = Github(get_best_robot_token(), per_page=100) + # Update the Mergeable Check at the final step + update_mergeable_check(gh, pr_info, CI_STATUS_NAME) commit = get_commit(gh, pr_info.sha) statuses = [ @@ -27,7 +30,8 @@ def main(): ] if not statuses: return - status = statuses[0] + # Take the latest status + status = statuses[-1] if status.state == "pending": post_commit_status( commit, From 5275062eaf3b4ac5a01cffa9dadc610677874ba0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Aug 2023 18:44:25 +0200 Subject: [PATCH 492/554] Fix trailing whitespaces --- tests/ci/build_check.py | 60 ++++++++++++++++++++--------------------- 1 file changed, 30 insertions(+), 30 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 6c9cd7e193b..04610466130 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -376,41 +376,41 @@ def main(): f""" INSERT INTO build_time_trace ( - pull_request_number, - commit_sha, - check_start_time, - check_name, - instance_type, - file, - library, - time, - pid, - tid, - ph, - ts, - dur, - cat, - name, - detail, - count, - avgMs, + pull_request_number, + commit_sha, + check_start_time, + check_name, + instance_type, + file, + library, + time, + pid, + tid, + ph, + ts, + dur, + cat, + name, + detail, + count, + avgMs, args_name ) SELECT {pr_info.number}, '{pr_info.sha}', '{stopwatch.start_time_str}', '{build_name}', '{instance_type}', * FROM input(' - file String, - library String, - time DateTime64(6), - pid UInt32, - tid UInt32, - ph String, + file String, + library String, + time DateTime64(6), + pid UInt32, + tid UInt32, + ph String, ts UInt64, - dur UInt64, - cat String, - name String, - detail String, - count UInt64, - avgMs UInt64, + dur UInt64, + cat String, + name String, + detail String, + count UInt64, + avgMs UInt64, args_name String') FORMAT JSONEachRow """ From ab64845740e080f6273c1874ad91d9cf7d099d32 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Aug 2023 18:45:32 +0200 Subject: [PATCH 493/554] Fix mypy --- tests/ci/build_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 04610466130..2712dba4845 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -1,5 +1,6 @@ #!/usr/bin/env python3 +from typing import List, Tuple import subprocess import logging import json @@ -8,7 +9,6 @@ import sys import time import urllib.parse import requests -from typing import List, Tuple from ci_config import CI_CONFIG, BuildConfig from docker_pull_helper import get_image_with_version From 5fa0edc3b2b81411c1adba39b5124a2b92882c98 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Aug 2023 18:47:16 +0200 Subject: [PATCH 494/554] Make mypy happy --- docker/test/style/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index ca27853de2d..148633d5c13 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -20,6 +20,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ yamllint \ locales \ && pip3 install black==23.1.0 boto3 codespell==2.2.1 mypy==1.3.0 PyGithub unidiff pylint==2.6.2 types-requests=2.31.0.2 types-urllib3=1.26.25.14 \ + && mypy --install-types \ && apt-get clean \ && rm -rf /root/.cache/pip From 473fb60e6fb4e27aa712435482bf49109ae3d64f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 7 Aug 2023 18:56:18 +0200 Subject: [PATCH 495/554] Added retry for TransportException for azure read & download --- .../IO/ReadBufferFromAzureBlobStorage.cpp | 48 +++++++++++++------ 1 file changed, 34 insertions(+), 14 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 129bb97be09..15b632a54d4 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -102,6 +102,19 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() size_t bytes_read = 0; size_t sleep_time_with_backoff_milliseconds = 100; + + auto handle_exception = [&, this](const auto & e, size_t i) + { + LOG_INFO(log, "Exception caught during Azure Read for file {} at attempt {}: {}", path, i, e.Message); + if (i + 1 == max_single_read_retries) + throw; + + sleepForMilliseconds(sleep_time_with_backoff_milliseconds); + sleep_time_with_backoff_milliseconds *= 2; + initialized = false; + initialize(); + }; + for (size_t i = 0; i < max_single_read_retries; ++i) { try @@ -111,16 +124,13 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() read_settings.remote_throttler->add(bytes_read, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); break; } + catch (const Azure::Core::Http::TransportException & e) + { + handle_exception(e, i); + } catch (const Azure::Storage::StorageException & e) { - LOG_INFO(log, "Exception caught during Azure Read for file {} at attempt {}: {}", path, i, e.Message); - if (i + 1 == max_single_read_retries) - throw; - - sleepForMilliseconds(sleep_time_with_backoff_milliseconds); - sleep_time_with_backoff_milliseconds *= 2; - initialized = false; - initialize(); + handle_exception(e, i); } } @@ -211,6 +221,17 @@ void ReadBufferFromAzureBlobStorage::initialize() blob_client = std::make_unique(blob_container_client->GetBlobClient(path)); size_t sleep_time_with_backoff_milliseconds = 100; + + auto handle_exception = [&, this](const auto & e, size_t i) + { + LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {} : {}", path, offset, i + 1, e.Message); + if (i + 1 == max_single_download_retries) + throw; + + sleepForMilliseconds(sleep_time_with_backoff_milliseconds); + sleep_time_with_backoff_milliseconds *= 2; + }; + for (size_t i = 0; i < max_single_download_retries; ++i) { try @@ -219,14 +240,13 @@ void ReadBufferFromAzureBlobStorage::initialize() data_stream = std::move(download_response.Value.BodyStream); break; } + catch (const Azure::Core::Http::TransportException & e) + { + handle_exception(e, i); + } catch (const Azure::Core::RequestFailedException & e) { - LOG_INFO(log, "Exception caught during Azure Download for file {} at offset {} at attempt {} : {}", path, offset, i + 1, e.Message); - if (i + 1 == max_single_download_retries) - throw; - - sleepForMilliseconds(sleep_time_with_backoff_milliseconds); - sleep_time_with_backoff_milliseconds *= 2; + handle_exception(e,i); } } From 2f414950b7b17e85bca9f8db0084b4fb5263a174 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 7 Aug 2023 20:57:42 +0200 Subject: [PATCH 496/554] Fix logging for asynchronous non-batched distributed sends (#52583) Before you may see the following: 2023.07.25 09:21:39.705559 [ 692 ] {6b5e1299-1b64-4dbb-b25d-45e10027db22} test_hkt5nnqj.dist_opentelemetry.DirectoryMonitor.default: Finished processing `` (took 37 ms) Because file_path and current_file are the references to the same variable in DistributedAsyncInsertDirectoryQueue::processFile(). Signed-off-by: Azat Khuzhin --- .../Distributed/DistributedAsyncInsertDirectoryQueue.cpp | 6 +++--- .../Distributed/DistributedAsyncInsertDirectoryQueue.h | 2 +- ...2417_opentelemetry_insert_on_distributed_table.reference | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp index d8d9a0c9d1e..51839ad973a 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.cpp @@ -419,7 +419,7 @@ catch (...) throw; } -void DistributedAsyncInsertDirectoryQueue::processFile(const std::string & file_path) +void DistributedAsyncInsertDirectoryQueue::processFile(std::string & file_path) { OpenTelemetry::TracingContextHolderPtr thread_trace_context; @@ -459,7 +459,7 @@ void DistributedAsyncInsertDirectoryQueue::processFile(const std::string & file_ if (isDistributedSendBroken(e.code(), e.isRemoteException())) { markAsBroken(file_path); - current_file.clear(); + file_path.clear(); } throw; } @@ -473,8 +473,8 @@ void DistributedAsyncInsertDirectoryQueue::processFile(const std::string & file_ auto dir_sync_guard = getDirectorySyncGuard(relative_path); markAsSend(file_path); - current_file.clear(); LOG_TRACE(log, "Finished processing `{}` (took {} ms)", file_path, watch.elapsedMilliseconds()); + file_path.clear(); } struct DistributedAsyncInsertDirectoryQueue::BatchHeader diff --git a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h index 9a8a235e265..45c355bb64e 100644 --- a/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h +++ b/src/Storages/Distributed/DistributedAsyncInsertDirectoryQueue.h @@ -100,7 +100,7 @@ private: void addFile(const std::string & file_path); void initializeFilesFromDisk(); void processFiles(); - void processFile(const std::string & file_path); + void processFile(std::string & file_path); void processFilesWithBatching(); void markAsBroken(const std::string & file_path); diff --git a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference index 98827438920..a0689a0a090 100644 --- a/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference +++ b/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference @@ -3,8 +3,8 @@ {"operation_name":"void DB::DistributedSink::writeToLocal(const Cluster::ShardInfo &, const Block &, size_t)","cluster":"test_cluster_two_shards_localhost","shard":"2","rows":"1","bytes":"8"} 1 ===2=== -{"operation_name":"void DB::DistributedAsyncInsertDirectoryQueue::processFile(const std::string &)","cluster":"test_cluster_two_shards_localhost","shard":"1","rows":"1","bytes":"8"} -{"operation_name":"void DB::DistributedAsyncInsertDirectoryQueue::processFile(const std::string &)","cluster":"test_cluster_two_shards_localhost","shard":"2","rows":"1","bytes":"8"} +{"operation_name":"void DB::DistributedAsyncInsertDirectoryQueue::processFile(std::string &)","cluster":"test_cluster_two_shards_localhost","shard":"1","rows":"1","bytes":"8"} +{"operation_name":"void DB::DistributedAsyncInsertDirectoryQueue::processFile(std::string &)","cluster":"test_cluster_two_shards_localhost","shard":"2","rows":"1","bytes":"8"} 3 2 ===3=== From 77e1693f3c9470d13d4e812cd3f58675ef47c321 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Aug 2023 21:18:56 +0200 Subject: [PATCH 497/554] Fix mypy --- tests/ci/build_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 2712dba4845..c654574385a 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -8,7 +8,7 @@ import os import sys import time import urllib.parse -import requests +import requests # type: ignore from ci_config import CI_CONFIG, BuildConfig from docker_pull_helper import get_image_with_version From 440424fd72d30a245622c0fa42328ed3809e8707 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Aug 2023 21:23:39 +0200 Subject: [PATCH 498/554] @felixoid said we do not need these --- docker/test/style/Dockerfile | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index 148633d5c13..bd1c26855b7 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -19,8 +19,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ shellcheck \ yamllint \ locales \ - && pip3 install black==23.1.0 boto3 codespell==2.2.1 mypy==1.3.0 PyGithub unidiff pylint==2.6.2 types-requests=2.31.0.2 types-urllib3=1.26.25.14 \ - && mypy --install-types \ + && 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 From 4bdf18acfb1df28b7ebbf644f68a4bf752e2dfb5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Aug 2023 21:25:16 +0200 Subject: [PATCH 499/554] Disable new parquet encoder --- 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 2842f5a0faa..9fe66d97456 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -961,7 +961,7 @@ 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_use_custom_encoder, false, "Use a 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) \ From 796f87411651e0bffc5b2bf4ee464b07bf23eac1 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Tue, 8 Aug 2023 01:25:16 +0400 Subject: [PATCH 500/554] metrics_perf_events_enabled turn off in perf tests (#52072) * metrics_perf_events_enabled turn off in perf tests * turn on metrics_perf_events_enabled for heating and profile requests * close_perf_descriptors fix the condition * do not read opend but disabled perf event * Revert "do not read opend but disabled perf event" This reverts commit eeab1c9c632350d91fffae8bbf936684a169e09c. --- .../config/users.d/perf-comparison-tweaks-users.xml | 2 +- docker/test/performance-comparison/perf.py | 2 ++ src/Interpreters/ThreadStatusExt.cpp | 10 ++++------ 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml b/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml index 093834943a3..dab41adca51 100644 --- a/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml +++ b/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml @@ -3,7 +3,7 @@ 1 1 - 1 + 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 ce30b93b34348700351e0d462edb5baa6ddb5685 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 9 Aug 2023 23:46:45 +0200 Subject: [PATCH 553/554] 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 c071f6755d70a5ce9f1924fc44a905cc3122da5d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:10:03 +0300 Subject: [PATCH 554/554] 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'"