From 2081408c15e330ea737bc99a0afe7d96346c13d3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 16 Jan 2023 15:10:31 +0100 Subject: [PATCH 001/184] 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 002/184] 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 003/184] 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 004/184] 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 564691e25b006069dbebc78ec653360d49a87d36 Mon Sep 17 00:00:00 2001 From: Nikita Keba Date: Thu, 25 May 2023 00:00:32 +0000 Subject: [PATCH 005/184] 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 f07999699ff28a0708ca05bf9a1dd7e6d3ddad5a Mon Sep 17 00:00:00 2001 From: Nikita Keba Date: Mon, 29 May 2023 17:00:38 +0000 Subject: [PATCH 006/184] 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 007/184] 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 008/184] 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 009/184] 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 010/184] 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 011/184] 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 012/184] 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 013/184] 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 014/184] 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 015/184] 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 016/184] 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 017/184] 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 018/184] 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 019/184] 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 020/184] 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 021/184] 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 022/184] 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 023/184] 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 11220777974125135c6054655a8231cd17877170 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 19 Jun 2023 13:36:01 +0000 Subject: [PATCH 024/184] 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 025/184] 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 81b5ad3b26822c1b22a71ee38202ccf07be88f8f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 27 Jun 2023 16:51:17 +0200 Subject: [PATCH 026/184] 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 027/184] 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 4a18ec62ea819b6049457ac8c8fd0391b57ef61f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 17 Jul 2023 14:52:11 +0200 Subject: [PATCH 028/184] 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 029/184] 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 671f8b34d06364e2c6e26a60ef6489f8c23e4ef7 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 17 Jul 2023 17:18:14 +0200 Subject: [PATCH 030/184] 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 031/184] 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 6cc8a9af5c60a18a2d5ab636205993701e7fbb9a Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 21 Jul 2023 11:19:12 +0200 Subject: [PATCH 032/184] 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 06b782d37a8a48f66db88a32f456e6cd9649b49b Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 26 Jul 2023 14:40:50 +0300 Subject: [PATCH 033/184] 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 034/184] 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 035/184] 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 036/184] 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 037/184] 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 bae407e9581aab765aa24f3cdcc18f823c665af5 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 27 Jul 2023 12:48:36 +0000 Subject: [PATCH 038/184] 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 039/184] 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 04ad661968f3264bd423e8c707e41612e3d4e405 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 27 Jul 2023 18:17:47 +0300 Subject: [PATCH 040/184] 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 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 041/184] 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 6687f37329318c6995d795f8069ca5123e7bcf61 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 27 Jul 2023 18:11:52 +0000 Subject: [PATCH 042/184] 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 043/184] 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 17fa8a87a87791a52a4873a329b522cdc84dbb12 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 28 Jul 2023 08:54:25 +0000 Subject: [PATCH 044/184] 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 e83e0ec2cd78a8bc09f34a7cde849531a59eafdb Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 28 Jul 2023 11:55:23 +0000 Subject: [PATCH 045/184] 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 046/184] 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 e0e6472195f87607a87da7207e20ea8609a1482f Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 28 Jul 2023 15:50:22 +0000 Subject: [PATCH 047/184] 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 df456021651b7d9c4b4458a15060a24bbfee9d73 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 31 Jul 2023 08:50:09 +0000 Subject: [PATCH 048/184] 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 af60b4bb313bc33771562b3dd830395548952041 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 31 Jul 2023 10:29:15 +0000 Subject: [PATCH 049/184] 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 9fb86f134b8e72efc214e9089a523e8bd866ff18 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 31 Jul 2023 12:04:27 +0000 Subject: [PATCH 050/184] 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 6a1cf5c6830be1f6a06210f2ef89564c7c6f0e3d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 31 Jul 2023 13:27:06 +0000 Subject: [PATCH 051/184] 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 76331a9b490daea5cf87f8089dc0acfeef85582d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 31 Jul 2023 14:20:03 +0000 Subject: [PATCH 052/184] 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 86a251931bf5b026589beca33a6c0d001ff0cc51 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 31 Jul 2023 14:53:40 +0000 Subject: [PATCH 053/184] 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 054/184] 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 fe6202c1cc505076165729ae14bc3841c2ee04e7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 31 Jul 2023 17:49:34 +0000 Subject: [PATCH 055/184] 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 731b780b84876a7fbb9d7c9a9cd10fe197ea65e0 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Mon, 31 Jul 2023 21:06:13 +0300 Subject: [PATCH 056/184] 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 2ed74507e77a308912cd9341b4e4c5d3ac5cd6fd Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Mon, 31 Jul 2023 22:04:54 +0300 Subject: [PATCH 057/184] 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 71e58699b5f3f8e61a8b748e2172bdf9ad00b4ac Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 1 Aug 2023 07:52:50 +0000 Subject: [PATCH 058/184] 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 946f2f0b3919bc4022195896fa4f843a6b08317a Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 1 Aug 2023 12:24:08 +0300 Subject: [PATCH 059/184] 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 060/184] 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 505b4d01121d3d6cfd48d281a68fe00ca9ecadf2 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 1 Aug 2023 13:39:23 +0300 Subject: [PATCH 061/184] 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 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 062/184] 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 9027ab968eaa3389c75d8717a031e28c86b213f6 Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Tue, 1 Aug 2023 15:04:53 +0300 Subject: [PATCH 063/184] 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 c24dc8e64eddf453f1a0524236e8497a7bdbcf62 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 1 Aug 2023 13:48:49 +0000 Subject: [PATCH 064/184] 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 5cc9600bbf718374ea8ca51d697d03328ac1e921 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Jul 2023 18:10:56 +0200 Subject: [PATCH 065/184] 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 aaba48f0df13aa9c314d604affc1ac491c4db5dd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Jul 2023 17:19:33 +0000 Subject: [PATCH 066/184] 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 9423976b7a88e23bd98078760a23cb86bcb179a3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 2 Aug 2023 07:40:59 +0000 Subject: [PATCH 067/184] 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 068/184] 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 069/184] 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 6e49d3add69aa77057b6ef47081aed87392404e2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 2 Aug 2023 12:34:43 +0000 Subject: [PATCH 070/184] 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 08aa90aa29760ca56f9520254f44f48cefaec13c Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Wed, 2 Aug 2023 17:26:59 +0300 Subject: [PATCH 071/184] 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 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 072/184] 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 073/184] 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 074/184] 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 075/184] 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 d43e76b14720ebdb1e8ad2668be71306e9e38d45 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 2 Aug 2023 16:47:01 +0000 Subject: [PATCH 076/184] 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 fe09a36f4447e4a2cd141b3cac9fb2b0ef995b16 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 2 Aug 2023 18:07:54 +0000 Subject: [PATCH 077/184] 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 078/184] 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 8e3c98ca7a78614edfe185b19a6037b83b7da9bf Mon Sep 17 00:00:00 2001 From: Victor Krasnov Date: Thu, 3 Aug 2023 11:25:26 +0300 Subject: [PATCH 079/184] 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 5cdc50b9ba3c85f448ff673ef2dd8f604874b734 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 3 Aug 2023 11:31:15 +0200 Subject: [PATCH 080/184] 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 081/184] 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 d9f136ce0ca640f65f37b6097366901d120db77c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 3 Aug 2023 10:15:02 +0000 Subject: [PATCH 082/184] 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 27701ed24e68157b85f0f104afc069d364751428 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 3 Aug 2023 11:44:24 +0000 Subject: [PATCH 083/184] 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 2a681f1577b70c7e0f250fdfa3e2ea2d9239d172 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 3 Aug 2023 14:32:03 +0200 Subject: [PATCH 084/184] 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 5c7788b1547647e0e0696dac85ea73e84ad0f306 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 3 Aug 2023 12:42:50 +0000 Subject: [PATCH 085/184] 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 13d5ef91641fc02e737a729e520eff483f4dd837 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 3 Aug 2023 14:43:03 +0000 Subject: [PATCH 086/184] 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 087/184] 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 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 088/184] 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 14a8ced90640519dea8843597435f376145aa88c Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 3 Aug 2023 12:29:31 -0400 Subject: [PATCH 089/184] 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 1e7daca85b84447a4a5577cb545d42fe31486df6 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 3 Aug 2023 22:06:33 +0000 Subject: [PATCH 090/184] 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 091/184] 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 092/184] 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 8e7eb7f3fa739ef4d02594ca25fccb815a706a73 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Aug 2023 08:59:55 +0000 Subject: [PATCH 093/184] 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 3cea474ccfb64e05b37dc9243d771bd75c2a635d Mon Sep 17 00:00:00 2001 From: Val Doroshchuk Date: Wed, 26 Jul 2023 15:05:41 +0200 Subject: [PATCH 094/184] 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 095/184] 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 096/184] 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 097/184] 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 098/184] 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 099/184] 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 100/184] 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 101/184] 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 102/184] 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 569adafd9a9e4d0c6ad52e7970321e94f230da0c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Aug 2023 10:33:25 +0000 Subject: [PATCH 103/184] 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 8aaa46ca9055a52f2541107af6171d67fe90170c Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 4 Aug 2023 11:07:33 +0000 Subject: [PATCH 104/184] 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 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 105/184] 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 106/184] 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 107/184] 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 108/184] 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 109/184] 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 110/184] 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 49ae853ae67009b817cac4f2c52848d328b38f25 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Aug 2023 13:57:18 +0000 Subject: [PATCH 111/184] 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 02ed17dfa54b46b6439401fffd1591d77b07e2b7 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 4 Aug 2023 14:16:33 +0000 Subject: [PATCH 112/184] 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 21b4f828733d9e7509dad717bdb592da9c79db53 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 4 Aug 2023 14:28:58 +0000 Subject: [PATCH 113/184] 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 114/184] 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 115/184] 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 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 116/184] 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 06717fa3ddf7f96cb2e6c34ed8454df05bb686cb Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 4 Aug 2023 22:58:22 +0200 Subject: [PATCH 117/184] 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 555c90a50c200a99d143485a950794238a8dd11c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 4 Aug 2023 23:26:52 +0200 Subject: [PATCH 118/184] 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 119/184] 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 120/184] 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 712ac83542ad65b41d8bef73a734c283539c1b68 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Sat, 5 Aug 2023 01:01:23 +0000 Subject: [PATCH 121/184] 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 1d793e69f25e99ba62a5f31ebcbd90bd27b67ce5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Aug 2023 14:29:49 +0000 Subject: [PATCH 122/184] 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 fd5880920750fe667e4a50a7fc5df13270c16800 Mon Sep 17 00:00:00 2001 From: Tanay Tummalapalli Date: Tue, 20 Jun 2023 02:00:07 +0530 Subject: [PATCH 123/184] 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 124/184] 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 d977e221b42c74dc131e8e741bff86eb1ab3f146 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 7 Aug 2023 03:26:59 +0200 Subject: [PATCH 125/184] 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 126/184] 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 127/184] 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 128/184] 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 129/184] 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 130/184] 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 131/184] 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 80b681022c1715da0672f8c3da287fffa2335425 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Mon, 7 Aug 2023 12:20:44 +0200 Subject: [PATCH 132/184] 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 133/184] 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 134/184] 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 135/184] 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 136/184] 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 137/184] 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 138/184] 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 139/184] 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 140/184] 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 141/184] 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 142/184] 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 143/184] 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 144/184] 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 145/184] 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 146/184] 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 147/184] 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 148/184] 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 149/184] 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 150/184] 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 151/184] 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 152/184] 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 153/184] 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 154/184] 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 2f414950b7b17e85bca9f8db0084b4fb5263a174 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 7 Aug 2023 20:57:42 +0200 Subject: [PATCH 155/184] 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 156/184] 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 157/184] @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 158/184] 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 159/184] 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