From 0a46532005b5be94c85d264de7cb8a21a2a3dee0 Mon Sep 17 00:00:00 2001 From: liyang830 Date: Mon, 14 Mar 2022 18:25:28 +0800 Subject: [PATCH 001/777] add check copier drop partition success --- programs/copier/ClusterCopier.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 4d491a06795..b1cafd4ad6d 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -598,6 +598,8 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t ss << "ALTER TABLE " << getQuotedTable(original_table) << ((partition_name == "'all'") ? " DROP PARTITION ID " : " DROP PARTITION ") << partition_name; UInt64 num_shards_drop_partition = executeQueryOnCluster(task_table.cluster_push, ss.str(), task_cluster->settings_push, ClusterExecutionMode::ON_EACH_SHARD); + if (num_shards_drop_partition != task_table.cluster_push->getShardCount()) + return TaskStatus::Error; LOG_INFO(log, "Drop partition {} in original table {} have been executed successfully on {} shards of {}", partition_name, getQuotedTable(original_table), num_shards_drop_partition, task_table.cluster_push->getShardCount()); From 2081408c15e330ea737bc99a0afe7d96346c13d3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 16 Jan 2023 15:10:31 +0100 Subject: [PATCH 002/777] 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 003/777] 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 004/777] 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 005/777] 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 230f79a3229daf11c169c20be909c0ea4f04958e Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Wed, 10 May 2023 11:51:25 +0300 Subject: [PATCH 006/777] Fix bug in StorageBuffer::reschedule() --- src/Storages/StorageBuffer.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index a4cb15d5711..757d3161067 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -996,8 +996,11 @@ void StorageBuffer::reschedule() std::unique_lock lock(buffer.tryLock()); if (lock.owns_lock()) { - min_first_write_time = buffer.first_write_time; - rows += buffer.data.rows(); + if (buffer.data) + { + min_first_write_time = std::min(min_first_write_time, buffer.first_write_time); + rows += buffer.data.rows(); + } } } From fa3f8bc32e22c88937183e6d8925aacfe885cf2c Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Fri, 26 May 2023 07:04:44 -0700 Subject: [PATCH 007/777] Fix endian issue in native protocol --- .../Serializations/SerializationArray.cpp | 4 +-- .../SerializationDecimalBase.cpp | 33 +++++++++++++++---- .../SerializationLowCardinality.cpp | 22 ++++++------- .../Serializations/SerializationNumber.cpp | 33 ++++++++++++++++--- src/IO/ReadHelpers.cpp | 2 +- src/IO/ReadHelpers.h | 14 +++++++- src/IO/WriteHelpers.cpp | 2 +- src/IO/WriteHelpers.h | 19 ++++++++++- 8 files changed, 101 insertions(+), 28 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index cedcca870dd..e01c1aea0e9 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -129,7 +129,7 @@ namespace for (size_t i = offset; i < end; ++i) { ColumnArray::Offset current_offset = offset_values[i]; - writeIntBinary(current_offset - prev_offset, ostr); + writeBinaryLittleEndian(current_offset - prev_offset, ostr); prev_offset = current_offset; } } @@ -145,7 +145,7 @@ namespace while (i < initial_size + limit && !istr.eof()) { ColumnArray::Offset current_size = 0; - readIntBinary(current_size, istr); + readBinaryLittleEndian(current_size, istr); if (unlikely(current_size > MAX_ARRAY_SIZE)) throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size is too large: {}", current_size); diff --git a/src/DataTypes/Serializations/SerializationDecimalBase.cpp b/src/DataTypes/Serializations/SerializationDecimalBase.cpp index 642ea1c7cd8..494d0aa9168 100644 --- a/src/DataTypes/Serializations/SerializationDecimalBase.cpp +++ b/src/DataTypes/Serializations/SerializationDecimalBase.cpp @@ -15,14 +15,14 @@ template void SerializationDecimalBase::serializeBinary(const Field & field, WriteBuffer & ostr, const FormatSettings &) const { FieldType x = field.get>(); - writeBinary(x, ostr); + writeBinaryLittleEndian(x, ostr); } template void SerializationDecimalBase::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { const FieldType & x = assert_cast(column).getElement(row_num); - writeBinary(x, ostr); + writeBinaryLittleEndian(x, ostr); } template @@ -34,15 +34,26 @@ void SerializationDecimalBase::serializeBinaryBulk(const IColumn & column, Wr if (limit == 0 || offset + limit > size) limit = size - offset; - - ostr.write(reinterpret_cast(&x[offset]), sizeof(FieldType) * limit); + if constexpr (std::endian::native == std::endian::big && sizeof(T) >= 2) + { + for (size_t i = 0; i < limit; i++) + { + auto tmp(x[offset+i]); + char *start = reinterpret_cast(&tmp); + char *end = start + sizeof(FieldType); + std::reverse(start, end); + ostr.write(reinterpret_cast(&tmp), sizeof(FieldType)); + } + } + else + ostr.write(reinterpret_cast(&x[offset]), sizeof(FieldType) * limit); } template void SerializationDecimalBase::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { typename FieldType::NativeType x; - readBinary(x, istr); + readBinaryLittleEndian(x, istr); field = DecimalField(T(x), this->scale); } @@ -50,7 +61,7 @@ template void SerializationDecimalBase::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { typename FieldType::NativeType x; - readBinary(x, istr); + readBinaryLittleEndian(x, istr); assert_cast(column).getData().push_back(FieldType(x)); } @@ -61,6 +72,16 @@ void SerializationDecimalBase::deserializeBinaryBulk(IColumn & column, ReadBu size_t initial_size = x.size(); x.resize(initial_size + limit); size_t size = istr.readBig(reinterpret_cast(&x[initial_size]), sizeof(FieldType) * limit); + if constexpr (std::endian::native == std::endian::big && sizeof(T) >= 2) + { + for (size_t i = 0; i < limit; i++) + { + char *start = reinterpret_cast(&x[initial_size + i]); + char *end = start + sizeof(FieldType); + std::reverse(start, end); + } + } + x.resize(initial_size + size / sizeof(FieldType)); } diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index 90b51c1a6f2..7e8bed3ad82 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -132,13 +132,13 @@ struct IndexesSerializationType val |= NeedGlobalDictionaryBit; if (need_update_dictionary) val |= NeedUpdateDictionary; - writeIntBinary(val, buffer); + writeBinaryLittleEndian(val, buffer); } void deserialize(ReadBuffer & buffer, const ISerialization::DeserializeBinaryBulkSettings & settings) { SerializationType val; - readIntBinary(val, buffer); + readBinaryLittleEndian(val, buffer); checkType(val); has_additional_keys = (val & HasAdditionalKeysBit) != 0; @@ -235,7 +235,7 @@ void SerializationLowCardinality::serializeBinaryBulkStatePrefix( /// Write version and create SerializeBinaryBulkState. UInt64 key_version = KeysSerializationVersion::SharedDictionariesWithAdditionalKeys; - writeIntBinary(key_version, *stream); + writeBinaryLittleEndian(key_version, *stream); state = std::make_shared(key_version); } @@ -259,7 +259,7 @@ void SerializationLowCardinality::serializeBinaryBulkStateSuffix( throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty stream in SerializationLowCardinality::serializeBinaryBulkStateSuffix"); UInt64 num_keys = nested_column->size(); - writeIntBinary(num_keys, *stream); + writeBinaryLittleEndian(num_keys, *stream); dict_inner_serialization->serializeBinaryBulk(*nested_column, *stream, 0, num_keys); low_cardinality_state->shared_dictionary = nullptr; } @@ -277,7 +277,7 @@ void SerializationLowCardinality::deserializeBinaryBulkStatePrefix( return; UInt64 keys_version; - readIntBinary(keys_version, *stream); + readBinaryLittleEndian(keys_version, *stream); state = std::make_shared(keys_version); } @@ -535,7 +535,7 @@ void SerializationLowCardinality::serializeBinaryBulkWithMultipleStreams( { const auto & nested_column = global_dictionary->getNestedNotNullableColumn(); UInt64 num_keys = nested_column->size(); - writeIntBinary(num_keys, *keys_stream); + writeBinaryLittleEndian(num_keys, *keys_stream); dict_inner_serialization->serializeBinaryBulk(*nested_column, *keys_stream, 0, num_keys); low_cardinality_state->shared_dictionary = nullptr; } @@ -543,12 +543,12 @@ void SerializationLowCardinality::serializeBinaryBulkWithMultipleStreams( if (need_additional_keys) { UInt64 num_keys = keys->size(); - writeIntBinary(num_keys, *indexes_stream); + writeBinaryLittleEndian(num_keys, *indexes_stream); dict_inner_serialization->serializeBinaryBulk(*keys, *indexes_stream, 0, num_keys); } UInt64 num_rows = positions->size(); - writeIntBinary(num_rows, *indexes_stream); + writeBinaryLittleEndian(num_rows, *indexes_stream); auto index_serialization = index_version.getDataType()->getDefaultSerialization(); index_serialization->serializeBinaryBulk(*positions, *indexes_stream, 0, num_rows); } @@ -584,7 +584,7 @@ void SerializationLowCardinality::deserializeBinaryBulkWithMultipleStreams( auto read_dictionary = [this, low_cardinality_state, keys_stream]() { UInt64 num_keys; - readIntBinary(num_keys, *keys_stream); + readBinaryLittleEndian(num_keys, *keys_stream); auto keys_type = removeNullable(dictionary_type); auto global_dict_keys = keys_type->createColumn(); @@ -597,7 +597,7 @@ void SerializationLowCardinality::deserializeBinaryBulkWithMultipleStreams( auto read_additional_keys = [this, low_cardinality_state, indexes_stream]() { UInt64 num_keys; - readIntBinary(num_keys, *indexes_stream); + readBinaryLittleEndian(num_keys, *indexes_stream); auto keys_type = removeNullable(dictionary_type); auto additional_keys = keys_type->createColumn(); @@ -703,7 +703,7 @@ void SerializationLowCardinality::deserializeBinaryBulkWithMultipleStreams( else low_cardinality_state->additional_keys = nullptr; - readIntBinary(low_cardinality_state->num_pending_rows, *indexes_stream); + readBinaryLittleEndian(low_cardinality_state->num_pending_rows, *indexes_stream); } size_t num_rows_to_read = std::min(limit, low_cardinality_state->num_pending_rows); diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index 8cabaec753d..73cc5112596 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -106,28 +106,28 @@ void SerializationNumber::serializeBinary(const Field & field, WriteBuffer & { /// ColumnVector::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64 typename ColumnVector::ValueType x = static_cast::ValueType>(field.get()); - writeBinary(x, ostr); + writeBinaryLittleEndian(x, ostr); } template void SerializationNumber::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { typename ColumnVector::ValueType x; - readBinary(x, istr); + readBinaryLittleEndian(x, istr); field = NearestFieldType(x); } template void SerializationNumber::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeBinary(assert_cast &>(column).getData()[row_num], ostr); + writeBinaryLittleEndian(assert_cast &>(column).getData()[row_num], ostr); } template void SerializationNumber::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { typename ColumnVector::ValueType x; - readBinary(x, istr); + readBinaryLittleEndian(x, istr); assert_cast &>(column).getData().push_back(x); } @@ -142,7 +142,21 @@ void SerializationNumber::serializeBinaryBulk(const IColumn & column, WriteBu limit = size - offset; if (limit) - ostr.write(reinterpret_cast(&x[offset]), sizeof(typename ColumnVector::ValueType) * limit); + { + if constexpr (std::endian::native == std::endian::big && sizeof(T) >= 2) + { + for (size_t i = 0; i < limit; i++) + { + auto tmp(x[offset+i]); + char *start = reinterpret_cast(&tmp); + char *end = start + sizeof(typename ColumnVector::ValueType); + std::reverse(start, end); + ostr.write(reinterpret_cast(&tmp), sizeof(typename ColumnVector::ValueType)); + } + } + else + ostr.write(reinterpret_cast(&x[offset]), sizeof(typename ColumnVector::ValueType) * limit); + } } template @@ -152,6 +166,15 @@ void SerializationNumber::deserializeBinaryBulk(IColumn & column, ReadBuffer size_t initial_size = x.size(); x.resize(initial_size + limit); size_t size = istr.readBig(reinterpret_cast(&x[initial_size]), sizeof(typename ColumnVector::ValueType) * limit); + if constexpr (std::endian::native == std::endian::big && sizeof(T) >= 2) + { + for (size_t i = 0; i < limit; i++) + { + char *start = reinterpret_cast(&x[initial_size + i]); + char *end = start + sizeof(typename ColumnVector::ValueType); + std::reverse(start, end); + } + } x.resize(initial_size + size / sizeof(typename ColumnVector::ValueType)); } diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 99d25ee6613..48a558b2b70 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1353,7 +1353,7 @@ Exception readException(ReadBuffer & buf, const String & additional_message, boo String stack_trace; bool has_nested = false; /// Obsolete - readBinary(code, buf); + readBinaryLittleEndian(code, buf); readBinary(name, buf); readBinary(message, buf); readBinary(stack_trace, buf); diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 32338552b66..1feacd2d8be 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1100,7 +1100,7 @@ inline void readBinary(LocalDate & x, ReadBuffer & buf) { readPODBinary(x, buf); template -requires is_arithmetic_v && (sizeof(T) <= 8) +requires std::is_integral_v && (sizeof(T) <= 8) inline void readBinaryEndian(T & x, ReadBuffer & buf) { readPODBinary(x, buf); @@ -1108,6 +1108,18 @@ inline void readBinaryEndian(T & x, ReadBuffer & buf) x = std::byteswap(x); } +template +requires is_decimal || std::is_floating_point_v +inline void readBinaryEndian(T & x, ReadBuffer & buf) +{ + readPODBinary(x, buf); + if constexpr (std::endian::native != endian) + { + char *start = reinterpret_cast(&x), *end = start + sizeof(T); + std::reverse(start, end); + } +} + template requires is_big_int_v inline void readBinaryEndian(T & x, ReadBuffer & buf) diff --git a/src/IO/WriteHelpers.cpp b/src/IO/WriteHelpers.cpp index a0eceddc6f6..02c5ad48d43 100644 --- a/src/IO/WriteHelpers.cpp +++ b/src/IO/WriteHelpers.cpp @@ -62,7 +62,7 @@ void writeIPv6Text(const IPv6 & ip, WriteBuffer & buf) void writeException(const Exception & e, WriteBuffer & buf, bool with_stack_trace) { - writeBinary(e.code(), buf); + writeBinaryLittleEndian(e.code(), buf); writeBinary(String(e.name()), buf); writeBinary(e.displayText() + getExtraExceptionInfo(e), buf); diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index cdbc952690c..fdc3158c7f7 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1174,7 +1174,7 @@ inline void writeNullTerminatedString(const String & s, WriteBuffer & buffer) template -requires is_arithmetic_v && (sizeof(T) <= 8) +requires std::is_integral_v && (sizeof(T) <= 8) inline void writeBinaryEndian(T x, WriteBuffer & buf) { if constexpr (std::endian::native != endian) @@ -1182,6 +1182,23 @@ inline void writeBinaryEndian(T x, WriteBuffer & buf) writePODBinary(x, buf); } +template +requires is_decimal || std::is_floating_point_v +inline void writeBinaryEndian(T x, WriteBuffer & buf) +{ + if constexpr (std::endian::native != endian) + { + T tmp(x); + char *start = reinterpret_cast(&tmp); + char *end = start + sizeof(T); + std::reverse(start, end); + + buf.write(reinterpret_cast(&tmp), sizeof(tmp)); /// NOLINT + } + else + writePODBinary(x, buf); +} + template requires is_big_int_v inline void writeBinaryEndian(const T & x, WriteBuffer & buf) From 28fb99cd260dcf7d6c36386364645f19c3ef7ab6 Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Mon, 5 Jun 2023 13:12:19 -0700 Subject: [PATCH 008/777] Maintain same aggregate function mergeY behavior for small and big endian machine --- src/AggregateFunctions/UniquesHashSet.h | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/UniquesHashSet.h b/src/AggregateFunctions/UniquesHashSet.h index ca6d31a716d..369fce9374f 100644 --- a/src/AggregateFunctions/UniquesHashSet.h +++ b/src/AggregateFunctions/UniquesHashSet.h @@ -108,7 +108,13 @@ private: inline size_t buf_size() const { return 1ULL << size_degree; } /// NOLINT inline size_t max_fill() const { return 1ULL << (size_degree - 1); } /// NOLINT inline size_t mask() const { return buf_size() - 1; } - inline size_t place(HashValue x) const { return (x >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); } + + inline size_t place(HashValue x) const { + if constexpr (std::endian::native == std::endian::little) + return (x >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); + else + return (std::byteswap(x) >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); + } /// The value is divided by 2 ^ skip_degree inline bool good(HashValue hash) const From 55c526b6e644c9bab04972711efc2512a64cbfdd Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Wed, 5 Jul 2023 10:20:37 -0400 Subject: [PATCH 009/777] Update src/AggregateFunctions/UniquesHashSet.h Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- src/AggregateFunctions/UniquesHashSet.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/UniquesHashSet.h b/src/AggregateFunctions/UniquesHashSet.h index 369fce9374f..3e501b29414 100644 --- a/src/AggregateFunctions/UniquesHashSet.h +++ b/src/AggregateFunctions/UniquesHashSet.h @@ -109,7 +109,8 @@ private: inline size_t max_fill() const { return 1ULL << (size_degree - 1); } /// NOLINT inline size_t mask() const { return buf_size() - 1; } - inline size_t place(HashValue x) const { + inline size_t place(HashValue x) const + { if constexpr (std::endian::native == std::endian::little) return (x >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); else From 77e45e5669eae8ae05c5acdc5b0cbf6647d1f50a Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 10 Jul 2023 20:23:03 -0400 Subject: [PATCH 010/777] Add `DROP ROLE ... FROM` and `CREATE ROLE ... AT` syntax --- src/Access/AccessControl.cpp | 2 +- src/Access/DiskAccessStorage.h | 2 +- src/Access/MultipleAccessStorage.cpp | 35 +++++++++ src/Access/MultipleAccessStorage.h | 5 ++ src/Access/UsersConfigAccessStorage.h | 2 +- .../Access/InterpreterCreateRoleQuery.cpp | 42 +++++++++-- .../InterpreterDropAccessEntityQuery.cpp | 18 +++-- src/Parsers/Access/ASTCreateRoleQuery.cpp | 6 ++ src/Parsers/Access/ASTCreateRoleQuery.h | 1 + .../Access/ASTDropAccessEntityQuery.cpp | 5 ++ src/Parsers/Access/ASTDropAccessEntityQuery.h | 1 + src/Parsers/Access/ParserCreateRoleQuery.cpp | 5 ++ .../Access/ParserDropAccessEntityQuery.cpp | 5 ++ src/Parsers/Access/parseUserName.h | 5 ++ .../System/StorageSystemRoleGrants.cpp | 7 +- .../__init__.py | 0 .../configs/roles.xml | 13 ++++ .../test.py | 72 +++++++++++++++++++ 18 files changed, 211 insertions(+), 15 deletions(-) create mode 100644 tests/integration/test_multi_access_storage_role_management/__init__.py create mode 100644 tests/integration/test_multi_access_storage_role_management/configs/roles.xml create mode 100644 tests/integration/test_multi_access_storage_role_management/test.py diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 6179c823b56..0c61a9e3e1b 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -415,7 +415,7 @@ void AccessControl::addStoragesFromUserDirectoriesConfig( String type = key_in_user_directories; if (size_t bracket_pos = type.find('['); bracket_pos != String::npos) type.resize(bracket_pos); - if ((type == "users_xml") || (type == "users_config")) + if ((type == "users.xml") || (type == "users_config")) type = UsersConfigAccessStorage::STORAGE_TYPE; else if ((type == "local") || (type == "local_directory")) type = DiskAccessStorage::STORAGE_TYPE; diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index 069a966c8e9..e3bd32c0292 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -13,7 +13,7 @@ class AccessChangesNotifier; class DiskAccessStorage : public IAccessStorage { public: - static constexpr char STORAGE_TYPE[] = "local directory"; + static constexpr char STORAGE_TYPE[] = "local_directory"; DiskAccessStorage(const String & storage_name_, const String & directory_path_, AccessChangesNotifier & changes_notifier_, bool readonly_, bool allow_backup_); ~DiskAccessStorage() override; diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index f36a670ddbf..f36beb3f80b 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -16,6 +16,7 @@ namespace ErrorCodes { extern const int ACCESS_ENTITY_ALREADY_EXISTS; extern const int ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND; + extern const int ACCESS_ENTITY_NOT_FOUND; } using Storage = IAccessStorage; @@ -178,6 +179,40 @@ ConstStoragePtr MultipleAccessStorage::getStorage(const UUID & id) const return const_cast(this)->getStorage(id); } +StoragePtr MultipleAccessStorage::findStorageByName(const DB::String & storage_name) +{ + auto storages = getStoragesInternal(); + for (const auto & storage : *storages) + { + if (storage->getStorageName() == storage_name) + return storage; + } + + return nullptr; +} + + +ConstStoragePtr MultipleAccessStorage::findStorageByName(const DB::String & storage_name) const +{ + return const_cast(this)->findStorageByName(storage_name); +} + + +StoragePtr MultipleAccessStorage::getStorageByName(const DB::String & storage_name) +{ + auto storage = findStorageByName(storage_name); + if (storage) + return storage; + + throw Exception(ErrorCodes::ACCESS_ENTITY_NOT_FOUND, "Access storage with name {} is not found", storage_name); +} + + +ConstStoragePtr MultipleAccessStorage::getStorageByName(const DB::String & storage_name) const +{ + return const_cast(this)->getStorageByName(storage_name); +} + AccessEntityPtr MultipleAccessStorage::readImpl(const UUID & id, bool throw_if_not_exists) const { if (auto storage = findStorage(id)) diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index 6a0c1bdfc02..1898c6ba5d3 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -41,6 +41,11 @@ public: ConstStoragePtr getStorage(const UUID & id) const; StoragePtr getStorage(const UUID & id); + ConstStoragePtr findStorageByName(const String & storage_name) const; + StoragePtr findStorageByName(const String & storage_name); + ConstStoragePtr getStorageByName(const String & storage_name) const; + StoragePtr getStorageByName(const String & storage_name); + bool exists(const UUID & id) const override; bool isBackupAllowed() const override; diff --git a/src/Access/UsersConfigAccessStorage.h b/src/Access/UsersConfigAccessStorage.h index b533ccbf200..d8ddf07ff40 100644 --- a/src/Access/UsersConfigAccessStorage.h +++ b/src/Access/UsersConfigAccessStorage.h @@ -20,7 +20,7 @@ class UsersConfigAccessStorage : public IAccessStorage { public: - static constexpr char STORAGE_TYPE[] = "users.xml"; + static constexpr char STORAGE_TYPE[] = "users_xml"; UsersConfigAccessStorage(const String & storage_name_, AccessControl & access_control_, bool allow_backup_); ~UsersConfigAccessStorage() override; diff --git a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp index 3386dfb8792..87abe002d3d 100644 --- a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp @@ -8,6 +8,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int ACCESS_ENTITY_ALREADY_EXISTS; +} + namespace { void updateRoleFromQueryImpl( @@ -52,6 +58,15 @@ BlockIO InterpreterCreateRoleQuery::execute() if (!query.cluster.empty()) return executeDDLQueryOnCluster(query_ptr, getContext()); + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + + if (!query.storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(query.storage_name); + storage = storage_ptr.get(); + } + if (query.alter) { auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr @@ -62,11 +77,11 @@ BlockIO InterpreterCreateRoleQuery::execute() }; if (query.if_exists) { - auto ids = access_control.find(query.names); - access_control.tryUpdate(ids, update_func); + auto ids = storage->find(query.names); + storage->tryUpdate(ids, update_func); } else - access_control.update(access_control.getIDs(query.names), update_func); + storage->update(storage->getIDs(query.names), update_func); } else { @@ -78,12 +93,27 @@ BlockIO InterpreterCreateRoleQuery::execute() new_roles.emplace_back(std::move(new_role)); } + if (!query.storage_name.empty()) + { + for (const auto & name : query.names) + { + auto id = access_control.find(name); + + if (!id) + continue; + + auto another_storage_ptr = access_control.findStorage(*id); + if (another_storage_ptr != storage_ptr) + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Role {} already exists in storage {}", name, another_storage_ptr->getStorageName()); + } + } + if (query.if_not_exists) - access_control.tryInsert(new_roles); + storage->tryInsert(new_roles); else if (query.or_replace) - access_control.insertOrReplace(new_roles); + storage->insertOrReplace(new_roles); else - access_control.insert(new_roles); + storage->insert(new_roles); } return {}; diff --git a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp index d4c37064065..54e3b95226c 100644 --- a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp @@ -26,18 +26,26 @@ BlockIO InterpreterDropAccessEntityQuery::execute() query.replaceEmptyDatabase(getContext()->getCurrentDatabase()); - auto do_drop = [&](const Strings & names) + auto do_drop = [&](const Strings & names, const String & storage_name) { + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + if (!storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(storage_name); + storage = storage_ptr.get(); + } + if (query.if_exists) - access_control.tryRemove(access_control.find(query.type, names)); + storage->tryRemove(storage->find(query.type, names)); else - access_control.remove(access_control.getIDs(query.type, names)); + storage->remove(storage->getIDs(query.type, names)); }; if (query.type == AccessEntityType::ROW_POLICY) - do_drop(query.row_policy_names->toStrings()); + do_drop(query.row_policy_names->toStrings(), query.storage_name); else - do_drop(query.names); + do_drop(query.names, query.storage_name); return {}; } diff --git a/src/Parsers/Access/ASTCreateRoleQuery.cpp b/src/Parsers/Access/ASTCreateRoleQuery.cpp index d624b9a9157..14946f2d3cd 100644 --- a/src/Parsers/Access/ASTCreateRoleQuery.cpp +++ b/src/Parsers/Access/ASTCreateRoleQuery.cpp @@ -71,6 +71,12 @@ void ASTCreateRoleQuery::formatImpl(const FormatSettings & format, FormatState & format.ostr << (format.hilite ? hilite_keyword : "") << " OR REPLACE" << (format.hilite ? hilite_none : ""); formatNames(names, format); + + if (!storage_name.empty()) + format.ostr << (format.hilite ? IAST::hilite_keyword : "") + << " AT " << (format.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(format); if (!new_name.empty()) diff --git a/src/Parsers/Access/ASTCreateRoleQuery.h b/src/Parsers/Access/ASTCreateRoleQuery.h index 42d1a4031b6..4e465553164 100644 --- a/src/Parsers/Access/ASTCreateRoleQuery.h +++ b/src/Parsers/Access/ASTCreateRoleQuery.h @@ -28,6 +28,7 @@ public: Strings names; String new_name; + String storage_name; std::shared_ptr settings; diff --git a/src/Parsers/Access/ASTDropAccessEntityQuery.cpp b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp index 88f2d7bce63..02b29f3fd2c 100644 --- a/src/Parsers/Access/ASTDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp @@ -53,6 +53,11 @@ void ASTDropAccessEntityQuery::formatImpl(const FormatSettings & settings, Forma else formatNames(names, settings); + if (!storage_name.empty()) + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") + << " FROM " << (settings.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(settings); } diff --git a/src/Parsers/Access/ASTDropAccessEntityQuery.h b/src/Parsers/Access/ASTDropAccessEntityQuery.h index f3a065c50df..32f4a8f8047 100644 --- a/src/Parsers/Access/ASTDropAccessEntityQuery.h +++ b/src/Parsers/Access/ASTDropAccessEntityQuery.h @@ -21,6 +21,7 @@ public: AccessEntityType type; bool if_exists = false; Strings names; + String storage_name; std::shared_ptr row_policy_names; String getID(char) const override; diff --git a/src/Parsers/Access/ParserCreateRoleQuery.cpp b/src/Parsers/Access/ParserCreateRoleQuery.cpp index da9749958ee..de9e5baed18 100644 --- a/src/Parsers/Access/ParserCreateRoleQuery.cpp +++ b/src/Parsers/Access/ParserCreateRoleQuery.cpp @@ -91,6 +91,10 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec String new_name; std::shared_ptr settings; String cluster; + String storage_name; + + if (ParserKeyword{"AT"}.ignore(pos, expected)) + parseStorageName(pos, expected, storage_name); while (true) { @@ -125,6 +129,7 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->names = std::move(names); query->new_name = std::move(new_name); query->settings = std::move(settings); + query->storage_name = std::move(storage_name); return true; } diff --git a/src/Parsers/Access/ParserDropAccessEntityQuery.cpp b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp index d21164bc1a2..d5ecdf27449 100644 --- a/src/Parsers/Access/ParserDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp @@ -53,6 +53,7 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & Strings names; std::shared_ptr row_policy_names; + String storage_name; String cluster; if ((type == AccessEntityType::USER) || (type == AccessEntityType::ROLE)) @@ -76,6 +77,9 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & return false; } + if (ParserKeyword{"FROM"}.ignore(pos, expected)) + parseStorageName(pos, expected, storage_name); + if (cluster.empty()) parseOnCluster(pos, expected, cluster); @@ -87,6 +91,7 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & query->cluster = std::move(cluster); query->names = std::move(names); query->row_policy_names = std::move(row_policy_names); + query->storage_name = std::move(storage_name); return true; } diff --git a/src/Parsers/Access/parseUserName.h b/src/Parsers/Access/parseUserName.h index fbad78e7bce..717911bf2f1 100644 --- a/src/Parsers/Access/parseUserName.h +++ b/src/Parsers/Access/parseUserName.h @@ -34,4 +34,9 @@ inline bool parseRoleNames(IParser::Pos & pos, Expected & expected, Strings & ro return parseUserNames(pos, expected, role_names); } +inline bool parseStorageName(IParser::Pos & pos, Expected & expected, String & storage_name) +{ + return parseUserName(pos, expected, storage_name); +} + } diff --git a/src/Storages/System/StorageSystemRoleGrants.cpp b/src/Storages/System/StorageSystemRoleGrants.cpp index cf5a24f88cd..8fcd0fd7cf5 100644 --- a/src/Storages/System/StorageSystemRoleGrants.cpp +++ b/src/Storages/System/StorageSystemRoleGrants.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -22,6 +23,7 @@ NamesAndTypesList StorageSystemRoleGrants::getNamesAndTypes() {"user_name", std::make_shared(std::make_shared())}, {"role_name", std::make_shared(std::make_shared())}, {"granted_role_name", std::make_shared()}, + {"granted_role_id", std::make_shared()}, {"granted_role_is_default", std::make_shared()}, {"with_admin_option", std::make_shared()}, }; @@ -45,12 +47,14 @@ void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr auto & column_role_name = assert_cast(assert_cast(*res_columns[column_index]).getNestedColumn()); auto & column_role_name_null_map = assert_cast(*res_columns[column_index++]).getNullMapData(); auto & column_granted_role_name = assert_cast(*res_columns[column_index++]); + auto & column_granted_role_id = assert_cast(*res_columns[column_index++]).getData(); auto & column_is_default = assert_cast(*res_columns[column_index++]).getData(); auto & column_admin_option = assert_cast(*res_columns[column_index++]).getData(); auto add_row = [&](const String & grantee_name, AccessEntityType grantee_type, const String & granted_role_name, + const UUID & granted_role_id, bool is_default, bool with_admin_option) { @@ -72,6 +76,7 @@ void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr assert(false); column_granted_role_name.insertData(granted_role_name.data(), granted_role_name.length()); + column_granted_role_id.push_back(granted_role_id.toUnderType()); column_is_default.push_back(is_default); column_admin_option.push_back(with_admin_option); }; @@ -90,7 +95,7 @@ void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr continue; bool is_default = !default_roles || default_roles->match(role_id); - add_row(grantee_name, grantee_type, *role_name, is_default, element.admin_option); + add_row(grantee_name, grantee_type, *role_name, role_id, is_default, element.admin_option); } } }; diff --git a/tests/integration/test_multi_access_storage_role_management/__init__.py b/tests/integration/test_multi_access_storage_role_management/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_multi_access_storage_role_management/configs/roles.xml b/tests/integration/test_multi_access_storage_role_management/configs/roles.xml new file mode 100644 index 00000000000..672c7e24728 --- /dev/null +++ b/tests/integration/test_multi_access_storage_role_management/configs/roles.xml @@ -0,0 +1,13 @@ + + + + + + + + + GRANT ALL ON *.* WITH GRANT OPTION + + + + diff --git a/tests/integration/test_multi_access_storage_role_management/test.py b/tests/integration/test_multi_access_storage_role_management/test.py new file mode 100644 index 00000000000..473a37695e9 --- /dev/null +++ b/tests/integration/test_multi_access_storage_role_management/test.py @@ -0,0 +1,72 @@ +import pytest +import os +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV +from helpers.client import QueryRuntimeException + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + stay_alive=True, +) + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + node.query("CREATE DATABASE mydb") + node.query("CREATE TABLE mydb.table1(x UInt32) ENGINE=Log") + + node.query("CREATE USER test_user") + node.query("CREATE USER test_user2") + yield cluster + + finally: + cluster.shutdown() + + +def test_role_from_different_storages(): + node.query("CREATE ROLE default_role") + node.query("GRANT SELECT ON system.* TO default_role") + + assert node.query("SHOW GRANTS FOR default_role") == TSV(["GRANT SELECT ON system.* TO default_role"]) + assert node.query("SHOW ROLES") == TSV(["default_role"]) + + node.query("GRANT default_role TO test_user") + + node.copy_file_to_container( + os.path.join(SCRIPT_DIR, "configs/roles.xml"), + "/etc/clickhouse-server/users.d/roles.xml", + ) + + node.restart_clickhouse() + + assert node.query("SELECT name, storage FROM system.roles") == TSV( + [ + ["default_role", "users_xml"], + ["default_role", "local_directory"] + ] + ) + + # Role from users.xml will have priority + assert node.query("SHOW GRANTS FOR default_role") == TSV(["GRANT ALL ON *.* TO default_role WITH GRANT OPTION"]) + + node.query("GRANT default_role TO test_user") + node.query("GRANT default_role TO test_user2") + assert node.query("SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user2'") == TSV( + ["62bedbf3-7fb1-94cb-3a35-e479693223b3"] # roles from users.xml have deterministic ids + ) + + node.query("DROP ROLE default_role FROM local_directory") + assert node.query("SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user'") == TSV(["62bedbf3-7fb1-94cb-3a35-e479693223b3"]) + + # Already exists + with pytest.raises(QueryRuntimeException): + node.query("CREATE ROLE default_role AT memory") + + node.query("CREATE ROLE other_role AT memory") + + assert node.query("SELECT storage FROM system.roles WHERE name = 'other_role'") == TSV(["memory"]) From ee591ef7f52f3f53dd8cb59a6acf33a734561f65 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 11 Jul 2023 00:33:34 +0000 Subject: [PATCH 011/777] Automatic style fix --- .../test.py | 29 ++++++++++++------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_multi_access_storage_role_management/test.py b/tests/integration/test_multi_access_storage_role_management/test.py index 473a37695e9..d9a4a73c56b 100644 --- a/tests/integration/test_multi_access_storage_role_management/test.py +++ b/tests/integration/test_multi_access_storage_role_management/test.py @@ -32,7 +32,9 @@ def test_role_from_different_storages(): node.query("CREATE ROLE default_role") node.query("GRANT SELECT ON system.* TO default_role") - assert node.query("SHOW GRANTS FOR default_role") == TSV(["GRANT SELECT ON system.* TO default_role"]) + assert node.query("SHOW GRANTS FOR default_role") == TSV( + ["GRANT SELECT ON system.* TO default_role"] + ) assert node.query("SHOW ROLES") == TSV(["default_role"]) node.query("GRANT default_role TO test_user") @@ -45,23 +47,28 @@ def test_role_from_different_storages(): node.restart_clickhouse() assert node.query("SELECT name, storage FROM system.roles") == TSV( - [ - ["default_role", "users_xml"], - ["default_role", "local_directory"] - ] + [["default_role", "users_xml"], ["default_role", "local_directory"]] ) # Role from users.xml will have priority - assert node.query("SHOW GRANTS FOR default_role") == TSV(["GRANT ALL ON *.* TO default_role WITH GRANT OPTION"]) + assert node.query("SHOW GRANTS FOR default_role") == TSV( + ["GRANT ALL ON *.* TO default_role WITH GRANT OPTION"] + ) node.query("GRANT default_role TO test_user") node.query("GRANT default_role TO test_user2") - assert node.query("SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user2'") == TSV( - ["62bedbf3-7fb1-94cb-3a35-e479693223b3"] # roles from users.xml have deterministic ids + assert node.query( + "SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user2'" + ) == TSV( + [ + "62bedbf3-7fb1-94cb-3a35-e479693223b3" + ] # roles from users.xml have deterministic ids ) node.query("DROP ROLE default_role FROM local_directory") - assert node.query("SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user'") == TSV(["62bedbf3-7fb1-94cb-3a35-e479693223b3"]) + assert node.query( + "SELECT granted_role_id FROM system.role_grants WHERE user_name = 'test_user'" + ) == TSV(["62bedbf3-7fb1-94cb-3a35-e479693223b3"]) # Already exists with pytest.raises(QueryRuntimeException): @@ -69,4 +76,6 @@ def test_role_from_different_storages(): node.query("CREATE ROLE other_role AT memory") - assert node.query("SELECT storage FROM system.roles WHERE name = 'other_role'") == TSV(["memory"]) + assert node.query( + "SELECT storage FROM system.roles WHERE name = 'other_role'" + ) == TSV(["memory"]) From 0e9f0e6837cbbb9c7fa784fc9139451ce9779c57 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Tue, 11 Jul 2023 11:09:01 -0700 Subject: [PATCH 012/777] Fix ForEach Aggregate state for s390x --- src/AggregateFunctions/AggregateFunctionForEach.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionForEach.h b/src/AggregateFunctions/AggregateFunctionForEach.h index 480b4cc690e..ec59fd2e616 100644 --- a/src/AggregateFunctions/AggregateFunctionForEach.h +++ b/src/AggregateFunctions/AggregateFunctionForEach.h @@ -240,7 +240,7 @@ public: void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { const AggregateFunctionForEachData & state = data(place); - writeBinary(state.dynamic_array_size, buf); + writeBinaryLittleEndian(state.dynamic_array_size, buf); const char * nested_state = state.array_of_aggregate_datas; for (size_t i = 0; i < state.dynamic_array_size; ++i) @@ -255,7 +255,7 @@ public: AggregateFunctionForEachData & state = data(place); size_t new_size = 0; - readBinary(new_size, buf); + readBinaryLittleEndian(new_size, buf); ensureAggregateData(place, new_size, *arena); From 1a4d48324e3ffe1f8c0c05a337b590ce8aa9ff33 Mon Sep 17 00:00:00 2001 From: pufit Date: Tue, 11 Jul 2023 19:58:28 -0400 Subject: [PATCH 013/777] Fix fast tests --- .../queries/0_stateless/01293_create_role.reference | 2 +- .../01294_create_settings_profile.reference | 12 ++++++------ .../0_stateless/01295_create_row_policy.reference | 6 +++--- .../queries/0_stateless/01297_create_quota.reference | 8 ++++---- .../02117_show_create_table_system.reference | 1 + 5 files changed, 15 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/01293_create_role.reference b/tests/queries/0_stateless/01293_create_role.reference index 9b3c4eabd47..42f091bddd5 100644 --- a/tests/queries/0_stateless/01293_create_role.reference +++ b/tests/queries/0_stateless/01293_create_role.reference @@ -28,7 +28,7 @@ CREATE ROLE r2_01293 CREATE ROLE r1_01293 SETTINGS readonly = 1 CREATE ROLE r2_01293 SETTINGS readonly = 1 -- system.roles -r1_01293 local directory +r1_01293 local_directory -- system.settings_profile_elements \N \N r1_01293 0 readonly 1 \N \N \N \N \N \N r2_01293 0 \N \N \N \N \N default diff --git a/tests/queries/0_stateless/01294_create_settings_profile.reference b/tests/queries/0_stateless/01294_create_settings_profile.reference index dbb73bca851..a10d5758752 100644 --- a/tests/queries/0_stateless/01294_create_settings_profile.reference +++ b/tests/queries/0_stateless/01294_create_settings_profile.reference @@ -53,12 +53,12 @@ CREATE SETTINGS PROFILE s4_01294 SETTINGS INHERIT readonly, INHERIT readonly CREATE SETTINGS PROFILE s5_01294 SETTINGS INHERIT readonly, readonly = 1 CREATE SETTINGS PROFILE s6_01294 SETTINGS INHERIT readonly, readonly CONST -- system.settings_profiles -s1_01294 local directory 0 0 [] [] -s2_01294 local directory 1 0 ['r1_01294'] [] -s3_01294 local directory 1 0 ['r1_01294'] [] -s4_01294 local directory 1 0 ['r1_01294'] [] -s5_01294 local directory 3 0 ['u1_01294'] [] -s6_01294 local directory 0 1 [] ['r1_01294','u1_01294'] +s1_01294 local_directory 0 0 [] [] +s2_01294 local_directory 1 0 ['r1_01294'] [] +s3_01294 local_directory 1 0 ['r1_01294'] [] +s4_01294 local_directory 1 0 ['r1_01294'] [] +s5_01294 local_directory 3 0 ['u1_01294'] [] +s6_01294 local_directory 0 1 [] ['r1_01294','u1_01294'] -- system.settings_profile_elements s2_01294 \N \N 0 readonly 0 \N \N \N \N s3_01294 \N \N 0 max_memory_usage 5000000 4000000 6000000 CONST \N diff --git a/tests/queries/0_stateless/01295_create_row_policy.reference b/tests/queries/0_stateless/01295_create_row_policy.reference index 6e3169b7fec..d73d9752bc1 100644 --- a/tests/queries/0_stateless/01295_create_row_policy.reference +++ b/tests/queries/0_stateless/01295_create_row_policy.reference @@ -30,6 +30,6 @@ CREATE ROW POLICY p5_01295 ON db2.table2 FOR SELECT USING a = b CREATE ROW POLICY p1_01295 ON db.table FOR SELECT USING 1 TO ALL CREATE ROW POLICY p2_01295 ON db.table FOR SELECT USING 1 TO ALL -- system.row_policies -p1_01295 ON db.table p1_01295 db table local directory (a < b) AND (c > d) 0 0 [] [] -p2_01295 ON db.table p2_01295 db table local directory id = currentUser() 1 0 ['u1_01295'] [] -p3_01295 ON db.table p3_01295 db table local directory 1 0 1 [] ['r1_01295'] +p1_01295 ON db.table p1_01295 db table local_directory (a < b) AND (c > d) 0 0 [] [] +p2_01295 ON db.table p2_01295 db table local_directory id = currentUser() 1 0 ['u1_01295'] [] +p3_01295 ON db.table p3_01295 db table local_directory 1 0 1 [] ['r1_01295'] diff --git a/tests/queries/0_stateless/01297_create_quota.reference b/tests/queries/0_stateless/01297_create_quota.reference index 6b844e0d2f6..308bbf79024 100644 --- a/tests/queries/0_stateless/01297_create_quota.reference +++ b/tests/queries/0_stateless/01297_create_quota.reference @@ -52,10 +52,10 @@ CREATE QUOTA q2_01297 FOR INTERVAL 1 day MAX errors = 5 CREATE QUOTA q1_01297 FOR INTERVAL 1 day TRACKING ONLY TO r1_01297 CREATE QUOTA q2_01297 FOR INTERVAL 1 day TRACKING ONLY TO r1_01297 -- system.quotas -q1_01297 local directory ['user_name'] [] 0 ['r1_01297'] [] -q2_01297 local directory [] [5259492] 0 ['r1_01297','u1_01297'] [] -q3_01297 local directory ['client_key','user_name'] [5259492,15778476] 0 [] [] -q4_01297 local directory [] [604800] 1 [] ['u1_01297'] +q1_01297 local_directory ['user_name'] [] 0 ['r1_01297'] [] +q2_01297 local_directory [] [5259492] 0 ['r1_01297','u1_01297'] [] +q3_01297 local_directory ['client_key','user_name'] [5259492,15778476] 0 [] [] +q4_01297 local_directory [] [604800] 1 [] ['u1_01297'] -- system.quota_limits q2_01297 5259492 0 100 \N \N 11 1000 10000 1001 10001 2.5 \N q3_01297 5259492 0 \N \N \N \N 1002 \N \N \N \N \N diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 3834b05601f..aaff6d92d32 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -955,6 +955,7 @@ CREATE TABLE system.role_grants `user_name` Nullable(String), `role_name` Nullable(String), `granted_role_name` String, + `granted_role_id` UUID, `granted_role_is_default` UInt8, `with_admin_option` UInt8 ) From 1376293e908b164f27e656c3f4f117f74966d737 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 7 Jul 2023 14:35:07 +0000 Subject: [PATCH 014/777] Fix mistakenly comma parsing as part of datetime in CSV best effort parsing --- .../Serializations/SerializationDateTime.cpp | 27 +++++++++++++++---- .../SerializationDateTime64.cpp | 27 +++++++++++++++---- .../02812_csv_date_time_with_comma.reference | 2 ++ .../02812_csv_date_time_with_comma.sql | 3 +++ 4 files changed, 49 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02812_csv_date_time_with_comma.reference create mode 100644 tests/queries/0_stateless/02812_csv_date_time_with_comma.sql diff --git a/src/DataTypes/Serializations/SerializationDateTime.cpp b/src/DataTypes/Serializations/SerializationDateTime.cpp index 7238d3ce190..dd3d97d6aa1 100644 --- a/src/DataTypes/Serializations/SerializationDateTime.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB { @@ -145,12 +146,28 @@ void SerializationDateTime::deserializeTextCSV(IColumn & column, ReadBuffer & is char maybe_quote = *istr.position(); if (maybe_quote == '\'' || maybe_quote == '\"') - ++istr.position(); - - readText(x, istr, settings, time_zone, utc_time_zone); - - if (maybe_quote == '\'' || maybe_quote == '\"') + { + readText(x, istr, settings, time_zone, utc_time_zone); assertChar(maybe_quote, istr); + } + else + { + if (settings.csv.delimiter != ',' || settings.date_time_input_format == FormatSettings::DateTimeInputFormat::Basic) + { + readText(x, istr, settings, time_zone, utc_time_zone); + } + /// Best effort parsing supports datetime in format like "01.01.2000, 00:00:00" + /// and can mistakenly read comma as a part of datetime. + /// For example data "...,01.01.2000,some string,..." cannot be parsed correctly. + /// To fix this problem we first read CSV string and then try to parse it as datetime. + else + { + String datetime_str; + readCSVString(datetime_str, istr, settings.csv); + ReadBufferFromString buf(datetime_str); + readText(x, buf, settings, time_zone, utc_time_zone); + } + } if (x < 0) x = 0; diff --git a/src/DataTypes/Serializations/SerializationDateTime64.cpp b/src/DataTypes/Serializations/SerializationDateTime64.cpp index 78c7ea56529..91078c73f80 100644 --- a/src/DataTypes/Serializations/SerializationDateTime64.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime64.cpp @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB { @@ -143,12 +144,28 @@ void SerializationDateTime64::deserializeTextCSV(IColumn & column, ReadBuffer & char maybe_quote = *istr.position(); if (maybe_quote == '\'' || maybe_quote == '\"') - ++istr.position(); - - readText(x, scale, istr, settings, time_zone, utc_time_zone); - - if (maybe_quote == '\'' || maybe_quote == '\"') + { + readText(x, scale, istr, settings, time_zone, utc_time_zone); assertChar(maybe_quote, istr); + } + else + { + if (settings.csv.delimiter != ',' || settings.date_time_input_format == FormatSettings::DateTimeInputFormat::Basic) + { + readText(x, scale, istr, settings, time_zone, utc_time_zone); + } + /// Best effort parsing supports datetime in format like "01.01.2000, 00:00:00" + /// and can mistakenly read comma as a part of datetime. + /// For example data "...,01.01.2000,some string,..." cannot be parsed correctly. + /// To fix this problem we first read CSV string and then try to parse it as datetime. + else + { + String datetime_str; + readCSVString(datetime_str, istr, settings.csv); + ReadBufferFromString buf(datetime_str); + readText(x, scale, buf, settings, time_zone, utc_time_zone); + } + } assert_cast(column).getData().push_back(x); } diff --git a/tests/queries/0_stateless/02812_csv_date_time_with_comma.reference b/tests/queries/0_stateless/02812_csv_date_time_with_comma.reference new file mode 100644 index 00000000000..f569df13dc1 --- /dev/null +++ b/tests/queries/0_stateless/02812_csv_date_time_with_comma.reference @@ -0,0 +1,2 @@ +2000-01-01 00:00:00 abc +2000-01-01 00:00:00.000 abc diff --git a/tests/queries/0_stateless/02812_csv_date_time_with_comma.sql b/tests/queries/0_stateless/02812_csv_date_time_with_comma.sql new file mode 100644 index 00000000000..ecd3cff6ad0 --- /dev/null +++ b/tests/queries/0_stateless/02812_csv_date_time_with_comma.sql @@ -0,0 +1,3 @@ +select * from format(CSV, 'c1 DateTime, c2 String', '01-01-2000,abc') settings date_time_input_format='best_effort'; +select * from format(CSV, 'c1 DateTime64(3), c2 String', '01-01-2000,abc') settings date_time_input_format='best_effort'; + From d2b52eeeef5a3a3aaf314cd585cb58375df3973f Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 17 Jul 2023 17:11:49 +0000 Subject: [PATCH 015/777] Fix --- src/DataTypes/Serializations/SerializationDateTime.cpp | 1 + src/DataTypes/Serializations/SerializationDateTime64.cpp | 1 + 2 files changed, 2 insertions(+) diff --git a/src/DataTypes/Serializations/SerializationDateTime.cpp b/src/DataTypes/Serializations/SerializationDateTime.cpp index dd3d97d6aa1..2ba24f5351b 100644 --- a/src/DataTypes/Serializations/SerializationDateTime.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime.cpp @@ -147,6 +147,7 @@ void SerializationDateTime::deserializeTextCSV(IColumn & column, ReadBuffer & is if (maybe_quote == '\'' || maybe_quote == '\"') { + ++istr.position(); readText(x, istr, settings, time_zone, utc_time_zone); assertChar(maybe_quote, istr); } diff --git a/src/DataTypes/Serializations/SerializationDateTime64.cpp b/src/DataTypes/Serializations/SerializationDateTime64.cpp index 91078c73f80..c5964f1bd97 100644 --- a/src/DataTypes/Serializations/SerializationDateTime64.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime64.cpp @@ -145,6 +145,7 @@ void SerializationDateTime64::deserializeTextCSV(IColumn & column, ReadBuffer & if (maybe_quote == '\'' || maybe_quote == '\"') { + ++istr.position(); readText(x, scale, istr, settings, time_zone, utc_time_zone); assertChar(maybe_quote, istr); } From 8e3bd222f00324667a1d41fb62f50adc91045b6c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:42:32 +0200 Subject: [PATCH 016/777] Add SQLTest --- .github/workflows/pull_request.yml | 36 +++++++ docker/images.json | 7 +- docker/test/sqltest/Dockerfile | 29 ++++++ docker/test/sqltest/run.sh | 41 ++++++++ docker/test/sqltest/test.py | 118 ++++++++++++++++++++++ tests/ci/ast_fuzzer_check.py | 2 +- tests/ci/sqltest.py | 157 +++++++++++++++++++++++++++++ tests/ci/tests/docker_images.json | 4 + 8 files changed, 391 insertions(+), 3 deletions(-) create mode 100644 docker/test/sqltest/Dockerfile create mode 100755 docker/test/sqltest/run.sh create mode 100644 docker/test/sqltest/test.py create mode 100644 tests/ci/sqltest.py diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index dd834959578..b3283c3afbe 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -5182,3 +5182,39 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" +############################################################################################## +##################################### SQL TEST ############################################### +############################################################################################## + SQLTest: + needs: [BuilderDebRelease] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/sqltest + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME="SQLTest" + REPO_COPY=${{runner.temp}}/sqltest/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v3 + with: + path: ${{ env.REPORTS_PATH }} + - name: Check out repository code + uses: ClickHouse/checkout@v1 + with: + clear-repository: true + - name: SQLTest + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 sqltest.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" diff --git a/docker/images.json b/docker/images.json index e8fc329a640..663fa21a6c9 100644 --- a/docker/images.json +++ b/docker/images.json @@ -155,11 +155,14 @@ }, "docker/docs/builder": { "name": "clickhouse/docs-builder", - "dependent": [ - ] + "dependent": [] }, "docker/test/sqllogic": { "name": "clickhouse/sqllogic-test", "dependent": [] + }, + "docker/test/sqltest": { + "name": "clickhouse/sqltest", + "dependent": [] } } diff --git a/docker/test/sqltest/Dockerfile b/docker/test/sqltest/Dockerfile new file mode 100644 index 00000000000..c58907a98a8 --- /dev/null +++ b/docker/test/sqltest/Dockerfile @@ -0,0 +1,29 @@ +# docker build -t clickhouse/sqltest . +ARG FROM_TAG=latest +FROM clickhouse/test-base:$FROM_TAG + +RUN apt-get update --yes \ + && env DEBIAN_FRONTEND=noninteractive \ + apt-get install --yes --no-install-recommends \ + wget \ + git \ + python3 \ + python3-dev \ + python3-pip \ + sudo \ + && apt-get clean + +RUN pip3 install \ + pyyaml \ + clickhouse-driver + +ARG sqltest_repo="https://github.com/elliotchance/sqltest/" + +RUN git clone ${sqltest_repo} + +ENV TZ=UTC +ENV MAX_RUN_TIME=900 +RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone + +COPY run.sh / +CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh new file mode 100755 index 00000000000..ef9ecaec33b --- /dev/null +++ b/docker/test/sqltest/run.sh @@ -0,0 +1,41 @@ +#!/bin/bash + +set -x +set -e +set -u +set -o pipefail + +BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-16_debug_none_unsplitted_disable_False_binary"} +BINARY_URL_TO_DOWNLOAD=${BINARY_URL_TO_DOWNLOAD:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/$BINARY_TO_DOWNLOAD/clickhouse"} + +function wget_with_retry +{ + for _ in 1 2 3 4; do + if wget -nv -nd -c "$1";then + return 0 + else + sleep 0.5 + fi + done + return 1 +} + +wget_with_retry "$BINARY_URL_TO_DOWNLOAD" +chmod +x clickhouse +./clickhouse install --noninteractive +clickhouse start + +# Wait for start +for _ in {1..100} +do + clickhouse-client --query "SELECT 1" && break ||: + sleep 1 +done + +# Run the test +pushd sqltest/standards/2016/ +./test.py +popd + +zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.log +zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.err.log diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py new file mode 100644 index 00000000000..7b38fdc52ad --- /dev/null +++ b/docker/test/sqltest/test.py @@ -0,0 +1,118 @@ +#!/usr/bin/env python3 + +import os +import yaml +import random +import string +from clickhouse_driver import Client + + +client = Client(host = 'localhost', port = 9000) +settings = {'default_table_engine': 'Memory', 'union_default_mode': 'DISTINCT', 'calculate_text_stack_trace': 0} + +database_name = 'sqltest_' + ''.join(random.choice(string.ascii_lowercase) for _ in range(10)) + +client.execute(f"DROP DATABASE IF EXISTS {database_name}", settings = settings) +client.execute(f"CREATE DATABASE {database_name}", settings = settings) + +client = Client(host = 'localhost', port = 9000, database = database_name) + +summary = {'success': 0, 'total': 0, 'results': {}} + +log_file = open("test.log", "w") +report_html_file = open("report.html", "w") + +with open('features.yml', 'r') as file: + yaml_content = yaml.safe_load(file) + + for category in yaml_content: + log_file.write(category.capitalize() + " features:") + summary['results'][category] = {'success': 0, 'total': 0, 'results': {}} + + for test in yaml_content[category]: + log_file.write(test + ": " + yaml_content[category][test]) + summary['results'][category]['results'][test] = {'success': 0, 'total': 0, 'description': yaml_content[category][test]} + + test_path = test[0] + "/" + test + ".tests.yml" + if os.path.exists(test_path): + with open(test_path, 'r') as test_file: + test_yaml_content = yaml.load_all(test_file, Loader = yaml.FullLoader) + + for test_case in test_yaml_content: + + queries = test_case['sql']; + if not isinstance(queries, list): + queries = [queries] + + for query in queries: + # Example: E011-01 + test_group = '' + if '-' in test: + test_group = test.split("-", 1)[0] + summary['results'][category]['results'][test_group]['total'] += 1 + summary['results'][category]['results'][test]['total'] += 1 + summary['results'][category]['total'] += 1 + summary['total'] += 1 + + log_file.write(query) + + try: + result = client.execute(query, settings = settings) + log_file.write(result) + + if test_group: + summary['results'][category]['results'][test_group]['success'] += 1 + summary['results'][category]['results'][test]['success'] += 1 + summary['results'][category]['success'] += 1 + summary['success'] += 1 + + except Exception as e: + log_file.write(f"Error occurred: {str(e)}") + +client.execute(f"DROP DATABASE {database_name}", settings = settings) + +def enable_color(ratio): + if ratio == 0: + return "" + elif ratio < 0.5: + return "" + elif ratio < 1: + return "" + else: + return "" + +reset_color = "" + +def print_ratio(indent, name, success, total, description): + report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}
".format( + ' ' * indent, + name, + enable_color(success / total), + success, + total, + success / total, + reset_color, + f" - {description}" if description else '')) + + +report_html_file.write("

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

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

") +report_html_file.write("

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

") +report_html_file.write("
") From 50dac21483b9003ad546da6d87340124bf108d69 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 03:54:09 +0200 Subject: [PATCH 021/777] Improvement --- docker/test/sqltest/test.py | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index 945446358c3..ae2d6406c78 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -2,6 +2,7 @@ import os import yaml +import html import random import string from clickhouse_driver import Client @@ -26,11 +27,11 @@ with open('features.yml', 'r') as file: yaml_content = yaml.safe_load(file) for category in yaml_content: - log_file.write(category.capitalize() + " features:") + log_file.write(category.capitalize() + " features:\n") summary['results'][category] = {'success': 0, 'total': 0, 'results': {}} for test in yaml_content[category]: - log_file.write(test + ": " + yaml_content[category][test]) + log_file.write(test + ": " + yaml_content[category][test] + "\n") summary['results'][category]['results'][test] = {'success': 0, 'total': 0, 'description': yaml_content[category][test]} test_path = test[0] + "/" + test + ".tests.yml" @@ -54,11 +55,11 @@ with open('features.yml', 'r') as file: summary['results'][category]['total'] += 1 summary['total'] += 1 - log_file.write(query) + log_file.write(query + "\n") try: result = client.execute(query, settings = settings) - log_file.write(result) + log_file.write(result + "\n") if test_group: summary['results'][category]['results'][test_group]['success'] += 1 @@ -67,7 +68,7 @@ with open('features.yml', 'r') as file: summary['success'] += 1 except Exception as e: - log_file.write(f"Error occurred: {str(e)}") + log_file.write(f"Error occurred: {str(e)}\n") client.execute(f"DROP DATABASE {database_name}", settings = settings) @@ -84,7 +85,7 @@ def enable_color(ratio): reset_color = "
" def print_ratio(indent, name, success, total, description): - report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}
".format( + report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}
\n".format( ' ' * indent, name, enable_color(success / total), @@ -92,10 +93,10 @@ def print_ratio(indent, name, success, total, description): total, success / total, reset_color, - f" - {description}" if description else '')) + f" - " + html.escape(description) if description else '')) -report_html_file.write("
")
+report_html_file.write("
\n")
 
 print_ratio(0, 'Total', summary['success'], summary['total'], '')
 
@@ -115,4 +116,4 @@ for category in summary['results']:
 
         print_ratio(4, test, test_summary['success'], test_summary['total'], test_summary['description'])
 
-report_html_file.write("
") +report_html_file.write("
\n") From 76677e009b8cf9a501fde1f7273fa5d546f4ffa3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 04:00:50 +0200 Subject: [PATCH 022/777] Improvement --- docker/test/sqltest/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index ae2d6406c78..cf2f976e05a 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -59,7 +59,7 @@ with open('features.yml', 'r') as file: try: result = client.execute(query, settings = settings) - log_file.write(result + "\n") + log_file.write(str(result) + "\n") if test_group: summary['results'][category]['results'][test_group]['success'] += 1 From fe5972953703edb8a7e13c6066e5af369359b82c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 04:01:00 +0200 Subject: [PATCH 023/777] Improvement --- docker/test/sqltest/run.sh | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index 24032874403..03678971f60 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -23,6 +23,12 @@ function wget_with_retry wget_with_retry "$BINARY_URL_TO_DOWNLOAD" chmod +x clickhouse ./clickhouse install --noninteractive + +echo " +users: + default: + access_management: 1" > /etc/clickhouse-server/users.d/access_management.yaml + clickhouse start # Wait for start From 9b10e0a5bc2b7e3f84614d1aa3fa84c60de76929 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 04:09:19 +0200 Subject: [PATCH 024/777] Improvement --- docker/test/sqltest/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index cf2f976e05a..ff321d3e5df 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -85,9 +85,9 @@ def enable_color(ratio): reset_color = "
" def print_ratio(indent, name, success, total, description): - report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}
\n".format( + report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}\n".format( ' ' * indent, - name, + name.capitalize(), enable_color(success / total), success, total, @@ -96,7 +96,7 @@ def print_ratio(indent, name, success, total, description): f" - " + html.escape(description) if description else '')) -report_html_file.write("
\n")
+report_html_file.write("
\n")
 
 print_ratio(0, 'Total', summary['success'], summary['total'], '')
 
@@ -114,6 +114,6 @@ for category in summary['results']:
         if test_summary['total'] == 0:
             continue
 
-        print_ratio(4, test, test_summary['success'], test_summary['total'], test_summary['description'])
+        print_ratio(6 if '-' in test else 4, test, test_summary['success'], test_summary['total'], test_summary['description'])
 
 report_html_file.write("
\n") From 891c01e6ab30c2d4377a118cb4c449134a56e17a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 19 Jul 2023 02:37:02 +0000 Subject: [PATCH 025/777] Automatic style fix --- docker/test/sqltest/test.py | 119 ++++++++++++++++++++++-------------- 1 file changed, 74 insertions(+), 45 deletions(-) diff --git a/docker/test/sqltest/test.py b/docker/test/sqltest/test.py index ff321d3e5df..5807ca79b02 100755 --- a/docker/test/sqltest/test.py +++ b/docker/test/sqltest/test.py @@ -8,69 +8,85 @@ import string from clickhouse_driver import Client -client = Client(host = 'localhost', port = 9000) -settings = {'default_table_engine': 'Memory', 'union_default_mode': 'DISTINCT', 'calculate_text_stack_trace': 0} +client = Client(host="localhost", port=9000) +settings = { + "default_table_engine": "Memory", + "union_default_mode": "DISTINCT", + "calculate_text_stack_trace": 0, +} -database_name = 'sqltest_' + ''.join(random.choice(string.ascii_lowercase) for _ in range(10)) +database_name = "sqltest_" + "".join( + random.choice(string.ascii_lowercase) for _ in range(10) +) -client.execute(f"DROP DATABASE IF EXISTS {database_name}", settings = settings) -client.execute(f"CREATE DATABASE {database_name}", settings = settings) +client.execute(f"DROP DATABASE IF EXISTS {database_name}", settings=settings) +client.execute(f"CREATE DATABASE {database_name}", settings=settings) -client = Client(host = 'localhost', port = 9000, database = database_name) +client = Client(host="localhost", port=9000, database=database_name) -summary = {'success': 0, 'total': 0, 'results': {}} +summary = {"success": 0, "total": 0, "results": {}} log_file = open("test.log", "w") report_html_file = open("report.html", "w") -with open('features.yml', 'r') as file: +with open("features.yml", "r") as file: yaml_content = yaml.safe_load(file) for category in yaml_content: log_file.write(category.capitalize() + " features:\n") - summary['results'][category] = {'success': 0, 'total': 0, 'results': {}} + summary["results"][category] = {"success": 0, "total": 0, "results": {}} for test in yaml_content[category]: log_file.write(test + ": " + yaml_content[category][test] + "\n") - summary['results'][category]['results'][test] = {'success': 0, 'total': 0, 'description': yaml_content[category][test]} + summary["results"][category]["results"][test] = { + "success": 0, + "total": 0, + "description": yaml_content[category][test], + } test_path = test[0] + "/" + test + ".tests.yml" if os.path.exists(test_path): - with open(test_path, 'r') as test_file: - test_yaml_content = yaml.load_all(test_file, Loader = yaml.FullLoader) + with open(test_path, "r") as test_file: + test_yaml_content = yaml.load_all(test_file, Loader=yaml.FullLoader) for test_case in test_yaml_content: - - queries = test_case['sql']; + queries = test_case["sql"] if not isinstance(queries, list): queries = [queries] for query in queries: # Example: E011-01 - test_group = '' - if '-' in test: + test_group = "" + if "-" in test: test_group = test.split("-", 1)[0] - summary['results'][category]['results'][test_group]['total'] += 1 - summary['results'][category]['results'][test]['total'] += 1 - summary['results'][category]['total'] += 1 - summary['total'] += 1 + summary["results"][category]["results"][test_group][ + "total" + ] += 1 + summary["results"][category]["results"][test]["total"] += 1 + summary["results"][category]["total"] += 1 + summary["total"] += 1 log_file.write(query + "\n") try: - result = client.execute(query, settings = settings) + result = client.execute(query, settings=settings) log_file.write(str(result) + "\n") if test_group: - summary['results'][category]['results'][test_group]['success'] += 1 - summary['results'][category]['results'][test]['success'] += 1 - summary['results'][category]['success'] += 1 - summary['success'] += 1 + summary["results"][category]["results"][test_group][ + "success" + ] += 1 + summary["results"][category]["results"][test][ + "success" + ] += 1 + summary["results"][category]["success"] += 1 + summary["success"] += 1 except Exception as e: log_file.write(f"Error occurred: {str(e)}\n") -client.execute(f"DROP DATABASE {database_name}", settings = settings) +client.execute(f"DROP DATABASE {database_name}", settings=settings) + def enable_color(ratio): if ratio == 0: @@ -82,38 +98,51 @@ def enable_color(ratio): else: return "" + reset_color = "" + def print_ratio(indent, name, success, total, description): - report_html_file.write("{}{}: {}{} / {} ({:.1%}){}{}\n".format( - ' ' * indent, - name.capitalize(), - enable_color(success / total), - success, - total, - success / total, - reset_color, - f" - " + html.escape(description) if description else '')) + report_html_file.write( + "{}{}: {}{} / {} ({:.1%}){}{}\n".format( + " " * indent, + name.capitalize(), + enable_color(success / total), + success, + total, + success / total, + reset_color, + f" - " + html.escape(description) if description else "", + ) + ) -report_html_file.write("
\n")
+report_html_file.write(
+    "
\n"
+)
 
-print_ratio(0, 'Total', summary['success'], summary['total'], '')
+print_ratio(0, "Total", summary["success"], summary["total"], "")
 
-for category in summary['results']:
-    cat_summary = summary['results'][category]
+for category in summary["results"]:
+    cat_summary = summary["results"][category]
 
-    if cat_summary['total'] == 0:
+    if cat_summary["total"] == 0:
         continue
 
-    print_ratio(2, category, cat_summary['success'], cat_summary['total'], '')
+    print_ratio(2, category, cat_summary["success"], cat_summary["total"], "")
 
-    for test in summary['results'][category]['results']:
-        test_summary = summary['results'][category]['results'][test]
+    for test in summary["results"][category]["results"]:
+        test_summary = summary["results"][category]["results"][test]
 
-        if test_summary['total'] == 0:
+        if test_summary["total"] == 0:
             continue
 
-        print_ratio(6 if '-' in test else 4, test, test_summary['success'], test_summary['total'], test_summary['description'])
+        print_ratio(
+            6 if "-" in test else 4,
+            test,
+            test_summary["success"],
+            test_summary["total"],
+            test_summary["description"],
+        )
 
 report_html_file.write("
\n") From dd411d8f547bf647e0854f8345a5d5c1597d724e Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 19 Jul 2023 09:14:10 -0700 Subject: [PATCH 026/777] Implement endianness-indepedent support for MergeTree checksums --- .../Serializations/SerializationNumber.cpp | 31 ++++++++++++++----- .../MergeTree/MergeTreeDataPartChecksum.cpp | 28 ++++++++--------- .../MergeTreeDataPartWriterCompact.cpp | 3 +- 3 files changed, 40 insertions(+), 22 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index 8cabaec753d..ca2616f4582 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -10,6 +10,8 @@ #include #include +#include + namespace DB { @@ -135,13 +137,25 @@ template void SerializationNumber::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const { const typename ColumnVector::Container & x = typeid_cast &>(column).getData(); - - size_t size = x.size(); - - if (limit == 0 || offset + limit > size) + if (const size_t size = x.size(); limit == 0 || offset + limit > size) limit = size - offset; - if (limit) + if (limit == 0) + return; + + if constexpr (std::endian::native == std::endian::big && sizeof(T) >= 2) + { + static constexpr auto to_little_endian = [](auto i) + { + transformEndianness(i); + return i; + }; + + std::ranges::for_each( + x | std::views::drop(offset) | std::views::take(limit) | std::views::transform(to_little_endian), + [&ostr](const auto & i) { ostr.write(reinterpret_cast(&i), sizeof(typename ColumnVector::ValueType)); }); + } + else ostr.write(reinterpret_cast(&x[offset]), sizeof(typename ColumnVector::ValueType) * limit); } @@ -149,10 +163,13 @@ template void SerializationNumber::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double /*avg_value_size_hint*/) const { typename ColumnVector::Container & x = typeid_cast &>(column).getData(); - size_t initial_size = x.size(); + const size_t initial_size = x.size(); x.resize(initial_size + limit); - size_t size = istr.readBig(reinterpret_cast(&x[initial_size]), sizeof(typename ColumnVector::ValueType) * limit); + const size_t size = istr.readBig(reinterpret_cast(&x[initial_size]), sizeof(typename ColumnVector::ValueType) * limit); x.resize(initial_size + size / sizeof(typename ColumnVector::ValueType)); + + if constexpr (std::endian::native == std::endian::big && sizeof(T) >= 2) + std::ranges::for_each(x | std::views::drop(initial_size), [](auto & i) { transformEndianness(i); }); } template class SerializationNumber; diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 6628cd68eaf..5a7b2dfbca8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -187,15 +187,15 @@ bool MergeTreeDataPartChecksums::readV3(ReadBuffer & in) String name; Checksum sum; - readBinary(name, in); + readStringBinary(name, in); readVarUInt(sum.file_size, in); - readPODBinary(sum.file_hash, in); - readBinary(sum.is_compressed, in); + readBinaryLittleEndian(sum.file_hash, in); + readBinaryLittleEndian(sum.is_compressed, in); if (sum.is_compressed) { readVarUInt(sum.uncompressed_size, in); - readPODBinary(sum.uncompressed_hash, in); + readBinaryLittleEndian(sum.uncompressed_hash, in); } files.emplace(std::move(name), sum); @@ -223,15 +223,15 @@ void MergeTreeDataPartChecksums::write(WriteBuffer & to) const const String & name = it.first; const Checksum & sum = it.second; - writeBinary(name, out); + writeStringBinary(name, out); writeVarUInt(sum.file_size, out); - writePODBinary(sum.file_hash, out); - writeBinary(sum.is_compressed, out); + writeBinaryLittleEndian(sum.file_hash, out); + writeBinaryLittleEndian(sum.is_compressed, out); if (sum.is_compressed) { writeVarUInt(sum.uncompressed_size, out); - writePODBinary(sum.uncompressed_hash, out); + writeBinaryLittleEndian(sum.uncompressed_hash, out); } } } @@ -339,9 +339,9 @@ void MinimalisticDataPartChecksums::serializeWithoutHeader(WriteBuffer & to) con writeVarUInt(num_compressed_files, to); writeVarUInt(num_uncompressed_files, to); - writePODBinary(hash_of_all_files, to); - writePODBinary(hash_of_uncompressed_files, to); - writePODBinary(uncompressed_hash_of_compressed_files, to); + writeBinaryLittleEndian(hash_of_all_files, to); + writeBinaryLittleEndian(hash_of_uncompressed_files, to); + writeBinaryLittleEndian(uncompressed_hash_of_compressed_files, to); } String MinimalisticDataPartChecksums::getSerializedString() const @@ -382,9 +382,9 @@ void MinimalisticDataPartChecksums::deserializeWithoutHeader(ReadBuffer & in) readVarUInt(num_compressed_files, in); readVarUInt(num_uncompressed_files, in); - readPODBinary(hash_of_all_files, in); - readPODBinary(hash_of_uncompressed_files, in); - readPODBinary(uncompressed_hash_of_compressed_files, in); + readBinaryLittleEndian(hash_of_all_files, in); + readBinaryLittleEndian(hash_of_uncompressed_files, in); + readBinaryLittleEndian(uncompressed_hash_of_compressed_files, in); } void MinimalisticDataPartChecksums::computeTotalChecksums(const MergeTreeDataPartChecksums & full_checksums_) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 5e1da21da5b..75e6aca0793 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -365,8 +365,9 @@ void MergeTreeDataPartWriterCompact::addToChecksums(MergeTreeDataPartChecksums & { uncompressed_size += stream->hashing_buf.count(); auto stream_hash = stream->hashing_buf.getHash(); + transformEndianness(stream_hash); uncompressed_hash = CityHash_v1_0_2::CityHash128WithSeed( - reinterpret_cast(&stream_hash), sizeof(stream_hash), uncompressed_hash); + reinterpret_cast(&stream_hash), sizeof(stream_hash), uncompressed_hash); } checksums.files[data_file_name].is_compressed = true; From 51e2c58a539a31a6268f2d4f50fd91701caaa915 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 21 Jun 2023 20:15:12 -0700 Subject: [PATCH 027/777] Implement endianness-independent SipHash and MergeTree checksum serialization --- programs/obfuscator/Obfuscator.cpp | 13 +- .../AggregateFunctionUniq.h | 3 +- src/AggregateFunctions/UniqVariadicHash.h | 8 +- src/Analyzer/IQueryTreeNode.cpp | 5 +- src/Client/QueryFuzzer.cpp | 3 +- src/Columns/ColumnUnique.h | 3 +- src/Common/SipHash.h | 112 +++++++++--------- src/Common/TransformEndianness.hpp | 3 +- src/Common/examples/hashes_test.cpp | 3 +- src/Common/getHashOfLoadedBinary.cpp | 3 +- src/Common/randomSeed.cpp | 2 +- src/DataTypes/Serializations/PathInData.cpp | 4 +- src/Functions/PolygonUtils.h | 4 +- src/Functions/array/arrayDistinct.cpp | 3 +- src/Functions/array/arrayEnumerateRanked.h | 6 +- src/IO/MMappedFileCache.h | 5 +- src/IO/UncompressedCache.h | 5 +- src/Interpreters/AggregationCommon.h | 6 +- src/Interpreters/AsynchronousInsertQueue.cpp | 4 +- src/Parsers/IAST.cpp | 4 +- .../Impl/ConstantExpressionTemplate.cpp | 3 +- .../Transforms/LimitByTransform.cpp | 5 +- src/Storages/LiveView/LiveViewSink.h | 7 +- src/Storages/LiveView/StorageLiveView.cpp | 3 +- src/Storages/MarkCache.h | 6 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 10 +- src/Storages/MergeTree/MarkRange.cpp | 12 +- .../MergeTree/MergeTreeDataPartChecksum.cpp | 15 +-- .../MergeTree/MergeTreeDataPartCompact.cpp | 2 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 2 +- .../MergeTree/MergeTreeDataPartWide.cpp | 6 +- .../MergeTreeDataPartWriterCompact.cpp | 12 +- .../MergeTreeDataPartWriterOnDisk.cpp | 6 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 12 +- .../MergeTree/MergeTreeMarksLoader.cpp | 12 +- src/Storages/MergeTree/MergeTreePartition.cpp | 10 +- .../ReplicatedMergeTreePartHeader.cpp | 4 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 9 +- 38 files changed, 138 insertions(+), 197 deletions(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 3042ae2bb57..43b31843afe 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -365,17 +365,14 @@ static void transformFixedString(const UInt8 * src, UInt8 * dst, size_t size, UI hash.update(seed); hash.update(i); + const auto checksum = getSipHash128AsArray(hash); if (size >= 16) { - char * hash_dst = reinterpret_cast(std::min(pos, end - 16)); - hash.get128(hash_dst); + auto * hash_dst = std::min(pos, end - 16); + memcpy(hash_dst, checksum.data(), checksum.size()); } else - { - char value[16]; - hash.get128(value); - memcpy(dst, value, end - dst); - } + memcpy(dst, checksum.data(), end - dst); pos += 16; ++i; @@ -401,7 +398,7 @@ static void transformUUID(const UUID & src_uuid, UUID & dst_uuid, UInt64 seed) hash.update(reinterpret_cast(&src), sizeof(UUID)); /// Saving version and variant from an old UUID - hash.get128(reinterpret_cast(&dst)); + // hash.get128Impl(reinterpret_cast(&dst)); dst.items[1] = (dst.items[1] & 0x1fffffffffffffffull) | (src.items[1] & 0xe000000000000000ull); dst.items[0] = (dst.items[0] & 0xffffffffffff0fffull) | (src.items[0] & 0x000000000000f000ull); diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index de68e9076a0..8310f02767c 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -298,10 +298,9 @@ struct Adder { StringRef value = column.getDataAt(row_num); - UInt128 key; SipHash hash; hash.update(value.data, value.size); - hash.get128(key); + const auto key = hash.get128(); data.set.template insert(key); } diff --git a/src/AggregateFunctions/UniqVariadicHash.h b/src/AggregateFunctions/UniqVariadicHash.h index 94f54a7a059..840380e7f0f 100644 --- a/src/AggregateFunctions/UniqVariadicHash.h +++ b/src/AggregateFunctions/UniqVariadicHash.h @@ -107,9 +107,7 @@ struct UniqVariadicHash ++column; } - UInt128 key; - hash.get128(key); - return key; + return hash.get128(); } }; @@ -131,9 +129,7 @@ struct UniqVariadicHash ++column; } - UInt128 key; - hash.get128(key); - return key; + return hash.get128(); } }; diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index f1056975f7f..7e5d778f1c1 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -229,10 +229,7 @@ IQueryTreeNode::Hash IQueryTreeNode::getTreeHash() const } } - Hash result; - hash_state.get128(result); - - return result; + return getSipHash128AsLoHi(hash_state); } QueryTreeNodePtr IQueryTreeNode::clone() const diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 5ce95c82528..f20c869e119 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -521,8 +521,7 @@ void QueryFuzzer::fuzzCreateQuery(ASTCreateQuery & create) if (create.storage) create.storage->updateTreeHash(sip_hash); - IAST::Hash hash; - sip_hash.get128(hash); + const auto hash = getSipHash128AsLoHi(sip_hash); /// Save only tables with unique definition. if (created_tables_hashes.insert(hash).second) diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 377255d80c7..fd0144cba12 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -670,8 +670,9 @@ UInt128 ColumnUnique::IncrementalHash::getHash(const ColumnType & co for (size_t i = 0; i < column_size; ++i) column.updateHashWithValue(i, sip_hash); + hash = sip_hash.get128(); + std::lock_guard lock(mutex); - sip_hash.get128(hash); cur_hash = hash; num_added_rows.store(column_size); } diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index cdec00d4bcc..e9d45c7201f 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -13,6 +13,8 @@ * (~ 700 MB/sec, 15 million strings per second) */ +#include "TransformEndianness.hpp" + #include #include #include @@ -22,14 +24,10 @@ #include #include - -namespace DB -{ -namespace ErrorCodes +namespace DB::ErrorCodes { extern const int LOGICAL_ERROR; } -} #define SIPROUND \ do \ @@ -90,6 +88,20 @@ private: SIPROUND; } + /// @brief Retrieves the result in some form with the endianness of the platform taken into account. + /// @warning This can only be done once! + void get128Impl(char * out) + { + finalize(); +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + unalignedStore(out + 8, v0 ^ v1); + unalignedStore(out, v2 ^ v3); +#else + unalignedStore(out, v0 ^ v1); + unalignedStore(out + 8, v2 ^ v3); +#endif + } + public: /// Arguments - seed. SipHash(UInt64 key0 = 0, UInt64 key1 = 0, bool is_reference_128_ = false) /// NOLINT @@ -161,60 +173,26 @@ public: } } - template + template ALWAYS_INLINE void update(const T & x) { if constexpr (std::endian::native == std::endian::big) { - T rev_x = x; - char *start = reinterpret_cast(&rev_x); - char *end = start + sizeof(T); - std::reverse(start, end); - update(reinterpret_cast(&rev_x), sizeof(rev_x)); /// NOLINT + auto transformed_x = x; + if constexpr (!std::is_same_v) + transformed_x = Transform()(x); + else + DB::transformEndianness(transformed_x); + + update(reinterpret_cast(&transformed_x), sizeof(transformed_x)); /// NOLINT } else update(reinterpret_cast(&x), sizeof(x)); /// NOLINT } - ALWAYS_INLINE void update(const std::string & x) - { - update(x.data(), x.length()); - } - - ALWAYS_INLINE void update(const std::string_view x) - { - update(x.data(), x.size()); - } - - /// Get the result in some form. This can only be done once! - - ALWAYS_INLINE void get128(char * out) - { - finalize(); -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - unalignedStore(out + 8, v0 ^ v1); - unalignedStore(out, v2 ^ v3); -#else - unalignedStore(out, v0 ^ v1); - unalignedStore(out + 8, v2 ^ v3); -#endif - } - - template - ALWAYS_INLINE void get128(T & lo, T & hi) - { - static_assert(sizeof(T) == 8); - finalize(); - lo = v0 ^ v1; - hi = v2 ^ v3; - } - - template - ALWAYS_INLINE void get128(T & dst) - { - static_assert(sizeof(T) == 16); - get128(reinterpret_cast(&dst)); - } + ALWAYS_INLINE void update(const std::string & x) { update(x.data(), x.length()); } + ALWAYS_INLINE void update(const std::string_view x) { update(x.data(), x.size()); } + ALWAYS_INLINE void update(const char * s) { update(std::string_view(s)); } UInt64 get64() { @@ -222,10 +200,23 @@ public: return v0 ^ v1 ^ v2 ^ v3; } + template + requires (sizeof(T) == 8) + ALWAYS_INLINE void get128(T & lo, T & hi) + { + finalize(); + lo = v0 ^ v1; + hi = v2 ^ v3; + } + UInt128 get128() { UInt128 res; - get128(res); +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + get128(res.items[1], res.items[0]); +#else + get128(res.items[0], res.items[1]); +#endif return res; } @@ -247,9 +238,7 @@ public: { lo = std::byteswap(lo); hi = std::byteswap(hi); - auto tmp = hi; - hi = lo; - lo = tmp; + std::swap(lo, hi); } UInt128 res = hi; @@ -265,11 +254,18 @@ public: #include -inline void sipHash128(const char * data, const size_t size, char * out) +inline std::array getSipHash128AsArray(SipHash & sip_hash) { - SipHash hash; - hash.update(data, size); - hash.get128(out); + std::array arr; + *reinterpret_cast(arr.data()) = sip_hash.get128(); + return arr; +} + +inline std::pair getSipHash128AsLoHi(SipHash & sip_hash) +{ + std::pair lo_hi; + sip_hash.get128(lo_hi.first, lo_hi.second); + return lo_hi; } inline UInt128 sipHash128Keyed(UInt64 key0, UInt64 key1, const char * data, const size_t size) diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index 0a9055dde15..ccfec78ab64 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -2,6 +2,7 @@ #include #include +#include #include @@ -46,7 +47,7 @@ inline void transformEndianness(T & value) } template -requires std::is_scoped_enum_v +requires std::is_enum_v || std::is_scoped_enum_v inline void transformEndianness(T & x) { using UnderlyingType = std::underlying_type_t; diff --git a/src/Common/examples/hashes_test.cpp b/src/Common/examples/hashes_test.cpp index eccf7c9b2e6..99479e79302 100644 --- a/src/Common/examples/hashes_test.cpp +++ b/src/Common/examples/hashes_test.cpp @@ -94,7 +94,8 @@ int main(int, char **) { SipHash hash; hash.update(strings[i].data(), strings[i].size()); - hash.get128(&hashes[i * 16]); + const auto hashed_value = getSipHash128AsArray(hash); + memcpy(&hashes[i * 16], hashed_value.data(), hashed_value.size()); } watch.stop(); diff --git a/src/Common/getHashOfLoadedBinary.cpp b/src/Common/getHashOfLoadedBinary.cpp index b81300b8536..6487bcd4f1c 100644 --- a/src/Common/getHashOfLoadedBinary.cpp +++ b/src/Common/getHashOfLoadedBinary.cpp @@ -37,8 +37,7 @@ SipHash getHashOfLoadedBinary() std::string getHashOfLoadedBinaryHex() { SipHash hash = getHashOfLoadedBinary(); - UInt128 checksum; - hash.get128(checksum); + const auto checksum = hash.get128(); return getHexUIntUppercase(checksum); } diff --git a/src/Common/randomSeed.cpp b/src/Common/randomSeed.cpp index 9f0ffd8a6c7..e1aa56fa811 100644 --- a/src/Common/randomSeed.cpp +++ b/src/Common/randomSeed.cpp @@ -39,7 +39,7 @@ DB::UInt64 randomSeed() #if defined(__linux__) struct utsname sysinfo; if (uname(&sysinfo) == 0) - hash.update(sysinfo); + hash.update(sysinfo); #endif return hash.get64(); diff --git a/src/DataTypes/Serializations/PathInData.cpp b/src/DataTypes/Serializations/PathInData.cpp index 2a5f508650f..cf78d7cbb14 100644 --- a/src/DataTypes/Serializations/PathInData.cpp +++ b/src/DataTypes/Serializations/PathInData.cpp @@ -65,9 +65,7 @@ UInt128 PathInData::getPartsHash(const Parts::const_iterator & begin, const Part hash.update(part_it->anonymous_array_level); } - UInt128 res; - hash.get128(res); - return res; + return hash.get128(); } void PathInData::buildPath(const Parts & other_parts) diff --git a/src/Functions/PolygonUtils.h b/src/Functions/PolygonUtils.h index 9aae42cce41..9c28e349413 100644 --- a/src/Functions/PolygonUtils.h +++ b/src/Functions/PolygonUtils.h @@ -635,9 +635,7 @@ UInt128 sipHash128(Polygon && polygon) for (auto & inner : inners) hash_ring(inner); - UInt128 res; - hash.get128(res); - return res; + return hash.get128(); } } diff --git a/src/Functions/array/arrayDistinct.cpp b/src/Functions/array/arrayDistinct.cpp index 527624794ea..ea331d6bdad 100644 --- a/src/Functions/array/arrayDistinct.cpp +++ b/src/Functions/array/arrayDistinct.cpp @@ -268,10 +268,9 @@ void FunctionArrayDistinct::executeHashed( if (nullable_col && (*src_null_map)[j]) continue; - UInt128 hash; SipHash hash_function; src_data.updateHashWithValue(j, hash_function); - hash_function.get128(hash); + const auto hash = hash_function.get128(); if (!set.find(hash)) { diff --git a/src/Functions/array/arrayEnumerateRanked.h b/src/Functions/array/arrayEnumerateRanked.h index 0733f1e2d43..3fc31e46f5e 100644 --- a/src/Functions/array/arrayEnumerateRanked.h +++ b/src/Functions/array/arrayEnumerateRanked.h @@ -134,18 +134,14 @@ private: /// Hash a set of keys into a UInt128 value. static inline UInt128 ALWAYS_INLINE hash128depths(const std::vector & indices, const ColumnRawPtrs & key_columns) { - UInt128 key; SipHash hash; - for (size_t j = 0, keys_size = key_columns.size(); j < keys_size; ++j) { // Debug: const auto & field = (*key_columns[j])[indices[j]]; DUMP(j, indices[j], field); key_columns[j]->updateHashWithValue(indices[j], hash); } - hash.get128(key); - - return key; + return hash.get128(); } diff --git a/src/IO/MMappedFileCache.h b/src/IO/MMappedFileCache.h index 0a8a80d15d0..bb30829ed69 100644 --- a/src/IO/MMappedFileCache.h +++ b/src/IO/MMappedFileCache.h @@ -33,15 +33,12 @@ public: /// Calculate key from path to file and offset. static UInt128 hash(const String & path_to_file, size_t offset, ssize_t length = -1) { - UInt128 key; - SipHash hash; hash.update(path_to_file.data(), path_to_file.size() + 1); hash.update(offset); hash.update(length); - hash.get128(key); - return key; + return hash.get128(); } template diff --git a/src/IO/UncompressedCache.h b/src/IO/UncompressedCache.h index 2e654b27ed7..b115a18014f 100644 --- a/src/IO/UncompressedCache.h +++ b/src/IO/UncompressedCache.h @@ -51,14 +51,11 @@ public: /// Calculate key from path to file and offset. static UInt128 hash(const String & path_to_file, size_t offset) { - UInt128 key; - SipHash hash; hash.update(path_to_file.data(), path_to_file.size() + 1); hash.update(offset); - hash.get128(key); - return key; + return hash.get128(); } template diff --git a/src/Interpreters/AggregationCommon.h b/src/Interpreters/AggregationCommon.h index 7ba9011f18b..77ca2c49b31 100644 --- a/src/Interpreters/AggregationCommon.h +++ b/src/Interpreters/AggregationCommon.h @@ -253,15 +253,11 @@ static inline T ALWAYS_INLINE packFixed( static inline UInt128 ALWAYS_INLINE hash128( /// NOLINT size_t i, size_t keys_size, const ColumnRawPtrs & key_columns) { - UInt128 key; SipHash hash; - for (size_t j = 0; j < keys_size; ++j) key_columns[j]->updateHashWithValue(i, hash); - hash.get128(key); - - return key; + return hash.get128(); } /** Serialize keys into a continuous chunk of memory. diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 00347663fbd..d174c626dd6 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -105,9 +105,7 @@ UInt128 AsynchronousInsertQueue::InsertQuery::calculateHash() const applyVisitor(FieldVisitorHash(siphash), setting.getValue()); } - UInt128 res; - siphash.get128(res); - return res; + return siphash.get128(); } bool AsynchronousInsertQueue::InsertQuery::operator==(const InsertQuery & other) const diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 0138372ce89..7c85b63ceff 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -118,9 +118,7 @@ IAST::Hash IAST::getTreeHash() const { SipHash hash_state; updateTreeHash(hash_state); - IAST::Hash res; - hash_state.get128(res); - return res; + return getSipHash128AsLoHi(hash_state); } diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 5d438d47de6..2d5264fa4e3 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -369,8 +369,7 @@ size_t ConstantExpressionTemplate::TemplateStructure::getTemplateHash(const ASTP /// Allows distinguish expression in the last column in Values format hash_state.update(salt); - IAST::Hash res128; - hash_state.get128(res128); + const auto res128 = getSipHash128AsLoHi(hash_state); size_t res = 0; boost::hash_combine(res, res128.first); boost::hash_combine(res, res128.second); diff --git a/src/Processors/Transforms/LimitByTransform.cpp b/src/Processors/Transforms/LimitByTransform.cpp index cb2804007bd..5e6d7dc816a 100644 --- a/src/Processors/Transforms/LimitByTransform.cpp +++ b/src/Processors/Transforms/LimitByTransform.cpp @@ -33,14 +33,11 @@ void LimitByTransform::transform(Chunk & chunk) for (UInt64 row = 0; row < num_rows; ++row) { - UInt128 key{}; SipHash hash; - for (auto position : key_positions) columns[position]->updateHashWithValue(row, hash); - hash.get128(key); - + const auto key = hash.get128(); auto count = keys_counts[key]++; if (count >= group_offset && (group_length > std::numeric_limits::max() - group_offset || count < group_length + group_offset)) diff --git a/src/Storages/LiveView/LiveViewSink.h b/src/Storages/LiveView/LiveViewSink.h index e163400f2af..792133ced64 100644 --- a/src/Storages/LiveView/LiveViewSink.h +++ b/src/Storages/LiveView/LiveViewSink.h @@ -32,11 +32,8 @@ public: void onFinish() override { - UInt128 key; - String key_str; - - new_hash->get128(key); - key_str = getHexUIntLowercase(key); + const auto key = new_hash->get128(); + const auto key_str = getHexUIntLowercase(key); std::lock_guard lock(storage.mutex); diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 173bb128c4a..5719529533e 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -681,7 +681,6 @@ QueryPipelineBuilder StorageLiveView::completeQuery(Pipes pipes) bool StorageLiveView::getNewBlocks(const std::lock_guard & lock) { SipHash hash; - UInt128 key; BlocksPtr new_blocks = std::make_shared(); BlocksMetadataPtr new_blocks_metadata = std::make_shared(); @@ -713,7 +712,7 @@ bool StorageLiveView::getNewBlocks(const std::lock_guard & lock) new_blocks->push_back(block); } - hash.get128(key); + const auto key = hash.get128(); /// Update blocks only if hash keys do not match /// NOTE: hash could be different for the same result diff --git a/src/Storages/MarkCache.h b/src/Storages/MarkCache.h index 0f4af57fc8e..495cebb080f 100644 --- a/src/Storages/MarkCache.h +++ b/src/Storages/MarkCache.h @@ -50,13 +50,9 @@ public: /// Calculate key from path to file and offset. static UInt128 hash(const String & path_to_file) { - UInt128 key; - SipHash hash; hash.update(path_to_file.data(), path_to_file.size() + 1); - hash.get128(key); - - return key; + return hash.get128(); } template diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 7050a98a4bc..91069d30c03 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -2052,14 +2052,8 @@ String IMergeTreeDataPart::getZeroLevelPartBlockID(std::string_view token) const hash.update(token.data(), token.size()); } - union - { - char bytes[16]; - UInt64 words[2]; - } hash_value; - hash.get128(hash_value.bytes); - - return info.partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1]); + const auto hash_value = hash.get128(); + return info.partition_id + "_" + toString(hash_value.items[0]) + "_" + toString(hash_value.items[1]); } IMergeTreeDataPart::uint128 IMergeTreeDataPart::getActualChecksumByFile(const String & file_name) const diff --git a/src/Storages/MergeTree/MarkRange.cpp b/src/Storages/MergeTree/MarkRange.cpp index c6c197919f4..bd8546f04cc 100644 --- a/src/Storages/MergeTree/MarkRange.cpp +++ b/src/Storages/MergeTree/MarkRange.cpp @@ -83,12 +83,12 @@ size_t MarkRanges::getNumberOfMarks() const void MarkRanges::serialize(WriteBuffer & out) const { - writeIntBinary(this->size(), out); + writeBinaryLittleEndian(this->size(), out); for (const auto & [begin, end] : *this) { - writeIntBinary(begin, out); - writeIntBinary(end, out); + writeBinaryLittleEndian(begin, out); + writeBinaryLittleEndian(end, out); } } @@ -100,13 +100,13 @@ String MarkRanges::describe() const void MarkRanges::deserialize(ReadBuffer & in) { size_t size = 0; - readIntBinary(size, in); + readBinaryLittleEndian(size, in); this->resize(size); for (size_t i = 0; i < size; ++i) { - readIntBinary((*this)[i].begin, in); - readIntBinary((*this)[i].end, in); + readBinaryLittleEndian((*this)[i].begin, in); + readBinaryLittleEndian((*this)[i].end, in); } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index 6628cd68eaf..d09cae7870f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -323,9 +323,7 @@ MergeTreeDataPartChecksums::Checksum::uint128 MergeTreeDataPartChecksums::getTot hash_of_all_files.update(checksum.file_hash); } - MergeTreeDataPartChecksums::Checksum::uint128 ret; - hash_of_all_files.get128(reinterpret_cast(&ret)); - return ret; + return getSipHash128AsLoHi(hash_of_all_files); } void MinimalisticDataPartChecksums::serialize(WriteBuffer & to) const @@ -415,14 +413,9 @@ void MinimalisticDataPartChecksums::computeTotalChecksums(const MergeTreeDataPar } } - auto get_hash = [] (SipHash & hash, uint128 & data) - { - hash.get128(data); - }; - - get_hash(hash_of_all_files_state, hash_of_all_files); - get_hash(hash_of_uncompressed_files_state, hash_of_uncompressed_files); - get_hash(uncompressed_hash_of_compressed_files_state, uncompressed_hash_of_compressed_files); + hash_of_all_files = getSipHash128AsLoHi(hash_of_all_files_state); + hash_of_uncompressed_files = getSipHash128AsLoHi(hash_of_uncompressed_files_state); + uncompressed_hash_of_compressed_files = getSipHash128AsLoHi(uncompressed_hash_of_compressed_files_state); } String MinimalisticDataPartChecksums::getSerializedString(const MergeTreeDataPartChecksums & full_checksums, bool minimalistic) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 9c47608e364..70e5ee05d2e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -115,7 +115,7 @@ void MergeTreeDataPartCompact::loadIndexGranularityImpl( { marks_reader->ignore(columns_count * sizeof(MarkInCompressedFile)); size_t granularity; - readIntBinary(granularity, *marks_reader); + readBinaryLittleEndian(granularity, *marks_reader); index_granularity_.appendMark(granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 468747a6c36..a8d38a1bff8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -167,7 +167,7 @@ IMergeTreeDataPart::Checksum MergeTreeDataPartInMemory::calculateBlockChecksum() column.column->updateHashFast(hash); checksum.uncompressed_size = block.bytes(); - hash.get128(checksum.uncompressed_hash); + checksum.uncompressed_hash = getSipHash128AsLoHi(hash); return checksum; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 2d886e2058b..846442fd3e0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -130,13 +130,13 @@ void MergeTreeDataPartWide::loadIndexGranularityImpl( MarkInCompressedFile mark; size_t granularity; - readBinary(mark.offset_in_compressed_file, *marks_reader); - readBinary(mark.offset_in_decompressed_block, *marks_reader); + readBinaryLittleEndian(mark.offset_in_compressed_file, *marks_reader); + readBinaryLittleEndian(mark.offset_in_decompressed_block, *marks_reader); ++marks_count; if (index_granularity_info_.mark_type.adaptive) { - readIntBinary(granularity, *marks_reader); + readBinaryLittleEndian(granularity, *marks_reader); index_granularity_.appendMark(granularity); } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 5e1da21da5b..9cd297c4e80 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -228,8 +228,8 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G }; - writeIntBinary(plain_hashing.count(), marks_out); - writeIntBinary(static_cast(0), marks_out); + writeBinaryLittleEndian(plain_hashing.count(), marks_out); + writeBinaryLittleEndian(static_cast(0), marks_out); writeColumnSingleGranule( block.getByName(name_and_type->name), data_part->getSerialization(name_and_type->name), @@ -239,7 +239,7 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G prev_stream->hashing_buf.next(); } - writeIntBinary(granule.rows_to_write, marks_out); + writeBinaryLittleEndian(granule.rows_to_write, marks_out); } } @@ -270,10 +270,10 @@ void MergeTreeDataPartWriterCompact::fillDataChecksums(IMergeTreeDataPart::Check { for (size_t i = 0; i < columns_list.size(); ++i) { - writeIntBinary(plain_hashing.count(), marks_out); - writeIntBinary(static_cast(0), marks_out); + writeBinaryLittleEndian(plain_hashing.count(), marks_out); + writeBinaryLittleEndian(static_cast(0), marks_out); } - writeIntBinary(static_cast(0), marks_out); + writeBinaryLittleEndian(static_cast(0), marks_out); } for (const auto & [_, stream] : streams_by_codec) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index f57ffa5ee14..79b72d4ae39 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -313,13 +313,13 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block if (stream.compressed_hashing.offset() >= settings.min_compress_block_size) stream.compressed_hashing.next(); - writeIntBinary(stream.plain_hashing.count(), marks_out); - writeIntBinary(stream.compressed_hashing.offset(), marks_out); + writeBinaryLittleEndian(stream.plain_hashing.count(), marks_out); + writeBinaryLittleEndian(stream.compressed_hashing.offset(), marks_out); /// Actually this numbers is redundant, but we have to store them /// to be compatible with the normal .mrk2 file format if (settings.can_use_adaptive_granularity) - writeIntBinary(1UL, marks_out); + writeBinaryLittleEndian(1UL, marks_out); } size_t pos = granule.start_row; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index f9fe6f2c8ab..bcf340e0f55 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -275,10 +275,10 @@ void MergeTreeDataPartWriterWide::flushMarkToFile(const StreamNameAndMark & stre Stream & stream = *column_streams[stream_with_mark.stream_name]; WriteBuffer & marks_out = stream.compress_marks ? stream.marks_compressed_hashing : stream.marks_hashing; - writeIntBinary(stream_with_mark.mark.offset_in_compressed_file, marks_out); - writeIntBinary(stream_with_mark.mark.offset_in_decompressed_block, marks_out); + writeBinaryLittleEndian(stream_with_mark.mark.offset_in_compressed_file, marks_out); + writeBinaryLittleEndian(stream_with_mark.mark.offset_in_decompressed_block, marks_out); if (settings.can_use_adaptive_granularity) - writeIntBinary(rows_in_mark, marks_out); + writeBinaryLittleEndian(rows_in_mark, marks_out); } StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( @@ -452,10 +452,10 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai "Incorrect number of marks in memory {}, on disk (at least) {}", index_granularity.getMarksCount(), mark_num + 1); - DB::readBinary(offset_in_compressed_file, *mrk_in); - DB::readBinary(offset_in_decompressed_block, *mrk_in); + readBinaryLittleEndian(offset_in_compressed_file, *mrk_in); + readBinaryLittleEndian(offset_in_decompressed_block, *mrk_in); if (settings.can_use_adaptive_granularity) - DB::readBinary(index_granularity_rows, *mrk_in); + readBinaryLittleEndian(index_granularity_rows, *mrk_in); else index_granularity_rows = data_part->index_granularity_info.fixed_index_granularity; diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index 5c722eec380..c9b22c8a03e 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -160,7 +160,7 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl() size_t granularity; reader->readStrict( reinterpret_cast(plain_marks.data() + i * columns_in_mark), columns_in_mark * sizeof(MarkInCompressedFile)); - readIntBinary(granularity, *reader); + readBinaryLittleEndian(granularity, *reader); } if (!reader->eof()) @@ -170,6 +170,16 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl() mrk_path, marks_count, expected_uncompressed_size); } +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + std::ranges::for_each( + plain_marks, + [](auto & plain_mark) + { + plain_mark.offset_in_compressed_file = std::byteswap(plain_mark.offset_in_compressed_file); + plain_mark.offset_in_decompressed_block = std::byteswap(plain_mark.offset_in_decompressed_block); + }); +#endif + auto res = std::make_shared(plain_marks); ProfileEvents::increment(ProfileEvents::LoadedMarksCount, marks_count * columns_in_mark); diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index bce33438229..63efd137b0b 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -265,12 +265,12 @@ String MergeTreePartition::getID(const Block & partition_key_sample) const for (const Field & field : value) applyVisitor(hashing_visitor, field); - char hash_data[16]; - hash.get128(hash_data); - result.resize(32); - for (size_t i = 0; i < 16; ++i) + const auto hash_data = getSipHash128AsArray(hash); + const auto hash_size = hash_data.size(); + result.resize(hash_size * 2); + for (size_t i = 0; i < hash_size; ++i) #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - writeHexByteLowercase(hash_data[16 - 1 - i], &result[2 * i]); + writeHexByteLowercase(hash_data[hash_size - 1 - i], &result[2 * i]); #else writeHexByteLowercase(hash_data[i], &result[2 * i]); #endif diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp index 88f4a3ec66f..24d907dbad6 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartHeader.cpp @@ -12,9 +12,7 @@ static std::array getSipHash(const String & str) { SipHash hash; hash.update(str.data(), str.size()); - std::array result; - hash.get128(result.data()); - return result; + return getSipHash128AsArray(hash); } ReplicatedMergeTreePartHeader ReplicatedMergeTreePartHeader::fromColumnsAndChecksumsZNodes( diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 393d418c083..1d0acd782b3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -254,14 +254,9 @@ namespace for (const auto & col : cols) col->updateHashWithValue(j, hash); } - union - { - char bytes[16]; - UInt64 words[2]; - } hash_value; - hash.get128(hash_value.bytes); - block_id_vec.push_back(partition_id + "_" + DB::toString(hash_value.words[0]) + "_" + DB::toString(hash_value.words[1])); + const auto hash_value = hash.get128(); + block_id_vec.push_back(partition_id + "_" + DB::toString(hash_value.items[0]) + "_" + DB::toString(hash_value.items[1])); } else block_id_vec.push_back(partition_id + "_" + std::string(token)); From 967be3e13cb742245c9124fea2c89c3acdeb75da Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 22 Jun 2023 08:41:30 -0700 Subject: [PATCH 028/777] Correct a line commented out by mistake --- programs/obfuscator/Obfuscator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 43b31843afe..31288b4aa01 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -398,7 +398,7 @@ static void transformUUID(const UUID & src_uuid, UUID & dst_uuid, UInt64 seed) hash.update(reinterpret_cast(&src), sizeof(UUID)); /// Saving version and variant from an old UUID - // hash.get128Impl(reinterpret_cast(&dst)); + dst = hash.get128(); dst.items[1] = (dst.items[1] & 0x1fffffffffffffffull) | (src.items[1] & 0xe000000000000000ull); dst.items[0] = (dst.items[0] & 0xffffffffffff0fffull) | (src.items[0] & 0x000000000000f000ull); From 6b96a3943de8799cd7531e66f575276099012757 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 23 Jun 2023 15:22:21 +0000 Subject: [PATCH 029/777] Update further uses of SipHash --- src/Common/SipHash.h | 16 +--------------- src/Core/SortDescription.cpp | 3 +-- src/Interpreters/Aggregator.cpp | 3 +-- src/Interpreters/JIT/CompileDAG.cpp | 4 +--- 4 files changed, 4 insertions(+), 22 deletions(-) diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index e9d45c7201f..c77fe7f2bcf 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -88,20 +88,6 @@ private: SIPROUND; } - /// @brief Retrieves the result in some form with the endianness of the platform taken into account. - /// @warning This can only be done once! - void get128Impl(char * out) - { - finalize(); -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - unalignedStore(out + 8, v0 ^ v1); - unalignedStore(out, v2 ^ v3); -#else - unalignedStore(out, v0 ^ v1); - unalignedStore(out + 8, v2 ^ v3); -#endif - } - public: /// Arguments - seed. SipHash(UInt64 key0 = 0, UInt64 key1 = 0, bool is_reference_128_ = false) /// NOLINT @@ -305,7 +291,7 @@ inline UInt64 sipHash64(const char * data, const size_t size) } template -UInt64 sipHash64(const T & x) +inline UInt64 sipHash64(const T & x) { SipHash hash; hash.update(x); diff --git a/src/Core/SortDescription.cpp b/src/Core/SortDescription.cpp index ae6aedf202d..9ba7df8ef24 100644 --- a/src/Core/SortDescription.cpp +++ b/src/Core/SortDescription.cpp @@ -133,8 +133,7 @@ void compileSortDescriptionIfNeeded(SortDescription & description, const DataTyp SipHash sort_description_dump_hash; sort_description_dump_hash.update(description_dump); - UInt128 sort_description_hash_key; - sort_description_dump_hash.get128(sort_description_hash_key); + const auto sort_description_hash_key = sort_description_dump_hash.get128(); { std::lock_guard lock(mutex); diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index c7d4b87694b..b02bfc5835c 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -694,8 +694,7 @@ void Aggregator::compileAggregateFunctionsIfNeeded() SipHash aggregate_functions_description_hash; aggregate_functions_description_hash.update(functions_description); - UInt128 aggregate_functions_description_hash_key; - aggregate_functions_description_hash.get128(aggregate_functions_description_hash_key); + const auto aggregate_functions_description_hash_key = aggregate_functions_description_hash.get128(); { std::lock_guard lock(mutex); diff --git a/src/Interpreters/JIT/CompileDAG.cpp b/src/Interpreters/JIT/CompileDAG.cpp index 6da17fb4c67..9c56341eae0 100644 --- a/src/Interpreters/JIT/CompileDAG.cpp +++ b/src/Interpreters/JIT/CompileDAG.cpp @@ -160,9 +160,7 @@ UInt128 CompileDAG::hash() const } } - UInt128 result; - hash.get128(result); - return result; + return hash.get128(); } } From ba4072f049bbf22a9b40f088527fe461ad36d910 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 5 Jul 2023 17:35:44 -0700 Subject: [PATCH 030/777] Adapt changes around SipHash --- contrib/cityhash102/include/city.h | 4 ++-- src/Analyzer/HashUtils.h | 4 ++-- src/Analyzer/IQueryTreeNode.h | 2 +- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- src/Common/SipHash.h | 10 ++++++---- src/Common/TransformEndianness.hpp | 9 +++++++++ src/IO/WriteHelpers.h | 9 +++++++++ src/Interpreters/ComparisonGraph.h | 2 +- src/Interpreters/Context.cpp | 8 ++++---- src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- src/Interpreters/MonotonicityCheckVisitor.h | 4 ++-- src/Interpreters/PreparedSets.cpp | 2 +- src/Interpreters/PreparedSets.h | 4 ++-- src/Interpreters/TreeOptimizer.cpp | 8 ++++---- src/Parsers/ASTSubquery.cpp | 6 ++---- src/Parsers/IAST.h | 2 +- src/Planner/PlannerActionsVisitor.cpp | 2 +- src/Planner/PlannerContext.cpp | 4 ++-- .../Formats/Impl/ConstantExpressionTemplate.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- src/Storages/System/StorageSystemQueryCache.cpp | 2 +- src/Storages/buildQueryTreeForShard.cpp | 4 ++-- 23 files changed, 59 insertions(+), 41 deletions(-) diff --git a/contrib/cityhash102/include/city.h b/contrib/cityhash102/include/city.h index 87363d16444..c98eb7e3585 100644 --- a/contrib/cityhash102/include/city.h +++ b/contrib/cityhash102/include/city.h @@ -73,8 +73,8 @@ struct uint128 uint128() = default; uint128(uint64 low64_, uint64 high64_) : low64(low64_), high64(high64_) {} - friend bool operator ==(const uint128 & x, const uint128 & y) { return (x.low64 == y.low64) && (x.high64 == y.high64); } - friend bool operator !=(const uint128 & x, const uint128 & y) { return !(x == y); } + + friend auto operator<=>(const uint128 &, const uint128 &) = default; }; inline uint64 Uint128Low64(const uint128 & x) { return x.low64; } diff --git a/src/Analyzer/HashUtils.h b/src/Analyzer/HashUtils.h index 2203e7d5203..3727ea1ea14 100644 --- a/src/Analyzer/HashUtils.h +++ b/src/Analyzer/HashUtils.h @@ -20,7 +20,7 @@ struct QueryTreeNodeWithHash {} QueryTreeNodePtrType node = nullptr; - std::pair hash; + CityHash_v1_0_2::uint128 hash; }; template @@ -55,6 +55,6 @@ struct std::hash> { size_t operator()(const DB::QueryTreeNodeWithHash & node_with_hash) const { - return node_with_hash.hash.first; + return node_with_hash.hash.low64; } }; diff --git a/src/Analyzer/IQueryTreeNode.h b/src/Analyzer/IQueryTreeNode.h index 763963b734a..3f6816696b4 100644 --- a/src/Analyzer/IQueryTreeNode.h +++ b/src/Analyzer/IQueryTreeNode.h @@ -106,7 +106,7 @@ public: */ bool isEqual(const IQueryTreeNode & rhs, CompareOptions compare_options = { .compare_aliases = true }) const; - using Hash = std::pair; + using Hash = CityHash_v1_0_2::uint128; using HashState = SipHash; /** Get tree hash identifying current tree diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index abf02547ccd..2d997bb7eff 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2032,7 +2032,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden auto & nearest_query_scope_query_node = nearest_query_scope->scope_node->as(); auto & mutable_context = nearest_query_scope_query_node.getMutableContext(); - auto scalar_query_hash_string = std::to_string(node_with_hash.hash.first) + '_' + std::to_string(node_with_hash.hash.second); + const auto scalar_query_hash_string = DB::toString(node_with_hash.hash); if (mutable_context->hasQueryContext()) mutable_context->getQueryContext()->addScalar(scalar_query_hash_string, scalar_block); diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index c77fe7f2bcf..00fe07648a5 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -24,6 +24,8 @@ #include #include +#include + namespace DB::ErrorCodes { extern const int LOGICAL_ERROR; @@ -247,11 +249,11 @@ inline std::array getSipHash128AsArray(SipHash & sip_hash) return arr; } -inline std::pair getSipHash128AsLoHi(SipHash & sip_hash) +inline CityHash_v1_0_2::uint128 getSipHash128AsLoHi(SipHash & sip_hash) { - std::pair lo_hi; - sip_hash.get128(lo_hi.first, lo_hi.second); - return lo_hi; + CityHash_v1_0_2::uint128 result; + sip_hash.get128(result.low64, result.high64); + return result; } inline UInt128 sipHash128Keyed(UInt64 key0, UInt64 key1, const char * data, const size_t size) diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index ccfec78ab64..c7bea2ef15a 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -4,6 +4,8 @@ #include #include +#include + #include namespace DB @@ -66,4 +68,11 @@ inline void transformEndianness(StrongTypedef & x) { transformEndianness(x.toUnderType()); } + +template +inline void transformEndianness(CityHash_v1_0_2::uint128 & x) +{ + transformEndianness(x.low64); + transformEndianness(x.high64); +} } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index aa4c9b17e48..a9d1c108061 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1161,6 +1161,15 @@ inline String toString(const T & x) return buf.str(); } +inline String toString(const CityHash_v1_0_2::uint128 & hash) +{ + WriteBufferFromOwnString buf; + writeText(hash.low64, buf); + writeChar('_', buf); + writeText(hash.high64, buf); + return buf.str(); +} + template inline String toStringWithFinalSeparator(const std::vector & x, const String & final_sep) { diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 70543227b58..4fd90dad371 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -118,7 +118,7 @@ private: { size_t operator() (const IAST::Hash & hash) const { - return hash.first; + return hash.low64; } }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a9055bbb1b9..f179e38b0f8 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1505,7 +1505,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const } } auto hash = table_expression->getTreeHash(); - String key = toString(hash.first) + '_' + toString(hash.second); + auto key = toString(hash); StoragePtr & res = table_function_results[key]; if (!res) { @@ -1656,7 +1656,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const auto new_hash = table_expression->getTreeHash(); if (hash != new_hash) { - key = toString(new_hash.first) + '_' + toString(new_hash.second); + key = toString(new_hash); table_function_results[key] = res; } } @@ -1665,8 +1665,8 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const TableFunctionPtr & table_function_ptr) { - auto hash = table_expression->getTreeHash(); - String key = toString(hash.first) + '_' + toString(hash.second); + const auto hash = table_expression->getTreeHash(); + const auto key = toString(hash); StoragePtr & res = table_function_results[key]; if (!res) diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 3d821b60e81..ded99df9f74 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -98,7 +98,7 @@ static auto getQueryInterpreter(const ASTSubquery & subquery, ExecuteScalarSubqu void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr & ast, Data & data) { auto hash = subquery.getTreeHash(); - auto scalar_query_hash_str = toString(hash.first) + "_" + toString(hash.second); + const auto scalar_query_hash_str = toString(hash); std::unique_ptr interpreter = nullptr; bool hit = false; diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 5b633fee9b6..08378c3158b 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -115,7 +115,7 @@ public: if (alias.empty()) { auto hash = subquery_or_table_name->getTreeHash(); - external_table_name = fmt::format("_data_{}_{}", hash.first, hash.second); + external_table_name = fmt::format("_data_{}", toString(hash)); } else external_table_name = alias; diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index c95f5209760..c1838fa105c 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -39,8 +39,8 @@ public: bool canOptimize(const ASTFunction & ast_function) const { /// if GROUP BY contains the same function ORDER BY shouldn't be optimized - auto hash = ast_function.getTreeHash(); - String key = toString(hash.first) + '_' + toString(hash.second); + const auto hash = ast_function.getTreeHash(); + const auto key = toString(hash); if (group_by_function_hashes.count(key)) return false; diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 67822ecf440..c2c01b912f9 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -207,7 +207,7 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) String PreparedSets::toString(const PreparedSets::Hash & key, const DataTypes & types) { WriteBufferFromOwnString buf; - buf << "__set_" << key.first << "_" << key.second; + buf << "__set_" << DB::toString(key); if (!types.empty()) { buf << "("; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index cb240f5260a..b953b8470e1 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -127,10 +127,10 @@ class PreparedSets { public: - using Hash = std::pair; + using Hash = CityHash_v1_0_2::uint128; struct Hashing { - UInt64 operator()(const Hash & key) const { return key.first ^ key.second; } + UInt64 operator()(const Hash & key) const { return key.low64 ^ key.high64; } }; using SetsFromTuple = std::unordered_map>, Hashing>; diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index fd4d2c9d846..1a7b5cd73ec 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -451,8 +451,8 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context { for (auto & elem : set->children) { - auto hash = elem->getTreeHash(); - String key = toString(hash.first) + '_' + toString(hash.second); + const auto hash = elem->getTreeHash(); + const auto key = toString(hash); group_by_hashes.insert(key); } } @@ -461,8 +461,8 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context { for (auto & elem : group_by->children) { - auto hash = elem->getTreeHash(); - String key = toString(hash.first) + '_' + toString(hash.second); + const auto hash = elem->getTreeHash(); + const auto key = toString(hash); group_by_hashes.insert(key); } } diff --git a/src/Parsers/ASTSubquery.cpp b/src/Parsers/ASTSubquery.cpp index a3408f12330..92adad666ed 100644 --- a/src/Parsers/ASTSubquery.cpp +++ b/src/Parsers/ASTSubquery.cpp @@ -19,11 +19,9 @@ void ASTSubquery::appendColumnNameImpl(WriteBuffer & ostr) const } else { - Hash hash = getTreeHash(); + const auto hash = getTreeHash(); writeCString("__subquery_", ostr); - writeText(hash.first, ostr); - ostr.write('_'); - writeText(hash.second, ostr); + writeString(toString(hash), ostr); } } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 7a8ab36518d..d0464c7b950 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -78,7 +78,7 @@ public: /** Get hash code, identifying this element and its subtree. */ - using Hash = std::pair; + using Hash = CityHash_v1_0_2::uint128; Hash getTreeHash() const; void updateTreeHash(SipHash & hash_state) const; virtual void updateTreeHashImpl(SipHash & hash_state) const; diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 7575828e64d..b39716c7712 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -165,7 +165,7 @@ public: case QueryTreeNodeType::LAMBDA: { auto lambda_hash = node->getTreeHash(); - result = "__lambda_" + toString(lambda_hash.first) + '_' + toString(lambda_hash.second); + result = "__lambda_" + toString(lambda_hash); break; } default: diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 3c75d4fbea8..f0e6e44ad8c 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -114,8 +114,8 @@ const ColumnIdentifier * PlannerContext::getColumnNodeIdentifierOrNull(const Que PlannerContext::SetKey PlannerContext::createSetKey(const QueryTreeNodePtr & set_source_node) { - auto set_source_hash = set_source_node->getTreeHash(); - return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); + const auto set_source_hash = set_source_node->getTreeHash(); + return "__set_" + toString(set_source_hash); } } diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 2d5264fa4e3..c388c370848 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -371,8 +371,8 @@ size_t ConstantExpressionTemplate::TemplateStructure::getTemplateHash(const ASTP const auto res128 = getSipHash128AsLoHi(hash_state); size_t res = 0; - boost::hash_combine(res, res128.first); - boost::hash_combine(res, res128.second); + boost::hash_combine(res, res128.low64); + boost::hash_combine(res, res128.high64); return res; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9c5e45aa488..fafa35d34c1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -187,8 +187,8 @@ static void checkSuspiciousIndices(const ASTFunction * index_function) std::unordered_set unique_index_expression_hashes; for (const auto & child : index_function->arguments->children) { - IAST::Hash hash = child->getTreeHash(); - UInt64 first_half_of_hash = hash.first; + const IAST::Hash hash = child->getTreeHash(); + const auto & first_half_of_hash = hash.low64; if (!unique_index_expression_hashes.emplace(first_half_of_hash).second) throw Exception(ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 117fb4e8a5c..b6ab2c54407 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -50,7 +50,7 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr res_columns[3]->insert(key.is_shared); res_columns[4]->insert(key.is_compressed); res_columns[5]->insert(std::chrono::system_clock::to_time_t(key.expires_at)); - res_columns[6]->insert(key.ast->getTreeHash().first); + res_columns[6]->insert(key.ast->getTreeHash().low64); } } diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index 1ee7d747fcc..18ec8e48c3e 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -232,8 +232,8 @@ TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, ContextMutablePtr & mutable_context, size_t subquery_depth) { - auto subquery_hash = subquery_node->getTreeHash(); - String temporary_table_name = fmt::format("_data_{}_{}", subquery_hash.first, subquery_hash.second); + const auto subquery_hash = subquery_node->getTreeHash(); + const auto temporary_table_name = fmt::format("_data_{}", toString(subquery_hash)); const auto & external_tables = mutable_context->getExternalTables(); auto external_table_it = external_tables.find(temporary_table_name); From b8a959da8927b05395a22052a969a803b1a0d1f4 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 6 Jul 2023 05:41:34 -0700 Subject: [PATCH 031/777] Correct a finding from clang-tidy --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 2d997bb7eff..98e5d326cbf 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2032,7 +2032,7 @@ void QueryAnalyzer::evaluateScalarSubqueryIfNeeded(QueryTreeNodePtr & node, Iden auto & nearest_query_scope_query_node = nearest_query_scope->scope_node->as(); auto & mutable_context = nearest_query_scope_query_node.getMutableContext(); - const auto scalar_query_hash_string = DB::toString(node_with_hash.hash); + auto scalar_query_hash_string = DB::toString(node_with_hash.hash); if (mutable_context->hasQueryContext()) mutable_context->getQueryContext()->addScalar(scalar_query_hash_string, scalar_block); From 1208e59b8e939adedf7ff7c12a91bb04b05471d5 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 14 Jul 2023 14:28:03 +0000 Subject: [PATCH 032/777] Inline hash geetter mthods for potential performance gains --- src/Common/SipHash.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index 00fe07648a5..22c2c61abdc 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -182,7 +182,7 @@ public: ALWAYS_INLINE void update(const std::string_view x) { update(x.data(), x.size()); } ALWAYS_INLINE void update(const char * s) { update(std::string_view(s)); } - UInt64 get64() + ALWAYS_INLINE UInt64 get64() { finalize(); return v0 ^ v1 ^ v2 ^ v3; @@ -197,7 +197,7 @@ public: hi = v2 ^ v3; } - UInt128 get128() + ALWAYS_INLINE UInt128 get128() { UInt128 res; #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ From 29094a22cf4cc787cc50f5d494b29dcf511cc765 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 20:00:00 +0200 Subject: [PATCH 033/777] Fix Docker --- docker/images.json | 1 + tests/ci/tests/docker_images.json | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/images.json b/docker/images.json index 663fa21a6c9..8e19401ba72 100644 --- a/docker/images.json +++ b/docker/images.json @@ -125,6 +125,7 @@ "docker/test/keeper-jepsen", "docker/test/server-jepsen", "docker/test/sqllogic", + "docker/test/sqltest", "docker/test/stateless" ] }, diff --git a/tests/ci/tests/docker_images.json b/tests/ci/tests/docker_images.json index 0637058e184..70db8760561 100644 --- a/tests/ci/tests/docker_images.json +++ b/tests/ci/tests/docker_images.json @@ -119,7 +119,8 @@ "docker/test/stateless", "docker/test/integration/base", "docker/test/fuzzer", - "docker/test/keeper-jepsen" + "docker/test/keeper-jepsen", + "docker/test/sqltest" ] }, "docker/test/integration/kerberized_hadoop": { From a3d0335f536668af3d6dd325e639d67cce624b46 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 19 Jul 2023 11:20:42 -0700 Subject: [PATCH 034/777] Fix a build error --- src/Parsers/ASTSetQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ASTSetQuery.cpp b/src/Parsers/ASTSetQuery.cpp index 76ad812e713..1b7b76fe231 100644 --- a/src/Parsers/ASTSetQuery.cpp +++ b/src/Parsers/ASTSetQuery.cpp @@ -69,9 +69,9 @@ void ASTSetQuery::appendColumnName(WriteBuffer & ostr) const Hash hash = getTreeHash(); writeCString("__settings_", ostr); - writeText(hash.first, ostr); + writeText(hash.low64, ostr); ostr.write('_'); - writeText(hash.second, ostr); + writeText(hash.high64, ostr); } } From f458108c44f2d9f0c92725f05d3959658c97ee6f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Jul 2023 22:15:23 +0200 Subject: [PATCH 035/777] Fix style --- docker/test/sqllogic/run.sh | 2 ++ tests/ci/sqltest.py | 1 - 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/sqllogic/run.sh b/docker/test/sqllogic/run.sh index 444252837a3..3b900c097e2 100755 --- a/docker/test/sqllogic/run.sh +++ b/docker/test/sqllogic/run.sh @@ -1,4 +1,6 @@ #!/bin/bash +# shellcheck disable=SC2015 + set -exu trap "exit" INT TERM diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index 5c20cc4849b..b752d4e4aee 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -11,7 +11,6 @@ from build_download_helper import get_build_name_for_check, read_build_urls from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse from commit_status_helper import ( RerunHelper, - format_description, get_commit, post_commit_status, ) From fa44f84377bc097cbe0c25852603de9be536f75e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Jul 2023 04:50:12 +0200 Subject: [PATCH 036/777] Fix style --- docker/test/sqllogic/run.sh | 1 - docker/test/sqltest/run.sh | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/sqllogic/run.sh b/docker/test/sqllogic/run.sh index 3b900c097e2..5be44fc148c 100755 --- a/docker/test/sqllogic/run.sh +++ b/docker/test/sqllogic/run.sh @@ -1,5 +1,4 @@ #!/bin/bash -# shellcheck disable=SC2015 set -exu trap "exit" INT TERM diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index 03678971f60..42aeef9df15 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -1,4 +1,5 @@ #!/bin/bash +# shellcheck disable=SC2015 set -x set -e From 2a00e2aa6ae7c417fdd5aafd63225ef44460de6f Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 20 Jul 2023 22:20:36 -0400 Subject: [PATCH 037/777] Implement `MOVE` query for the access entities. --- src/Access/AccessControl.cpp | 4 +- src/Access/AccessControl.h | 2 +- src/Access/DiskAccessStorage.cpp | 4 +- src/Access/DiskAccessStorage.h | 2 +- src/Access/IAccessStorage.cpp | 46 +++++++-- src/Access/IAccessStorage.h | 21 +++- src/Access/MemoryAccessStorage.cpp | 4 +- src/Access/MemoryAccessStorage.h | 2 +- src/Access/MultipleAccessStorage.cpp | 40 +++++++- src/Access/MultipleAccessStorage.h | 9 +- src/Access/ReplicatedAccessStorage.cpp | 4 +- src/Access/ReplicatedAccessStorage.h | 2 +- .../Access/InterpreterCreateQuotaQuery.cpp | 36 +++++-- .../Access/InterpreterCreateRoleQuery.cpp | 8 +- .../InterpreterCreateRowPolicyQuery.cpp | 38 ++++++-- .../InterpreterCreateSettingsProfileQuery.cpp | 37 +++++-- .../Access/InterpreterCreateUserQuery.cpp | 33 +++++-- .../InterpreterMoveAccessEntityQuery.cpp | 96 +++++++++++++++++++ .../Access/InterpreterMoveAccessEntityQuery.h | 24 +++++ src/Interpreters/InterpreterFactory.cpp | 6 ++ src/Parsers/Access/ASTCreateQuotaQuery.cpp | 6 ++ src/Parsers/Access/ASTCreateQuotaQuery.h | 1 + src/Parsers/Access/ASTCreateRoleQuery.cpp | 2 +- .../Access/ASTCreateRowPolicyQuery.cpp | 5 + src/Parsers/Access/ASTCreateRowPolicyQuery.h | 1 + .../Access/ASTCreateSettingsProfileQuery.cpp | 6 ++ .../Access/ASTCreateSettingsProfileQuery.h | 1 + src/Parsers/Access/ASTCreateUserQuery.cpp | 5 + src/Parsers/Access/ASTCreateUserQuery.h | 1 + .../Access/ASTDropAccessEntityQuery.cpp | 4 +- .../Access/ASTMoveAccessEntityQuery.cpp | 64 +++++++++++++ src/Parsers/Access/ASTMoveAccessEntityQuery.h | 32 +++++++ src/Parsers/Access/ParserCreateQuotaQuery.cpp | 6 ++ src/Parsers/Access/ParserCreateRoleQuery.cpp | 6 +- .../Access/ParserCreateRowPolicyQuery.cpp | 6 ++ .../ParserCreateSettingsProfileQuery.cpp | 6 ++ src/Parsers/Access/ParserCreateUserQuery.cpp | 5 + .../Access/ParserMoveAccessEntityQuery.cpp | 93 ++++++++++++++++++ .../Access/ParserMoveAccessEntityQuery.h | 17 ++++ src/Parsers/Access/parseUserName.h | 3 +- src/Parsers/IAST.h | 1 + src/Parsers/ParserQuery.cpp | 3 + .../integration/test_grant_and_revoke/test.py | 4 +- .../configs/memory.xml | 5 + .../configs/roles.xml | 4 - .../test.py | 67 ++++++++++++- tests/integration/test_role/test.py | 2 +- .../integration/test_settings_profile/test.py | 14 +-- .../integration/test_user_directories/test.py | 46 ++++----- 49 files changed, 731 insertions(+), 103 deletions(-) create mode 100644 src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp create mode 100644 src/Interpreters/Access/InterpreterMoveAccessEntityQuery.h create mode 100644 src/Parsers/Access/ASTMoveAccessEntityQuery.cpp create mode 100644 src/Parsers/Access/ASTMoveAccessEntityQuery.h create mode 100644 src/Parsers/Access/ParserMoveAccessEntityQuery.cpp create mode 100644 src/Parsers/Access/ParserMoveAccessEntityQuery.h create mode 100644 tests/integration/test_multi_access_storage_role_management/configs/memory.xml diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 0c61a9e3e1b..22b00d5dde7 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -525,9 +525,9 @@ scope_guard AccessControl::subscribeForChanges(const std::vector & ids, co return changes_notifier->subscribeForChanges(ids, handler); } -std::optional AccessControl::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) +std::optional AccessControl::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) { - auto id = MultipleAccessStorage::insertImpl(entity, replace_if_exists, throw_if_exists); + auto id = MultipleAccessStorage::insertImpl(entity, replace_if_exists, throw_if_exists, set_id); if (id) changes_notifier->sendNotifications(); return id; diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 2a8293a49e7..85feaa70462 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -229,7 +229,7 @@ private: class CustomSettingsPrefixes; class PasswordComplexityRules; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 938881fafff..acbd7b12569 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -498,9 +498,9 @@ std::optional> DiskAccessStorage::readNameWi } -std::optional DiskAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +std::optional DiskAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) { - UUID id = generateRandomID(); + UUID id = set_id ? *set_id : generateRandomID(); if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk= */ true)) return id; diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index e3bd32c0292..3541c3228e3 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -39,7 +39,7 @@ private: std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 9468e8d220a..11bc97f6cc3 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -93,6 +93,17 @@ String IAccessStorage::readName(const UUID & id) const } +bool IAccessStorage::exists(const std::vector & ids) const +{ + for (const auto & id : ids) + { + if (!exists(id)) + return false; + } + + return true; +} + std::optional IAccessStorage::readName(const UUID & id, bool throw_if_not_exists) const { if (auto name_and_type = readNameWithType(id, throw_if_not_exists)) @@ -167,15 +178,26 @@ UUID IAccessStorage::insert(const AccessEntityPtr & entity) return *insert(entity, /* replace_if_exists = */ false, /* throw_if_exists = */ true); } - std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { - return insertImpl(entity, replace_if_exists, throw_if_exists); + return *insert(entity, replace_if_exists, throw_if_exists, /* set_id = */ std::nullopt); } +std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) +{ + return insertImpl(entity, replace_if_exists, throw_if_exists, set_id); +} std::vector IAccessStorage::insert(const std::vector & multiple_entities, bool replace_if_exists, bool throw_if_exists) { + return insert(multiple_entities, /* ids = */ {}, replace_if_exists, throw_if_exists); +} + +std::vector IAccessStorage::insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists, bool throw_if_exists) +{ + if (!ids.empty()) + assert(multiple_entities.size() == ids.size()); + if (multiple_entities.empty()) return {}; @@ -189,16 +211,24 @@ std::vector IAccessStorage::insert(const std::vector & mu std::vector successfully_inserted; try { - std::vector ids; - for (const auto & entity : multiple_entities) + std::vector new_ids; + for (size_t i = 0; i < multiple_entities.size(); ++i) { - if (auto id = insertImpl(entity, replace_if_exists, throw_if_exists)) + const auto & entity = multiple_entities[i]; + + std::optional id; + if (!ids.empty()) + id = ids[i]; + + auto new_id = insertImpl(entity, replace_if_exists, throw_if_exists, id); + + if (new_id) { successfully_inserted.push_back(entity); - ids.push_back(*id); + new_ids.push_back(*new_id); } } - return ids; + return new_ids; } catch (Exception & e) { @@ -244,7 +274,7 @@ std::vector IAccessStorage::insertOrReplace(const std::vector IAccessStorage::insertImpl(const AccessEntityPtr & entity, bool, bool) +std::optional IAccessStorage::insertImpl(const AccessEntityPtr & entity, bool, bool, std::optional) { if (isReadOnly()) throwReadonlyCannotInsert(entity->getType(), entity->getName()); diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index aa3947201e7..894220e95ad 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -92,6 +92,7 @@ public: /// Returns whether there is an entity with such identifier in the storage. virtual bool exists(const UUID & id) const = 0; + bool exists(const std::vector & ids) const; /// Reads an entity. Throws an exception if not found. template @@ -100,6 +101,9 @@ public: template std::shared_ptr read(const String & name, bool throw_if_not_exists = true) const; + template + std::vector read(const std::vector & ids, bool throw_if_not_exists = true) const; + /// Reads an entity. Returns nullptr if not found. template std::shared_ptr tryRead(const UUID & id) const; @@ -128,7 +132,9 @@ public: /// Throws an exception if the specified name already exists. UUID insert(const AccessEntityPtr & entity); std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); + std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id); std::vector insert(const std::vector & multiple_entities, bool replace_if_exists = false, bool throw_if_exists = true); + std::vector insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists = false, bool throw_if_exists = true); /// Inserts an entity to the storage. Returns ID of a new entry in the storage. std::optional tryInsert(const AccessEntityPtr & entity); @@ -179,7 +185,7 @@ protected: virtual std::vector findAllImpl(AccessEntityType type) const = 0; virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const = 0; virtual std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const; - virtual std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); + virtual std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id); virtual bool removeImpl(const UUID & id, bool throw_if_not_exists); virtual bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); virtual std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const; @@ -240,6 +246,19 @@ std::shared_ptr IAccessStorage::read(const String & name, bo } +template +std::vector IAccessStorage::read(const std::vector & ids, bool throw_if_not_exists) const +{ + std::vector result; + result.reserve(ids.size()); + + for (const auto & id : ids) + result.push_back(read(id, throw_if_not_exists)); + + return result; +} + + template std::shared_ptr IAccessStorage::tryRead(const UUID & id) const { diff --git a/src/Access/MemoryAccessStorage.cpp b/src/Access/MemoryAccessStorage.cpp index c4192ee4552..16f3b3793a0 100644 --- a/src/Access/MemoryAccessStorage.cpp +++ b/src/Access/MemoryAccessStorage.cpp @@ -63,9 +63,9 @@ AccessEntityPtr MemoryAccessStorage::readImpl(const UUID & id, bool throw_if_not } -std::optional MemoryAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +std::optional MemoryAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) { - UUID id = generateRandomID(); + UUID id = set_id ? *set_id : generateRandomID(); if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists)) return id; diff --git a/src/Access/MemoryAccessStorage.h b/src/Access/MemoryAccessStorage.h index b63132147da..7c19ab9c4ab 100644 --- a/src/Access/MemoryAccessStorage.h +++ b/src/Access/MemoryAccessStorage.h @@ -44,7 +44,7 @@ private: std::optional findImpl(AccessEntityType type, const String & name) const override; std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index f36beb3f80b..169dc889509 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -213,6 +213,42 @@ ConstStoragePtr MultipleAccessStorage::getStorageByName(const DB::String & stora return const_cast(this)->getStorageByName(storage_name); } +StoragePtr MultipleAccessStorage::findExcludingStorage(AccessEntityType type, const DB::String & name, DB::MultipleAccessStorage::StoragePtr exclude) const +{ + auto storages = getStoragesInternal(); + for (const auto & storage : *storages) + { + if (storage == exclude) + continue; + + if (storage->find(type, name)) + return storage; + } + + return nullptr; +} + +void MultipleAccessStorage::moveAccessEntities(const std::vector & ids, const String & source_storage_name, const String & destination_storage_name) +{ + auto source_storage = findStorageByName(source_storage_name); + auto destination_storage = findStorageByName(destination_storage_name); + + auto to_move = source_storage->read(ids); + source_storage->remove(ids); + + try + { + destination_storage->insert(to_move, ids); + } + catch (Exception & e) + { + e.addMessage("while moving access entities"); + + source_storage->insert(to_move, ids); + throw; + } +} + AccessEntityPtr MultipleAccessStorage::readImpl(const UUID & id, bool throw_if_not_exists) const { if (auto storage = findStorage(id)) @@ -280,7 +316,7 @@ void MultipleAccessStorage::reload(ReloadMode reload_mode) } -std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) +std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) { std::shared_ptr storage_for_insertion; @@ -303,7 +339,7 @@ std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & en getStorageName()); } - auto id = storage_for_insertion->insert(entity, replace_if_exists, throw_if_exists); + auto id = storage_for_insertion->insert(entity, replace_if_exists, throw_if_exists, set_id); if (id) { std::lock_guard lock{mutex}; diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index 1898c6ba5d3..1d95296b796 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -46,6 +46,11 @@ public: ConstStoragePtr getStorageByName(const String & storage_name) const; StoragePtr getStorageByName(const String & storage_name); + /// Search for an access entity storage, excluding one. Returns nullptr if not found. + StoragePtr findExcludingStorage(AccessEntityType type, const String & name, StoragePtr exclude) const; + + void moveAccessEntities(const std::vector & ids, const String & source_storage_name, const String & destination_storage_name); + bool exists(const UUID & id) const override; bool isBackupAllowed() const override; @@ -58,7 +63,7 @@ protected: std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override; @@ -70,6 +75,8 @@ private: std::shared_ptr nested_storages TSA_GUARDED_BY(mutex); mutable CacheBase ids_cache TSA_GUARDED_BY(mutex); mutable std::mutex mutex; + + mutable std::mutex move_mutex; }; } diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index 56d68be9268..efa9dbbdfaa 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -108,9 +108,9 @@ static void retryOnZooKeeperUserError(size_t attempts, Func && function) } } -std::optional ReplicatedAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +std::optional ReplicatedAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) { - const UUID id = generateRandomID(); + const UUID id = set_id ? *set_id : generateRandomID(); if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists)) return id; diff --git a/src/Access/ReplicatedAccessStorage.h b/src/Access/ReplicatedAccessStorage.h index 555d58e6b04..0a0c53d3bc4 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -46,7 +46,7 @@ private: std::unique_ptr watching_thread; std::shared_ptr> watched_queue; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; + std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp index 83a620d11c6..e271497ff5c 100644 --- a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp @@ -14,6 +14,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int ACCESS_ENTITY_ALREADY_EXISTS; +} + namespace { void updateQuotaFromQueryImpl( @@ -90,6 +96,15 @@ BlockIO InterpreterCreateQuotaQuery::execute() if (query.roles) roles_from_query = RolesOrUsersSet{*query.roles, access_control, getContext()->getUserID()}; + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + + if (!query.storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(query.storage_name); + storage = storage_ptr.get(); + } + if (query.alter) { auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr @@ -100,11 +115,11 @@ BlockIO InterpreterCreateQuotaQuery::execute() }; if (query.if_exists) { - auto ids = access_control.find(query.names); - access_control.tryUpdate(ids, update_func); + auto ids = storage->find(query.names); + storage->tryUpdate(ids, update_func); } else - access_control.update(access_control.getIDs(query.names), update_func); + storage->update(storage->getIDs(query.names), update_func); } else { @@ -116,12 +131,21 @@ BlockIO InterpreterCreateQuotaQuery::execute() new_quotas.emplace_back(std::move(new_quota)); } + if (!query.storage_name.empty()) + { + for (const auto & name : query.names) + { + if (auto another_storage_ptr = access_control.findExcludingStorage(AccessEntityType::QUOTA, name, storage_ptr)) + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Quota {} already exists in storage {}", name, another_storage_ptr->getStorageName()); + } + } + if (query.if_not_exists) - access_control.tryInsert(new_quotas); + storage->tryInsert(new_quotas); else if (query.or_replace) - access_control.insertOrReplace(new_quotas); + storage->insertOrReplace(new_quotas); else - access_control.insert(new_quotas); + storage->insert(new_quotas); } return {}; diff --git a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp index 87abe002d3d..f6c3103a7a9 100644 --- a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp @@ -97,13 +97,7 @@ BlockIO InterpreterCreateRoleQuery::execute() { for (const auto & name : query.names) { - auto id = access_control.find(name); - - if (!id) - continue; - - auto another_storage_ptr = access_control.findStorage(*id); - if (another_storage_ptr != storage_ptr) + if (auto another_storage_ptr = access_control.findExcludingStorage(AccessEntityType::ROLE, name, storage_ptr)) throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Role {} already exists in storage {}", name, another_storage_ptr->getStorageName()); } } diff --git a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp index 87dc9039c17..a938d7afc16 100644 --- a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp @@ -14,6 +14,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int ACCESS_ENTITY_ALREADY_EXISTS; +} + namespace { void updateRowPolicyFromQueryImpl( @@ -66,6 +72,16 @@ BlockIO InterpreterCreateRowPolicyQuery::execute() if (query.roles) roles_from_query = RolesOrUsersSet{*query.roles, access_control, getContext()->getUserID()}; + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + + if (!query.storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(query.storage_name); + storage = storage_ptr.get(); + } + + Strings names = query.names->toStrings(); if (query.alter) { auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr @@ -74,14 +90,13 @@ BlockIO InterpreterCreateRowPolicyQuery::execute() updateRowPolicyFromQueryImpl(*updated_policy, query, {}, roles_from_query); return updated_policy; }; - Strings names = query.names->toStrings(); if (query.if_exists) { - auto ids = access_control.find(names); - access_control.tryUpdate(ids, update_func); + auto ids = storage->find(names); + storage->tryUpdate(ids, update_func); } else - access_control.update(access_control.getIDs(names), update_func); + storage->update(storage->getIDs(names), update_func); } else { @@ -93,12 +108,21 @@ BlockIO InterpreterCreateRowPolicyQuery::execute() new_policies.emplace_back(std::move(new_policy)); } + if (!query.storage_name.empty()) + { + for (const auto & name : names) + { + if (auto another_storage_ptr = access_control.findExcludingStorage(AccessEntityType::ROW_POLICY, name, storage_ptr)) + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Row policy {} already exists in storage {}", name, another_storage_ptr->getStorageName()); + } + } + if (query.if_not_exists) - access_control.tryInsert(new_policies); + storage->tryInsert(new_policies); else if (query.or_replace) - access_control.insertOrReplace(new_policies); + storage->insertOrReplace(new_policies); else - access_control.insert(new_policies); + storage->insert(new_policies); } return {}; diff --git a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp index 0727b6f2182..bac0146e821 100644 --- a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp @@ -10,6 +10,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int ACCESS_ENTITY_ALREADY_EXISTS; +} + namespace { void updateSettingsProfileFromQueryImpl( @@ -67,6 +73,16 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute() if (query.to_roles) roles_from_query = RolesOrUsersSet{*query.to_roles, access_control, getContext()->getUserID()}; + + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + + if (!query.storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(query.storage_name); + storage = storage_ptr.get(); + } + if (query.alter) { auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr @@ -77,11 +93,11 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute() }; if (query.if_exists) { - auto ids = access_control.find(query.names); - access_control.tryUpdate(ids, update_func); + auto ids = storage->find(query.names); + storage->tryUpdate(ids, update_func); } else - access_control.update(access_control.getIDs(query.names), update_func); + storage->update(storage->getIDs(query.names), update_func); } else { @@ -93,12 +109,21 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute() new_profiles.emplace_back(std::move(new_profile)); } + if (!query.storage_name.empty()) + { + for (const auto & name : query.names) + { + if (auto another_storage_ptr = access_control.findExcludingStorage(AccessEntityType::SETTINGS_PROFILE, name, storage_ptr)) + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "Settings profile {} already exists in storage {}", name, another_storage_ptr->getStorageName()); + } + } + if (query.if_not_exists) - access_control.tryInsert(new_profiles); + storage->tryInsert(new_profiles); else if (query.or_replace) - access_control.insertOrReplace(new_profiles); + storage->insertOrReplace(new_profiles); else - access_control.insert(new_profiles); + storage->insert(new_profiles); } return {}; diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 165937560cc..f1d921a9eb2 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -17,6 +17,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int ACCESS_ENTITY_ALREADY_EXISTS; } namespace { @@ -139,6 +140,16 @@ BlockIO InterpreterCreateUserQuery::execute() if (!query.cluster.empty()) return executeDDLQueryOnCluster(query_ptr, getContext()); + IAccessStorage * storage = &access_control; + MultipleAccessStorage::StoragePtr storage_ptr; + + if (!query.storage_name.empty()) + { + storage_ptr = access_control.getStorageByName(query.storage_name); + storage = storage_ptr.get(); + } + + Strings names = query.names->toStrings(); if (query.alter) { std::optional grantees_from_query; @@ -152,14 +163,13 @@ BlockIO InterpreterCreateUserQuery::execute() return updated_user; }; - Strings names = query.names->toStrings(); if (query.if_exists) { - auto ids = access_control.find(names); - access_control.tryUpdate(ids, update_func); + auto ids = storage->find(names); + storage->tryUpdate(ids, update_func); } else - access_control.update(access_control.getIDs(names), update_func); + storage->update(storage->getIDs(names), update_func); } else { @@ -171,13 +181,22 @@ BlockIO InterpreterCreateUserQuery::execute() new_users.emplace_back(std::move(new_user)); } + if (!query.storage_name.empty()) + { + for (const auto & name : names) + { + if (auto another_storage_ptr = access_control.findExcludingStorage(AccessEntityType::USER, name, storage_ptr)) + throw Exception(ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS, "User {} already exists in storage {}", name, another_storage_ptr->getStorageName()); + } + } + std::vector ids; if (query.if_not_exists) - ids = access_control.tryInsert(new_users); + ids = storage->tryInsert(new_users); else if (query.or_replace) - ids = access_control.insertOrReplace(new_users); + ids = storage->insertOrReplace(new_users); else - ids = access_control.insert(new_users); + ids = storage->insert(new_users); if (query.grantees) { diff --git a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp new file mode 100644 index 00000000000..ac7ff4efcb6 --- /dev/null +++ b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp @@ -0,0 +1,96 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int ACCESS_ENTITY_NOT_FOUND; +} + + +BlockIO InterpreterMoveAccessEntityQuery::execute() +{ + auto & query = query_ptr->as(); + auto & access_control = getContext()->getAccessControl(); + getContext()->checkAccess(getRequiredAccess()); + + if (!query.cluster.empty()) + return executeDDLQueryOnCluster(query_ptr, getContext()); + + query.replaceEmptyDatabase(getContext()->getCurrentDatabase()); + + std::vector ids; + if (query.type == AccessEntityType::ROW_POLICY) + ids = access_control.find(query.type, query.row_policy_names->toStrings()); + else + ids = access_control.find(query.type, query.names); + + if (ids.empty()) + return {}; + + /// Validate that all entities are from the same storage. + const auto source_storage = access_control.findStorage(ids.front()); + if (!source_storage->exists(ids)) + throw Exception(ErrorCodes::ACCESS_ENTITY_NOT_FOUND, "All access entities must be from the same storage in order to be moved"); + + access_control.moveAccessEntities(ids, source_storage->getStorageName(), query.storage_name); + return {}; +} + + +AccessRightsElements InterpreterMoveAccessEntityQuery::getRequiredAccess() const +{ + const auto & query = query_ptr->as(); + AccessRightsElements res; + switch (query.type) + { + case AccessEntityType::USER: + { + res.emplace_back(AccessType::DROP_USER); + res.emplace_back(AccessType::CREATE_USER); + return res; + } + case AccessEntityType::ROLE: + { + res.emplace_back(AccessType::DROP_ROLE); + res.emplace_back(AccessType::CREATE_ROLE); + return res; + } + case AccessEntityType::SETTINGS_PROFILE: + { + res.emplace_back(AccessType::DROP_SETTINGS_PROFILE); + res.emplace_back(AccessType::CREATE_SETTINGS_PROFILE); + return res; + } + case AccessEntityType::ROW_POLICY: + { + if (query.row_policy_names) + { + for (const auto & row_policy_name : query.row_policy_names->full_names) + { + res.emplace_back(AccessType::DROP_ROW_POLICY, row_policy_name.database, row_policy_name.table_name); + res.emplace_back(AccessType::CREATE_ROW_POLICY, row_policy_name.database, row_policy_name.table_name); + } + } + return res; + } + case AccessEntityType::QUOTA: + { + res.emplace_back(AccessType::DROP_QUOTA); + res.emplace_back(AccessType::CREATE_QUOTA); + return res; + } + case AccessEntityType::MAX: + break; + } + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{}: type is not supported by DROP query", toString(query.type)); +} + +} diff --git a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.h b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.h new file mode 100644 index 00000000000..1a70bf7c065 --- /dev/null +++ b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.h @@ -0,0 +1,24 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class AccessRightsElements; + +class InterpreterMoveAccessEntityQuery : public IInterpreter, WithMutableContext +{ +public: + InterpreterMoveAccessEntityQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) {} + + BlockIO execute() override; + +private: + AccessRightsElements getRequiredAccess() const; + + ASTPtr query_ptr; +}; + +} diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 9cd1f2a251c..27c94119750 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -39,6 +39,7 @@ #include #include #include +#include #include #include #include @@ -96,6 +97,7 @@ #include #include #include +#include #include #include #include @@ -314,6 +316,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { return std::make_unique(query, context); diff --git a/src/Parsers/Access/ASTCreateQuotaQuery.cpp b/src/Parsers/Access/ASTCreateQuotaQuery.cpp index 56abedf5235..091b62b0a9f 100644 --- a/src/Parsers/Access/ASTCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ASTCreateQuotaQuery.cpp @@ -170,6 +170,12 @@ void ASTCreateQuotaQuery::formatImpl(const FormatSettings & settings, FormatStat settings.ostr << (settings.hilite ? hilite_keyword : "") << " OR REPLACE" << (settings.hilite ? hilite_none : ""); formatNames(names, settings); + + if (!storage_name.empty()) + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") + << " IN " << (settings.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(settings); if (!new_name.empty()) diff --git a/src/Parsers/Access/ASTCreateQuotaQuery.h b/src/Parsers/Access/ASTCreateQuotaQuery.h index f5eb59800ec..aecbbb00f9a 100644 --- a/src/Parsers/Access/ASTCreateQuotaQuery.h +++ b/src/Parsers/Access/ASTCreateQuotaQuery.h @@ -38,6 +38,7 @@ public: Strings names; String new_name; std::optional key_type; + String storage_name; struct Limits { diff --git a/src/Parsers/Access/ASTCreateRoleQuery.cpp b/src/Parsers/Access/ASTCreateRoleQuery.cpp index 14946f2d3cd..eeeb34c97e4 100644 --- a/src/Parsers/Access/ASTCreateRoleQuery.cpp +++ b/src/Parsers/Access/ASTCreateRoleQuery.cpp @@ -74,7 +74,7 @@ void ASTCreateRoleQuery::formatImpl(const FormatSettings & format, FormatState & if (!storage_name.empty()) format.ostr << (format.hilite ? IAST::hilite_keyword : "") - << " AT " << (format.hilite ? IAST::hilite_none : "") + << " IN " << (format.hilite ? IAST::hilite_none : "") << backQuoteIfNeed(storage_name); formatOnCluster(format); diff --git a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp index ca888be2cfe..a423581e300 100644 --- a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp @@ -168,6 +168,11 @@ void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, Format settings.ostr << " "; names->format(settings); + if (!storage_name.empty()) + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") + << " IN " << (settings.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(settings); assert(names->cluster.empty()); diff --git a/src/Parsers/Access/ASTCreateRowPolicyQuery.h b/src/Parsers/Access/ASTCreateRowPolicyQuery.h index 32d98fab822..3f2418e7287 100644 --- a/src/Parsers/Access/ASTCreateRowPolicyQuery.h +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.h @@ -35,6 +35,7 @@ public: bool if_exists = false; bool if_not_exists = false; bool or_replace = false; + String storage_name; std::shared_ptr names; String new_short_name; diff --git a/src/Parsers/Access/ASTCreateSettingsProfileQuery.cpp b/src/Parsers/Access/ASTCreateSettingsProfileQuery.cpp index 56ddef433ef..3379486d789 100644 --- a/src/Parsers/Access/ASTCreateSettingsProfileQuery.cpp +++ b/src/Parsers/Access/ASTCreateSettingsProfileQuery.cpp @@ -81,6 +81,12 @@ void ASTCreateSettingsProfileQuery::formatImpl(const FormatSettings & format, Fo format.ostr << (format.hilite ? hilite_keyword : "") << " OR REPLACE" << (format.hilite ? hilite_none : ""); formatNames(names, format); + + if (!storage_name.empty()) + format.ostr << (format.hilite ? IAST::hilite_keyword : "") + << " IN " << (format.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(format); if (!new_name.empty()) diff --git a/src/Parsers/Access/ASTCreateSettingsProfileQuery.h b/src/Parsers/Access/ASTCreateSettingsProfileQuery.h index c1a64998f29..be01aae1e26 100644 --- a/src/Parsers/Access/ASTCreateSettingsProfileQuery.h +++ b/src/Parsers/Access/ASTCreateSettingsProfileQuery.h @@ -28,6 +28,7 @@ public: bool if_exists = false; bool if_not_exists = false; bool or_replace = false; + String storage_name; Strings names; String new_name; diff --git a/src/Parsers/Access/ASTCreateUserQuery.cpp b/src/Parsers/Access/ASTCreateUserQuery.cpp index 0611545adf0..dbb4c1c6d57 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.cpp +++ b/src/Parsers/Access/ASTCreateUserQuery.cpp @@ -208,6 +208,11 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState & format.ostr << " "; names->format(format); + if (!storage_name.empty()) + format.ostr << (format.hilite ? IAST::hilite_keyword : "") + << " IN " << (format.hilite ? IAST::hilite_none : "") + << backQuoteIfNeed(storage_name); + formatOnCluster(format); if (new_name) diff --git a/src/Parsers/Access/ASTCreateUserQuery.h b/src/Parsers/Access/ASTCreateUserQuery.h index 62ddbfd0040..06940e67d91 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.h +++ b/src/Parsers/Access/ASTCreateUserQuery.h @@ -45,6 +45,7 @@ public: std::shared_ptr names; std::optional new_name; + String storage_name; std::shared_ptr auth_data; diff --git a/src/Parsers/Access/ASTDropAccessEntityQuery.cpp b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp index 02b29f3fd2c..bcd7105d0e9 100644 --- a/src/Parsers/Access/ASTDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp @@ -54,8 +54,8 @@ void ASTDropAccessEntityQuery::formatImpl(const FormatSettings & settings, Forma formatNames(names, settings); if (!storage_name.empty()) - settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") - << " FROM " << (settings.hilite ? IAST::hilite_none : "") + settings.ostr << (settings.hilite ? hilite_keyword : "") + << " FROM " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(storage_name); formatOnCluster(settings); diff --git a/src/Parsers/Access/ASTMoveAccessEntityQuery.cpp b/src/Parsers/Access/ASTMoveAccessEntityQuery.cpp new file mode 100644 index 00000000000..285f07854c0 --- /dev/null +++ b/src/Parsers/Access/ASTMoveAccessEntityQuery.cpp @@ -0,0 +1,64 @@ +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + void formatNames(const Strings & names, const IAST::FormatSettings & settings) + { + bool need_comma = false; + for (const auto & name : names) + { + if (std::exchange(need_comma, true)) + settings.ostr << ','; + settings.ostr << ' ' << backQuoteIfNeed(name); + } + } +} + +String ASTMoveAccessEntityQuery::getID(char) const +{ + return String("MOVE ") + toString(type) + " query"; +} + +ASTPtr ASTMoveAccessEntityQuery::clone() const +{ + auto res = std::make_shared(*this); + + if (row_policy_names) + res->row_policy_names = std::static_pointer_cast(row_policy_names->clone()); + + return res; +} + +void ASTMoveAccessEntityQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") + << "MOVE " << AccessEntityTypeInfo::get(type).name + << (settings.hilite ? hilite_none : ""); + + if (type == AccessEntityType::ROW_POLICY) + { + settings.ostr << " "; + row_policy_names->format(settings); + } + else + formatNames(names, settings); + + settings.ostr << (settings.hilite ? hilite_keyword : "") + << " TO " << (settings.hilite ? hilite_none : "") + << backQuoteIfNeed(storage_name); + + formatOnCluster(settings); +} + +void ASTMoveAccessEntityQuery::replaceEmptyDatabase(const String & current_database) const +{ + if (row_policy_names) + row_policy_names->replaceEmptyDatabase(current_database); +} +} diff --git a/src/Parsers/Access/ASTMoveAccessEntityQuery.h b/src/Parsers/Access/ASTMoveAccessEntityQuery.h new file mode 100644 index 00000000000..aa2b3b0f98c --- /dev/null +++ b/src/Parsers/Access/ASTMoveAccessEntityQuery.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ +class ASTRowPolicyNames; + +/** MOVE {USER | ROLE | QUOTA | [ROW] POLICY | [SETTINGS] PROFILE} [IF EXISTS] name [,...] [ON [database.]table [,...]] TO storage_name + */ +class ASTMoveAccessEntityQuery : public IAST, public ASTQueryWithOnCluster +{ +public: + AccessEntityType type; + Strings names; + std::shared_ptr row_policy_names; + + String storage_name; + + String getID(char) const override; + ASTPtr clone() const override; + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; + ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } + + void replaceEmptyDatabase(const String & current_database) const; + + QueryKind getQueryKind() const override { return QueryKind::Move; } +}; +} diff --git a/src/Parsers/Access/ParserCreateQuotaQuery.cpp b/src/Parsers/Access/ParserCreateQuotaQuery.cpp index 2330ba7771c..05c2820b012 100644 --- a/src/Parsers/Access/ParserCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ParserCreateQuotaQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -288,6 +289,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe std::optional key_type; std::vector all_limits; String cluster; + String storage_name; while (true) { @@ -310,6 +312,9 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + continue; + break; } @@ -332,6 +337,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->key_type = key_type; query->all_limits = std::move(all_limits); query->roles = std::move(roles); + query->storage_name = std::move(storage_name); return true; } diff --git a/src/Parsers/Access/ParserCreateRoleQuery.cpp b/src/Parsers/Access/ParserCreateRoleQuery.cpp index de9e5baed18..b548c07ec70 100644 --- a/src/Parsers/Access/ParserCreateRoleQuery.cpp +++ b/src/Parsers/Access/ParserCreateRoleQuery.cpp @@ -93,9 +93,6 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec String cluster; String storage_name; - if (ParserKeyword{"AT"}.ignore(pos, expected)) - parseStorageName(pos, expected, storage_name); - while (true) { if (alter && new_name.empty() && (names.size() == 1) && parseRenameTo(pos, expected, new_name)) @@ -114,6 +111,9 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + continue; + break; } diff --git a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp index 2c25fc14e7d..b2177ca4ade 100644 --- a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -245,6 +246,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & String new_short_name; std::optional is_restrictive; std::vector> filters; + String storage_name; while (true) { @@ -271,6 +273,9 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + continue; + break; } @@ -294,6 +299,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & query->is_restrictive = is_restrictive; query->filters = std::move(filters); query->roles = std::move(roles); + query->storage_name = std::move(storage_name); return true; } diff --git a/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp b/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp index c58a3035dc6..634ff68427a 100644 --- a/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp +++ b/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -111,6 +112,7 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec String new_name; std::shared_ptr settings; String cluster; + String storage_name; while (true) { @@ -130,6 +132,9 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + continue; + break; } @@ -152,6 +157,7 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec query->new_name = std::move(new_name); query->settings = std::move(settings); query->to_roles = std::move(to_roles); + query->storage_name = std::move(storage_name); return true; } diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index 0344fb99c04..d95d41296dc 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -414,6 +414,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec std::shared_ptr grantees; std::shared_ptr default_database; String cluster; + String storage_name; while (true) { @@ -480,6 +481,9 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec } } + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + continue; + break; } @@ -514,6 +518,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->settings = std::move(settings); query->grantees = std::move(grantees); query->default_database = std::move(default_database); + query->storage_name = std::move(storage_name); if (query->auth_data) query->children.push_back(query->auth_data); diff --git a/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp b/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp new file mode 100644 index 00000000000..6ac41b77b84 --- /dev/null +++ b/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp @@ -0,0 +1,93 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + bool parseEntityType(IParserBase::Pos & pos, Expected & expected, AccessEntityType & type) + { + for (auto i : collections::range(AccessEntityType::MAX)) + { + const auto & type_info = AccessEntityTypeInfo::get(i); + if (ParserKeyword{type_info.name}.ignore(pos, expected) + || (!type_info.alias.empty() && ParserKeyword{type_info.alias}.ignore(pos, expected))) + { + type = i; + return true; + } + } + return false; + } + + + bool parseOnCluster(IParserBase::Pos & pos, Expected & expected, String & cluster) + { + return IParserBase::wrapParseImpl(pos, [&] + { + return ParserKeyword{"ON"}.ignore(pos, expected) && ASTQueryWithOnCluster::parse(pos, cluster, expected); + }); + } +} + + +bool ParserMoveAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (!ParserKeyword{"MOVE"}.ignore(pos, expected)) + return false; + + AccessEntityType type; + if (!parseEntityType(pos, expected, type)) + return false; + + Strings names; + std::shared_ptr row_policy_names; + String storage_name; + String cluster; + + if ((type == AccessEntityType::USER) || (type == AccessEntityType::ROLE)) + { + if (!parseUserNames(pos, expected, names)) + return false; + } + else if (type == AccessEntityType::ROW_POLICY) + { + ParserRowPolicyNames parser; + ASTPtr ast; + parser.allowOnCluster(); + if (!parser.parse(pos, ast, expected)) + return false; + row_policy_names = typeid_cast>(ast); + cluster = std::exchange(row_policy_names->cluster, ""); + } + else + { + if (!parseIdentifiersOrStringLiterals(pos, expected, names)) + return false; + } + + if (!ParserKeyword{"TO"}.ignore(pos, expected) || !parseStorageName(pos, expected, storage_name)) + return false; + + if (cluster.empty()) + parseOnCluster(pos, expected, cluster); + + auto query = std::make_shared(); + node = query; + + query->type = type; + query->cluster = std::move(cluster); + query->names = std::move(names); + query->row_policy_names = std::move(row_policy_names); + query->storage_name = std::move(storage_name); + + return true; +} +} diff --git a/src/Parsers/Access/ParserMoveAccessEntityQuery.h b/src/Parsers/Access/ParserMoveAccessEntityQuery.h new file mode 100644 index 00000000000..3accec883dd --- /dev/null +++ b/src/Parsers/Access/ParserMoveAccessEntityQuery.h @@ -0,0 +1,17 @@ +#pragma once + +#include + + +namespace DB +{ +/** Parses queries like + * MOVE {USER | ROLE | QUOTA | [ROW] POLICY | [SETTINGS] PROFILE} [IF EXISTS] name [,...] [ON [database.]table [,...]] TO storage_name + */ +class ParserMoveAccessEntityQuery : public IParserBase +{ +protected: + const char * getName() const override { return "MOVE access entity query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/src/Parsers/Access/parseUserName.h b/src/Parsers/Access/parseUserName.h index 717911bf2f1..64fed2ce21c 100644 --- a/src/Parsers/Access/parseUserName.h +++ b/src/Parsers/Access/parseUserName.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -36,7 +37,7 @@ inline bool parseRoleNames(IParser::Pos & pos, Expected & expected, Strings & ro inline bool parseStorageName(IParser::Pos & pos, Expected & expected, String & storage_name) { - return parseUserName(pos, expected, storage_name); + return parseIdentifierOrStringLiteral(pos, expected, storage_name); } } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index aa5302a15b9..92fb8ed275b 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -290,6 +290,7 @@ public: Alter, Grant, Revoke, + Move, System, Set, Use, diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 77e7b58e6b1..7ed69940bed 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include @@ -54,6 +55,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserCreateIndexQuery create_index_p; ParserDropIndexQuery drop_index_p; ParserDropAccessEntityQuery drop_access_entity_p; + ParserMoveAccessEntityQuery move_access_entity_p; ParserGrantQuery grant_p; ParserSetRoleQuery set_role_p; ParserExternalDDLQuery external_ddl_p; @@ -80,6 +82,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_index_p.parse(pos, node, expected) || drop_index_p.parse(pos, node, expected) || drop_access_entity_p.parse(pos, node, expected) + || move_access_entity_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected) || external_ddl_p.parse(pos, node, expected) || transaction_control_p.parse(pos, node, expected) diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index ee5d4b5df93..8e040a18a88 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -471,7 +471,7 @@ def test_introspection(): [ [ "A", - "local directory", + "local_directory", "no_password", "{}", "['::/0']", @@ -484,7 +484,7 @@ def test_introspection(): ], [ "B", - "local directory", + "local_directory", "no_password", "{}", "['::/0']", diff --git a/tests/integration/test_multi_access_storage_role_management/configs/memory.xml b/tests/integration/test_multi_access_storage_role_management/configs/memory.xml new file mode 100644 index 00000000000..c2f60c8bbc1 --- /dev/null +++ b/tests/integration/test_multi_access_storage_role_management/configs/memory.xml @@ -0,0 +1,5 @@ + + + + + diff --git a/tests/integration/test_multi_access_storage_role_management/configs/roles.xml b/tests/integration/test_multi_access_storage_role_management/configs/roles.xml index 672c7e24728..faaf1ee27ae 100644 --- a/tests/integration/test_multi_access_storage_role_management/configs/roles.xml +++ b/tests/integration/test_multi_access_storage_role_management/configs/roles.xml @@ -1,8 +1,4 @@ - - - - diff --git a/tests/integration/test_multi_access_storage_role_management/test.py b/tests/integration/test_multi_access_storage_role_management/test.py index d9a4a73c56b..fb4bbd6b718 100644 --- a/tests/integration/test_multi_access_storage_role_management/test.py +++ b/tests/integration/test_multi_access_storage_role_management/test.py @@ -8,6 +8,7 @@ cluster = ClickHouseCluster(__file__) node = cluster.add_instance( "node", stay_alive=True, + main_configs=["configs/memory.xml"] ) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -28,6 +29,68 @@ def started_cluster(): cluster.shutdown() +def execute_test_for_access_type(access_type: str, system_table_name: str): + node.query(f"CREATE {access_type} test1 IN local_directory") + node.query(f"CREATE {access_type} test2 IN local_directory") + node.query(f"CREATE {access_type} test3 IN local_directory") + + node.query(f"CREATE {access_type} test4 IN memory") + node.query(f"CREATE {access_type} test5 IN memory") + node.query(f"CREATE {access_type} test6 IN memory") + + # Already exists + with pytest.raises(QueryRuntimeException): + node.query(f"CREATE {access_type} test1 IN memory") + + node.query(f"MOVE {access_type} test1 TO memory") + assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test1'") == TSV(["memory"]) + + node.query(f"MOVE {access_type} test2 TO local_directory") + assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'") == TSV(["local_directory"]) + + node.query(f"MOVE {access_type} test2,test3 TO memory") + assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'") == TSV(["memory"]) + assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test3'") == TSV(["memory"]) + + node.query(f"MOVE {access_type} test4,test5 TO local_directory") + + # Different storages + with pytest.raises(QueryRuntimeException): + node.query(f"MOVE {access_type} test4,test1 TO memory") + + # Doesn't exist + with pytest.raises(QueryRuntimeException): + node.query(f"MOVE {access_type} test7 TO local_directory") + + # Storage doesn't exist + with pytest.raises(QueryRuntimeException): + node.query(f"MOVE {access_type} test6 TO non_existing_storage") + + # Unwriteable storage + with pytest.raises(QueryRuntimeException): + node.query(f"MOVE {access_type} test6 TO users_xml") + + +def test_roles(): + execute_test_for_access_type("ROLE", "roles") + + +def test_users(): + execute_test_for_access_type("USER", "users") + + +def test_settings_profiles(): + execute_test_for_access_type("SETTINGS PROFILE", "settings_profiles") + + +def test_quotas(): + execute_test_for_access_type("QUOTA", "quotas") + + +def test_row_policies(): + execute_test_for_access_type("ROW POLICY", "row_policies") + + def test_role_from_different_storages(): node.query("CREATE ROLE default_role") node.query("GRANT SELECT ON system.* TO default_role") @@ -72,9 +135,9 @@ def test_role_from_different_storages(): # Already exists with pytest.raises(QueryRuntimeException): - node.query("CREATE ROLE default_role AT memory") + node.query("CREATE ROLE default_role IN memory") - node.query("CREATE ROLE other_role AT memory") + node.query("CREATE ROLE other_role IN memory") assert node.query( "SELECT storage FROM system.roles WHERE name = 'other_role'" diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index 9a352f43eab..43a402d9fda 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -287,7 +287,7 @@ def test_introspection(): assert instance.query( "SELECT name, storage from system.roles WHERE name IN ('R1', 'R2') ORDER BY name" - ) == TSV([["R1", "local directory"], ["R2", "local directory"]]) + ) == TSV([["R1", "local_directory"], ["R2", "local_directory"]]) assert instance.query( "SELECT * from system.grants WHERE user_name IN ('A', 'B') OR role_name IN ('R1', 'R2') ORDER BY user_name, role_name, access_type, database, table, column, is_partial_revoke, grant_option" diff --git a/tests/integration/test_settings_profile/test.py b/tests/integration/test_settings_profile/test.py index 13acc3f1042..8770cc82064 100644 --- a/tests/integration/test_settings_profile/test.py +++ b/tests/integration/test_settings_profile/test.py @@ -88,7 +88,7 @@ def test_smoke(): ) ) assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 1, 0, "['robin']", "[]"] + ["xyz", "local_directory", 1, 0, "['robin']", "[]"] ] assert system_settings_profile_elements(profile_name="xyz") == [ [ @@ -120,7 +120,7 @@ def test_smoke(): instance.query("SET max_memory_usage = 80000000", user="robin") instance.query("SET max_memory_usage = 120000000", user="robin") assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 1, 0, "[]", "[]"] + ["xyz", "local_directory", 1, 0, "[]", "[]"] ] assert system_settings_profile_elements(user_name="robin") == [] @@ -201,7 +201,7 @@ def test_settings_from_granted_role(): ) ) assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 2, 0, "[]", "[]"] + ["xyz", "local_directory", 2, 0, "[]", "[]"] ] assert system_settings_profile_elements(profile_name="xyz") == [ [ @@ -276,7 +276,7 @@ def test_settings_from_granted_role(): ) ) assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 2, 0, "['worker']", "[]"] + ["xyz", "local_directory", 2, 0, "['worker']", "[]"] ] instance.query("ALTER SETTINGS PROFILE xyz TO NONE") @@ -293,7 +293,7 @@ def test_settings_from_granted_role(): ) instance.query("SET max_memory_usage = 120000000", user="robin") assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 2, 0, "[]", "[]"] + ["xyz", "local_directory", 2, 0, "[]", "[]"] ] @@ -323,7 +323,7 @@ def test_inheritance(): ) assert system_settings_profile("xyz") == [ - ["xyz", "local directory", 1, 0, "[]", "[]"] + ["xyz", "local_directory", 1, 0, "[]", "[]"] ] assert system_settings_profile_elements(profile_name="xyz") == [ [ @@ -340,7 +340,7 @@ def test_inheritance(): ] ] assert system_settings_profile("alpha") == [ - ["alpha", "local directory", 1, 0, "['robin']", "[]"] + ["alpha", "local_directory", 1, 0, "['robin']", "[]"] ] assert system_settings_profile_elements(profile_name="alpha") == [ ["alpha", "\\N", "\\N", 0, "\\N", "\\N", "\\N", "\\N", "\\N", "xyz"] diff --git a/tests/integration/test_user_directories/test.py b/tests/integration/test_user_directories/test.py index 45afb86f464..704fb30b2fd 100644 --- a/tests/integration/test_user_directories/test.py +++ b/tests/integration/test_user_directories/test.py @@ -38,14 +38,14 @@ def test_old_style(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users2.xml"}', 1, ], [ - "local directory", - "local directory", + "local_directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access2\\\\/"}', 2, ], @@ -62,20 +62,20 @@ def test_local_directories(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users3.xml"}', 1, ], [ - "local directory", - "local directory", + "local_directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access3\\\\/"}', 2, ], [ "local directory (ro)", - "local directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access3-ro\\\\/","readonly":true}', 3, ], @@ -92,8 +92,8 @@ def test_relative_path(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users4.xml"}', 1, ] @@ -110,8 +110,8 @@ def test_memory(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users5.xml"}', 1, ], @@ -129,20 +129,20 @@ def test_mixed_style(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users6.xml"}', 1, ], [ - "local directory", - "local directory", + "local_directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access6\\\\/"}', 2, ], [ - "local directory", - "local directory", + "local_directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access6a\\\\/"}', 3, ], @@ -160,14 +160,14 @@ def test_duplicates(): assert node.query("SELECT * FROM system.user_directories") == TSV( [ [ - "users.xml", - "users.xml", + "users_xml", + "users_xml", '{"path":"\\\\/etc\\\\/clickhouse-server\\\\/users7.xml"}', 1, ], [ - "local directory", - "local directory", + "local_directory", + "local_directory", '{"path":"\\\\/var\\\\/lib\\\\/clickhouse\\\\/access7\\\\/"}', 2, ], From a8c08b34a6ca179d2b89da87d920916978bf4129 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 21 Jul 2023 02:39:21 +0000 Subject: [PATCH 038/777] Automatic style fix --- .../test.py | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_multi_access_storage_role_management/test.py b/tests/integration/test_multi_access_storage_role_management/test.py index fb4bbd6b718..4c2696158ec 100644 --- a/tests/integration/test_multi_access_storage_role_management/test.py +++ b/tests/integration/test_multi_access_storage_role_management/test.py @@ -6,9 +6,7 @@ from helpers.client import QueryRuntimeException cluster = ClickHouseCluster(__file__) node = cluster.add_instance( - "node", - stay_alive=True, - main_configs=["configs/memory.xml"] + "node", stay_alive=True, main_configs=["configs/memory.xml"] ) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -43,14 +41,22 @@ def execute_test_for_access_type(access_type: str, system_table_name: str): node.query(f"CREATE {access_type} test1 IN memory") node.query(f"MOVE {access_type} test1 TO memory") - assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test1'") == TSV(["memory"]) + assert node.query( + f"SELECT storage FROM system.{system_table_name} WHERE name = 'test1'" + ) == TSV(["memory"]) node.query(f"MOVE {access_type} test2 TO local_directory") - assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'") == TSV(["local_directory"]) + assert node.query( + f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'" + ) == TSV(["local_directory"]) node.query(f"MOVE {access_type} test2,test3 TO memory") - assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'") == TSV(["memory"]) - assert node.query(f"SELECT storage FROM system.{system_table_name} WHERE name = 'test3'") == TSV(["memory"]) + assert node.query( + f"SELECT storage FROM system.{system_table_name} WHERE name = 'test2'" + ) == TSV(["memory"]) + assert node.query( + f"SELECT storage FROM system.{system_table_name} WHERE name = 'test3'" + ) == TSV(["memory"]) node.query(f"MOVE {access_type} test4,test5 TO local_directory") From 3398355e4649beaa2b0a3180aa687c364c245e81 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 02:04:30 +0200 Subject: [PATCH 039/777] Add something with unclear purpose --- docker/README.md | 2 +- tests/ci/docker_test.py | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/README.md b/docker/README.md index ec52ddd143e..76fdbf0f3f0 100644 --- a/docker/README.md +++ b/docker/README.md @@ -2,4 +2,4 @@ This directory contain Dockerfiles for `clickhouse-client` and `clickhouse-server`. They are updated in each release. -Also there is bunch of images for testing and CI. They are listed in `images.json` file and updated on each commit to master. If you need to add another image, place information about it into `images.json`. +Also, there is a bunch of images for testing and CI. They are listed in `images.json` file and updated on each commit to master. If you need to add another image, place information about it into `images.json`. diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index d5d27f73694..61319041b9f 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -40,6 +40,7 @@ class TestDockerImageCheck(unittest.TestCase): [ di.DockerImage("docker/test/base", "clickhouse/test-base", False), di.DockerImage("docker/docs/builder", "clickhouse/docs-builder", True), + di.DockerImage("docker/test/sqltest", "clickhouse/sqltest", True), di.DockerImage( "docker/test/stateless", "clickhouse/stateless-test", From c3c8ea9c27669a548e2404e2ed6c2e8f3c734dc9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 Jul 2023 17:30:55 +0200 Subject: [PATCH 040/777] Do something --- tests/ci/docker_test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 61319041b9f..8542de412d2 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -40,7 +40,12 @@ class TestDockerImageCheck(unittest.TestCase): [ di.DockerImage("docker/test/base", "clickhouse/test-base", False), di.DockerImage("docker/docs/builder", "clickhouse/docs-builder", True), - di.DockerImage("docker/test/sqltest", "clickhouse/sqltest", True), + di.DockerImage( + "docker/test/sqltest", + "clickhouse/sqltest", + True, + "clickhouse/test-base", # type: ignore + ), di.DockerImage( "docker/test/stateless", "clickhouse/stateless-test", From fbe9f8d0f653cedd30640c42968df1ac8cf78dff Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 Jul 2023 14:12:01 +0000 Subject: [PATCH 041/777] fix recalculation of skip indexes and projcetion in ALTER DELETE --- src/Interpreters/MutationsInterpreter.cpp | 89 +++++++++++------ src/Interpreters/MutationsInterpreter.h | 3 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 ++ src/Storages/MergeTree/IMergeTreeDataPart.h | 3 + src/Storages/MergeTree/MutateTask.cpp | 98 +++++++++++++++++-- src/Storages/StorageInMemoryMetadata.cpp | 6 +- src/Storages/StorageInMemoryMetadata.h | 4 +- ...alter_delete_indexes_projections.reference | 6 ++ ...02832_alter_delete_indexes_projections.sql | 26 +++++ 9 files changed, 201 insertions(+), 40 deletions(-) create mode 100644 tests/queries/0_stateless/02832_alter_delete_indexes_projections.reference create mode 100644 tests/queries/0_stateless/02832_alter_delete_indexes_projections.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 25c52ad8925..1a7cbb45999 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -113,13 +113,14 @@ QueryTreeNodePtr prepareQueryAffectedQueryTree(const std::vector & has_index_or_projection) + const StorageInMemoryMetadata::HasDependencyCallback & has_dependency) { NameSet new_updated_columns = updated_columns; ColumnDependencies dependencies; + while (!new_updated_columns.empty()) { - auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true, has_index_or_projection); + auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true, has_dependency); new_updated_columns.clear(); for (const auto & dependency : new_dependencies) { @@ -292,9 +293,14 @@ bool MutationsInterpreter::Source::materializeTTLRecalculateOnly() const return data && data->getSettings()->materialize_ttl_recalculate_only; } -bool MutationsInterpreter::Source::hasIndexOrProjection(const String & file_name) const +bool MutationsInterpreter::Source::hasSecondaryIndex(const String & name) const { - return part && part->checksums.has(file_name); + return part && part->hasSecondaryIndex(name); +} + +bool MutationsInterpreter::Source::hasProjection(const String & name) const +{ + return part && part->hasProjection(name); } static Names getAvailableColumnsWithVirtuals(StorageMetadataPtr metadata_snapshot, const IStorage & storage) @@ -533,13 +539,24 @@ void MutationsInterpreter::prepare(bool dry_run) validateUpdateColumns(source, metadata_snapshot, updated_columns, column_to_affected_materialized); } - std::function has_index_or_projection - = [&](const String & file_name) { return source.hasIndexOrProjection(file_name); }; + StorageInMemoryMetadata::HasDependencyCallback has_dependency = + [&](const String & name, ColumnDependency::Kind kind) + { + if (kind == ColumnDependency::PROJECTION) + return source.hasProjection(name); + + if (kind == ColumnDependency::SKIP_INDEX) + return source.hasSecondaryIndex(name); + + return true; + }; if (settings.recalculate_dependencies_of_updated_columns) - dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns, has_index_or_projection); + dependencies = getAllColumnDependencies(metadata_snapshot, updated_columns, has_dependency); + bool has_alter_delete = false; std::vector read_columns; + /// First, break a sequence of commands into stages. for (auto & command : commands) { @@ -558,6 +575,7 @@ void MutationsInterpreter::prepare(bool dry_run) predicate = makeASTFunction("isZeroOrNull", predicate); stages.back().filters.push_back(predicate); + has_alter_delete = true; } else if (command.type == MutationCommand::UPDATE) { @@ -692,8 +710,7 @@ void MutationsInterpreter::prepare(bool dry_run) if (it == std::cend(indices_desc)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown index: {}", command.index_name); - if (!source.hasIndexOrProjection("skp_idx_" + it->name + ".idx") - && !source.hasIndexOrProjection("skp_idx_" + it->name + ".idx2")) + if (!source.hasSecondaryIndex(it->name)) { auto query = (*it).expression_list_ast->clone(); auto syntax_result = TreeRewriter(context).analyze(query, all_columns); @@ -707,7 +724,7 @@ void MutationsInterpreter::prepare(bool dry_run) { mutation_kind.set(MutationKind::MUTATE_INDEX_PROJECTION); const auto & projection = projections_desc.get(command.projection_name); - if (!source.hasIndexOrProjection(projection.getDirectoryName())) + if (!source.hasProjection(projection.name)) { for (const auto & column : projection.required_columns) dependencies.emplace(column, ColumnDependency::PROJECTION); @@ -732,7 +749,7 @@ void MutationsInterpreter::prepare(bool dry_run) // just recalculate ttl_infos without remove expired data auto all_columns_vec = all_columns.getNames(); auto new_dependencies = metadata_snapshot->getColumnDependencies( - NameSet(all_columns_vec.begin(), all_columns_vec.end()), false, has_index_or_projection); + NameSet(all_columns_vec.begin(), all_columns_vec.end()), false, has_dependency); for (const auto & dependency : new_dependencies) { if (dependency.kind == ColumnDependency::TTL_EXPRESSION) @@ -758,7 +775,7 @@ void MutationsInterpreter::prepare(bool dry_run) auto all_columns_vec = all_columns.getNames(); auto all_dependencies = getAllColumnDependencies( - metadata_snapshot, NameSet(all_columns_vec.begin(), all_columns_vec.end()), has_index_or_projection); + metadata_snapshot, NameSet(all_columns_vec.begin(), all_columns_vec.end()), has_dependency); for (const auto & dependency : all_dependencies) { @@ -767,7 +784,7 @@ void MutationsInterpreter::prepare(bool dry_run) } /// Recalc only skip indices and projections of columns which could be updated by TTL. - auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true, has_index_or_projection); + auto new_dependencies = metadata_snapshot->getColumnDependencies(new_updated_columns, true, has_dependency); for (const auto & dependency : new_dependencies) { if (dependency.kind == ColumnDependency::SKIP_INDEX || dependency.kind == ColumnDependency::PROJECTION) @@ -861,30 +878,44 @@ void MutationsInterpreter::prepare(bool dry_run) for (const auto & index : metadata_snapshot->getSecondaryIndices()) { - if (source.hasIndexOrProjection("skp_idx_" + index.name + ".idx") || source.hasIndexOrProjection("skp_idx_" + index.name + ".idx2")) + if (!source.hasSecondaryIndex(index.name)) + continue; + + if (has_alter_delete) { - const auto & index_cols = index.expression->getRequiredColumns(); - bool changed = std::any_of( - index_cols.begin(), - index_cols.end(), - [&](const auto & col) { return updated_columns.contains(col) || changed_columns.contains(col); }); - if (changed) - materialized_indices.insert(index.name); + materialized_indices.insert(index.name); + continue; } + + const auto & index_cols = index.expression->getRequiredColumns(); + bool changed = std::any_of( + index_cols.begin(), + index_cols.end(), + [&](const auto & col) { return updated_columns.contains(col) || changed_columns.contains(col); }); + + if (changed) + materialized_indices.insert(index.name); } for (const auto & projection : metadata_snapshot->getProjections()) { - if (source.hasIndexOrProjection(projection.getDirectoryName())) + if (!source.hasProjection(projection.name)) + continue; + + if (has_alter_delete) { - const auto & projection_cols = projection.required_columns; - bool changed = std::any_of( - projection_cols.begin(), - projection_cols.end(), - [&](const auto & col) { return updated_columns.contains(col) || changed_columns.contains(col); }); - if (changed) - materialized_projections.insert(projection.name); + materialized_projections.insert(projection.name); + continue; } + + const auto & projection_cols = projection.required_columns; + bool changed = std::any_of( + projection_cols.begin(), + projection_cols.end(), + [&](const auto & col) { return updated_columns.contains(col) || changed_columns.contains(col); }); + + if (changed) + materialized_projections.insert(projection.name); } /// Stages might be empty when we materialize skip indices or projections which don't add any diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index d783b503531..9b4caaae231 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -120,7 +120,8 @@ public: bool supportsLightweightDelete() const; bool hasLightweightDeleteMask() const; bool materializeTTLRecalculateOnly() const; - bool hasIndexOrProjection(const String & file_name) const; + bool hasSecondaryIndex(const String & name) const; + bool hasProjection(const String & name) const; void read( Stage & first_stage, diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 7050a98a4bc..1ab8dc7fb05 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1983,6 +1983,12 @@ IndexSize IMergeTreeDataPart::getSecondaryIndexSize(const String & secondary_ind return ColumnSize{}; } +bool IMergeTreeDataPart::hasSecondaryIndex(const String & index_name) const +{ + auto file_name = INDEX_FILE_PREFIX + index_name; + return checksums.has(file_name + ".idx") || checksums.has(file_name + ".idx2"); +} + void IMergeTreeDataPart::accumulateColumnSizes(ColumnToSize & column_to_size) const { for (const auto & [column_name, size] : columns_sizes) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index af6906e004d..bfb472ca50d 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -122,6 +122,9 @@ public: /// Otherwise return information about secondary index size on disk. IndexSize getSecondaryIndexSize(const String & secondary_index_name) const; + /// Returns true if there is materialized index with specified name in part. + bool hasSecondaryIndex(const String & index_name) const; + /// Return information about column size on disk for all columns in part ColumnSize getTotalColumnsSize() const { return total_columns_size; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 491c36433ca..fa5ad9858e8 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -69,6 +69,7 @@ static void splitAndModifyMutationCommands( { NameSet mutated_columns; NameSet dropped_columns; + NameSet extra_columns_for_indices_and_projections; for (const auto & command : commands) { @@ -85,6 +86,41 @@ static void splitAndModifyMutationCommands( if (command.type == MutationCommand::Type::MATERIALIZE_COLUMN) mutated_columns.emplace(command.column_name); + + if (command.type == MutationCommand::Type::MATERIALIZE_INDEX) + { + const auto & all_indices = metadata_snapshot->getSecondaryIndices(); + for (const auto & index : all_indices) + { + if (index.name == command.index_name) + { + auto required_columns = index.expression->getRequiredColumns(); + for (const auto & column : required_columns) + { + if (!part_columns.has(column)) + extra_columns_for_indices_and_projections.insert(column); + } + break; + } + } + } + + if (command.type == MutationCommand::Type::MATERIALIZE_PROJECTION) + { + const auto & all_projections = metadata_snapshot->getProjections(); + for (const auto & projection : all_projections) + { + if (projection.name == command.projection_name) + { + for (const auto & column : projection.required_columns) + { + if (!part_columns.has(column)) + extra_columns_for_indices_and_projections.insert(column); + } + break; + } + } + } } else if (command.type == MutationCommand::Type::DROP_INDEX || command.type == MutationCommand::Type::DROP_PROJECTION) { @@ -187,6 +223,25 @@ static void splitAndModifyMutationCommands( }); } } + + for (const auto & column_name : extra_columns_for_indices_and_projections) + { + if (mutated_columns.contains(column_name)) + continue; + + auto data_type = metadata_snapshot->getColumns().getColumn( + GetColumnsOptions::AllPhysical, + column_name).type; + + for_interpreter.push_back( + MutationCommand + { + .type = MutationCommand::Type::READ_COLUMN, + .column_name = column_name, + .data_type = std::move(data_type), + } + ); + } } else { @@ -453,6 +508,7 @@ static ExecuteTTLType shouldExecuteTTL(const StorageMetadataPtr & metadata_snaps /// Return set of indices which should be recalculated during mutation also /// wraps input stream into additional expression stream static std::set getIndicesToRecalculate( + const MergeTreeDataPartPtr & source_part, QueryPipelineBuilder & builder, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, @@ -463,10 +519,15 @@ static std::set getIndicesToRecalculate( std::set indices_to_recalc; ASTPtr indices_recalc_expr_list = std::make_shared(); const auto & indices = metadata_snapshot->getSecondaryIndices(); + bool is_full_part_storage = isFullPartStorage(source_part->getDataPartStorage()); for (const auto & index : indices) { - if (materialized_indices.contains(index.name)) + bool need_recalculate = + materialized_indices.contains(index.name) + || (!is_full_part_storage && source_part->hasSecondaryIndex(index.name)); + + if (need_recalculate) { if (indices_to_recalc.insert(index_factory.get(index)).second) { @@ -496,15 +557,23 @@ static std::set getIndicesToRecalculate( } static std::set getProjectionsToRecalculate( + const MergeTreeDataPartPtr & source_part, const StorageMetadataPtr & metadata_snapshot, const NameSet & materialized_projections) { std::set projections_to_recalc; + bool is_full_part_storage = isFullPartStorage(source_part->getDataPartStorage()); + for (const auto & projection : metadata_snapshot->getProjections()) { - if (materialized_projections.contains(projection.name)) + bool need_recalculate = + materialized_projections.contains(projection.name) + || (!is_full_part_storage && source_part->hasProjection(projection.name)); + + if (need_recalculate) projections_to_recalc.insert(&projection); } + return projections_to_recalc; } @@ -1279,14 +1348,20 @@ private: removed_indices.insert(command.column_name); } + bool is_full_part_storage = isFullPartStorage(ctx->new_data_part->getDataPartStorage()); const auto & indices = ctx->metadata_snapshot->getSecondaryIndices(); + MergeTreeIndices skip_indices; for (const auto & idx : indices) { if (removed_indices.contains(idx.name)) continue; - if (ctx->materialized_indices.contains(idx.name)) + bool need_recalculate = + ctx->materialized_indices.contains(idx.name) + || (!is_full_part_storage && ctx->source_part->hasSecondaryIndex(idx.name)); + + if (need_recalculate) { skip_indices.push_back(MergeTreeIndexFactory::instance().get(idx)); } @@ -1319,7 +1394,11 @@ private: if (removed_projections.contains(projection.name)) continue; - if (ctx->materialized_projections.contains(projection.name)) + bool need_recalculate = + ctx->materialized_projections.contains(projection.name) + || (!is_full_part_storage && ctx->source_part->hasProjection(projection.name)); + + if (need_recalculate) { ctx->projections_to_build.push_back(&projection); } @@ -1920,9 +1999,16 @@ bool MutateTask::prepare() else /// TODO: check that we modify only non-key columns in this case. { ctx->indices_to_recalc = MutationHelpers::getIndicesToRecalculate( - ctx->mutating_pipeline_builder, ctx->metadata_snapshot, ctx->context, ctx->materialized_indices); + ctx->source_part, + ctx->mutating_pipeline_builder, + ctx->metadata_snapshot, + ctx->context, + ctx->materialized_indices); - ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate(ctx->metadata_snapshot, ctx->materialized_projections); + ctx->projections_to_recalc = MutationHelpers::getProjectionsToRecalculate( + ctx->source_part, + ctx->metadata_snapshot, + ctx->materialized_projections); ctx->files_to_skip = MutationHelpers::collectFilesToSkip( ctx->source_part, diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index afe75349864..b1da509c9b0 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -239,7 +239,7 @@ bool StorageInMemoryMetadata::hasAnyGroupByTTL() const ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( const NameSet & updated_columns, bool include_ttl_target, - const std::function & has_indice_or_projection) const + const HasDependencyCallback & has_dependency) const { if (updated_columns.empty()) return {}; @@ -268,13 +268,13 @@ ColumnDependencies StorageInMemoryMetadata::getColumnDependencies( for (const auto & index : getSecondaryIndices()) { - if (has_indice_or_projection("skp_idx_" + index.name + ".idx") || has_indice_or_projection("skp_idx_" + index.name + ".idx2")) + if (has_dependency(index.name, ColumnDependency::SKIP_INDEX)) add_dependent_columns(index.expression, indices_columns); } for (const auto & projection : getProjections()) { - if (has_indice_or_projection(projection.getDirectoryName())) + if (has_dependency(projection.getDirectoryName(), ColumnDependency::PROJECTION)) add_dependent_columns(&projection, projections_columns); } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 4ed7eb8bf29..30b2b303492 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -147,12 +147,14 @@ struct StorageInMemoryMetadata TTLDescriptions getGroupByTTLs() const; bool hasAnyGroupByTTL() const; + using HasDependencyCallback = std::function; + /// Returns columns, which will be needed to calculate dependencies (skip indices, projections, /// TTL expressions) if we update @updated_columns set of columns. ColumnDependencies getColumnDependencies( const NameSet & updated_columns, bool include_ttl_target, - const std::function & has_indice_or_projection) const; + const HasDependencyCallback & has_dependency) const; /// Block with ordinary + materialized columns. Block getSampleBlock() const; diff --git a/tests/queries/0_stateless/02832_alter_delete_indexes_projections.reference b/tests/queries/0_stateless/02832_alter_delete_indexes_projections.reference new file mode 100644 index 00000000000..f14acdf9e6d --- /dev/null +++ b/tests/queries/0_stateless/02832_alter_delete_indexes_projections.reference @@ -0,0 +1,6 @@ +2 +0 +3355402240 +3355402240 +3321851904 +3321851904 diff --git a/tests/queries/0_stateless/02832_alter_delete_indexes_projections.sql b/tests/queries/0_stateless/02832_alter_delete_indexes_projections.sql new file mode 100644 index 00000000000..b87230e57d1 --- /dev/null +++ b/tests/queries/0_stateless/02832_alter_delete_indexes_projections.sql @@ -0,0 +1,26 @@ +set mutations_sync = 2; + +drop table if exists t_delete_skip_index; + +create table t_delete_skip_index (x UInt32, y String, index i y type minmax granularity 3) engine = MergeTree order by tuple(); +insert into t_delete_skip_index select number, toString(number) from numbers(8192 * 10); + +select count() from t_delete_skip_index where y in (4, 5); +alter table t_delete_skip_index delete where x < 8192; +select count() from t_delete_skip_index where y in (4, 5); + +drop table if exists t_delete_skip_index; +drop table if exists t_delete_projection; + +create table t_delete_projection (x UInt32, y UInt64, projection p (select sum(y))) engine = MergeTree order by tuple(); +insert into t_delete_projection select number, toString(number) from numbers(8192 * 10); + +select sum(y) from t_delete_projection settings optimize_use_projections = 0; +select sum(y) from t_delete_projection settings optimize_use_projections = 0, force_optimize_projection = 1; + +alter table t_delete_projection delete where x < 8192; + +select sum(y) from t_delete_projection settings optimize_use_projections = 0; +select sum(y) from t_delete_projection settings optimize_use_projections = 0, force_optimize_projection = 1; + +drop table if exists t_delete_projection; From 393481181d5fb7c878af2f1e0bbbd5d9c8d56807 Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 24 Jul 2023 13:35:07 -0400 Subject: [PATCH 042/777] Remove Trailing whitespaces --- src/Parsers/Access/ASTCreateRowPolicyQuery.cpp | 2 +- src/Parsers/Access/ASTCreateUserQuery.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp index a423581e300..e95959703ee 100644 --- a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp @@ -172,7 +172,7 @@ void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, Format settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " IN " << (settings.hilite ? IAST::hilite_none : "") << backQuoteIfNeed(storage_name); - + formatOnCluster(settings); assert(names->cluster.empty()); diff --git a/src/Parsers/Access/ASTCreateUserQuery.cpp b/src/Parsers/Access/ASTCreateUserQuery.cpp index dbb4c1c6d57..b7290c16013 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.cpp +++ b/src/Parsers/Access/ASTCreateUserQuery.cpp @@ -212,7 +212,7 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState & format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " IN " << (format.hilite ? IAST::hilite_none : "") << backQuoteIfNeed(storage_name); - + formatOnCluster(format); if (new_name) From a5d7391fbf953d6395ecc5adba48a59e1c248811 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 25 Jul 2023 15:34:38 +0800 Subject: [PATCH 043/777] add new function startsWithUTF8 and endsWithUTF8 --- .../functions/string-functions.md | 28 +++++++ src/Functions/FunctionStartsEndsWith.h | 77 ++++++++++++++++--- src/Functions/endsWithUTF8.cpp | 16 ++++ src/Functions/startsWithUTF8.cpp | 16 ++++ .../02833_starts_ends_with_utf8.reference | 29 +++++++ .../02833_starts_ends_with_utf8.sql | 19 +++++ 6 files changed, 175 insertions(+), 10 deletions(-) create mode 100644 src/Functions/endsWithUTF8.cpp create mode 100644 src/Functions/startsWithUTF8.cpp create mode 100644 tests/queries/0_stateless/02833_starts_ends_with_utf8.reference create mode 100644 tests/queries/0_stateless/02833_starts_ends_with_utf8.sql diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 9890d257e84..a398a7c0993 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -693,6 +693,22 @@ Returns whether string `str` ends with `suffix`. endsWith(str, suffix) ``` +## endsWithUTF8 + +Returns whether string `str` ends with `suffix`, assuming that both strings contain svalid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + +**Syntax** + +```sql +endsWithUTF8(str, suffix) +``` + +**Example** + +``` sql +SELECT endsWithUTF8('中国', '国'); +``` + ## startsWith Returns whether string `str` starts with `prefix`. @@ -709,6 +725,18 @@ startsWith(str, prefix) SELECT startsWith('Spider-Man', 'Spi'); ``` +## startsWithUTF8 + +Returns whether string `str` starts with `prefix`, assuming that both string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. + + +**Example** + +``` sql +SELECT startsWithUTF8('中国', '中'); +``` + + ## trim Removes the specified characters from the start or end of a string. If not specified otherwise, the function removes whitespace (ASCII-character 32). diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index b15bab47ae0..3565af326fe 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -28,10 +28,23 @@ namespace ErrorCodes struct NameStartsWith { static constexpr auto name = "startsWith"; + static constexpr auto is_utf8 = false; }; struct NameEndsWith { static constexpr auto name = "endsWith"; + static constexpr auto is_utf8 = false; +}; + +struct NameStartsWithUTF8 +{ + static constexpr auto name = "startsWithUTF8"; + static constexpr auto is_utf8 = true; +}; +struct NameEndsWithUTF8 +{ + static constexpr auto name = "endsWithUTF8"; + static constexpr auto is_utf8 = true; }; DECLARE_MULTITARGET_CODE( @@ -41,6 +54,7 @@ class FunctionStartsEndsWith : public IFunction { public: static constexpr auto name = Name::name; + static constexpr auto is_utf8 = Name::is_utf8; String getName() const override { @@ -64,7 +78,8 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (isStringOrFixedString(arguments[0]) && isStringOrFixedString(arguments[1])) + if (!is_utf8 && isStringOrFixedString(arguments[0]) && isStringOrFixedString(arguments[1]) + || isString(arguments[0]) && isString(arguments[1])) return std::make_shared(); if (isArray(arguments[0]) && isArray(arguments[1])) @@ -78,8 +93,11 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { auto data_type = arguments[0].type; - if (isStringOrFixedString(*data_type)) + + if (!is_utf8 && isStringOrFixedString(*data_type)) return executeImplString(arguments, {}, input_rows_count); + if (is_utf8 && isString(*data_type)) + return executeImplStringUTF8(arguments, {}, input_rows_count); if (isArray(data_type)) return executeImplArray(arguments, {}, input_rows_count); return {}; @@ -131,7 +149,6 @@ private: typename ColumnVector::Container & vec_res = col_res->getData(); vec_res.resize(input_rows_count); - if (const ColumnString * haystack = checkAndGetColumn(haystack_column)) dispatch(StringSource(*haystack), needle_column, vec_res); else if (const ColumnFixedString * haystack_fixed = checkAndGetColumn(haystack_column)) @@ -146,6 +163,26 @@ private: return col_res; } + ColumnPtr executeImplStringUTF8(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const + { + const IColumn * haystack_column = arguments[0].column.get(); + const IColumn * needle_column = arguments[1].column.get(); + + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_res = col_res->getData(); + + vec_res.resize(input_rows_count); + if (const ColumnString * haystack = checkAndGetColumn(haystack_column)) + dispatchUTF8(UTF8StringSource(*haystack), needle_column, vec_res); + else if (const ColumnConst * haystack_const = checkAndGetColumnConst(haystack_column)) + dispatchUTF8>(ConstSource(*haystack_const), needle_column, vec_res); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal combination of columns as arguments of function {}", getName()); + + return col_res; + } + + template void dispatch(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray & res_data) const { @@ -161,6 +198,17 @@ private: throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal combination of columns as arguments of function {}", getName()); } + template + void dispatchUTF8(HaystackSource haystack_source, const IColumn * needle_column, PaddedPODArray & res_data) const + { + if (const ColumnString * needle = checkAndGetColumn(needle_column)) + execute(haystack_source, UTF8StringSource(*needle), res_data); + else if (const ColumnConst * needle_const = checkAndGetColumnConst(needle_column)) + execute>(haystack_source, ConstSource(*needle_const), res_data); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal combination of columns as arguments of function {}", getName()); + } + template static void execute(HaystackSource haystack_source, NeedleSource needle_source, PaddedPODArray & res_data) { @@ -172,18 +220,27 @@ private: auto needle = needle_source.getWhole(); if (needle.size > haystack.size) - { res_data[row_num] = false; - } else { - if constexpr (std::is_same_v) - { + if constexpr (std::is_same_v) /// startsWith res_data[row_num] = StringRef(haystack.data, needle.size) == StringRef(needle.data, needle.size); - } - else /// endsWith - { + else if constexpr (std::is_same_v) /// endsWith res_data[row_num] = StringRef(haystack.data + haystack.size - needle.size, needle.size) == StringRef(needle.data, needle.size); + else /// startsWithUTF8 or endsWithUTF8 + { + auto length = UTF8::countCodePoints(needle.data, needle.size); + + if constexpr (std::is_same_v) + { + auto slice = haystack_source.getSliceFromLeft(0, length); + res_data[row_num] = StringRef(slice.data, slice.size) == StringRef(needle.data, needle.size); + } + else + { + auto slice = haystack_source.getSliceFromRight(length); + res_data[row_num] = StringRef(slice.data, slice.size) == StringRef(needle.data, needle.size); + } } } diff --git a/src/Functions/endsWithUTF8.cpp b/src/Functions/endsWithUTF8.cpp new file mode 100644 index 00000000000..377683ce47c --- /dev/null +++ b/src/Functions/endsWithUTF8.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + + +namespace DB +{ + +using FunctionEndsWithUTF8 = FunctionStartsEndsWith; + +REGISTER_FUNCTION(EndsWithUTF8) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/startsWithUTF8.cpp b/src/Functions/startsWithUTF8.cpp new file mode 100644 index 00000000000..e623a55f4dc --- /dev/null +++ b/src/Functions/startsWithUTF8.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + + +namespace DB +{ + +using FunctionStartsWithUTF8 = FunctionStartsEndsWith; + +REGISTER_FUNCTION(StartsWithUTF8) +{ + factory.registerFunction(); +} + +} diff --git a/tests/queries/0_stateless/02833_starts_ends_with_utf8.reference b/tests/queries/0_stateless/02833_starts_ends_with_utf8.reference new file mode 100644 index 00000000000..ca2a5bc50f8 --- /dev/null +++ b/tests/queries/0_stateless/02833_starts_ends_with_utf8.reference @@ -0,0 +1,29 @@ +-- { echoOn } +select startsWithUTF8('富强民主文明和谐', '富强'); +1 +select startsWithUTF8('富强民主文明和谐', '\xe5'); +0 +select startsWithUTF8('富强民主文明和谐', ''); +1 +SELECT startsWithUTF8('123', '123'); +1 +SELECT startsWithUTF8('123', '12'); +1 +SELECT startsWithUTF8('123', '1234'); +0 +SELECT startsWithUTF8('123', ''); +1 +select endsWithUTF8('富强民主文明和谐', '和谐'); +1 +select endsWithUTF8('富强民主文明和谐', '\x90'); +0 +select endsWithUTF8('富强民主文明和谐', ''); +1 +SELECT endsWithUTF8('123', '3'); +1 +SELECT endsWithUTF8('123', '23'); +1 +SELECT endsWithUTF8('123', '32'); +0 +SELECT endsWithUTF8('123', ''); +1 diff --git a/tests/queries/0_stateless/02833_starts_ends_with_utf8.sql b/tests/queries/0_stateless/02833_starts_ends_with_utf8.sql new file mode 100644 index 00000000000..3a783dc280e --- /dev/null +++ b/tests/queries/0_stateless/02833_starts_ends_with_utf8.sql @@ -0,0 +1,19 @@ +-- { echoOn } +select startsWithUTF8('富强民主文明和谐', '富强'); +select startsWithUTF8('富强民主文明和谐', '\xe5'); +select startsWithUTF8('富强民主文明和谐', ''); + +SELECT startsWithUTF8('123', '123'); +SELECT startsWithUTF8('123', '12'); +SELECT startsWithUTF8('123', '1234'); +SELECT startsWithUTF8('123', ''); + +select endsWithUTF8('富强民主文明和谐', '和谐'); +select endsWithUTF8('富强民主文明和谐', '\x90'); +select endsWithUTF8('富强民主文明和谐', ''); + +SELECT endsWithUTF8('123', '3'); +SELECT endsWithUTF8('123', '23'); +SELECT endsWithUTF8('123', '32'); +SELECT endsWithUTF8('123', ''); +-- { echoOff } From acb33c1d810982291e7f208340dfba50d7f6c327 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 25 Jul 2023 15:39:49 +0800 Subject: [PATCH 044/777] fix code style --- src/Functions/FunctionStartsEndsWith.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/FunctionStartsEndsWith.h b/src/Functions/FunctionStartsEndsWith.h index 3565af326fe..7671129fcfc 100644 --- a/src/Functions/FunctionStartsEndsWith.h +++ b/src/Functions/FunctionStartsEndsWith.h @@ -41,6 +41,7 @@ struct NameStartsWithUTF8 static constexpr auto name = "startsWithUTF8"; static constexpr auto is_utf8 = true; }; + struct NameEndsWithUTF8 { static constexpr auto name = "endsWithUTF8"; From 23897b823d7a98c122e385ed9452f8e930132825 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 25 Jul 2023 20:41:16 +0800 Subject: [PATCH 045/777] fix code style --- docs/en/sql-reference/functions/string-functions.md | 2 +- .../02415_all_new_functions_must_be_documented.reference | 2 ++ utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index a398a7c0993..f36de0dc0d8 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -695,7 +695,7 @@ endsWith(str, suffix) ## endsWithUTF8 -Returns whether string `str` ends with `suffix`, assuming that both strings contain svalid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. +Returns whether string `str` ends with `suffix`, assuming that both strings contain valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. **Syntax** diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 247df64f741..d70214e9930 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -263,6 +263,7 @@ encodeURLComponent encodeURLFormComponent encodeXMLComponent endsWith +endsWithUTF8 equals erf erfc @@ -665,6 +666,7 @@ splitByString splitByWhitespace sqrt startsWith +startsWithUTF8 subBitmap substring substringUTF8 diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 65271830555..3cc25108eb1 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1394,6 +1394,7 @@ encodings encryptions endian endsWith +endsWithUTF8 enum enum's enums @@ -2200,6 +2201,7 @@ src stacktrace stacktraces startsWith +startsWithUTF8 statbox stateful stddev From 5ab6b34e1191c97d3a590f83345015e059ad3edb Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Tue, 25 Jul 2023 13:27:56 -0700 Subject: [PATCH 046/777] Fix codec delta endian issue for s390x --- src/Compression/CompressionCodecDelta.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Compression/CompressionCodecDelta.cpp b/src/Compression/CompressionCodecDelta.cpp index 37f9230da14..90d3197b374 100644 --- a/src/Compression/CompressionCodecDelta.cpp +++ b/src/Compression/CompressionCodecDelta.cpp @@ -73,8 +73,8 @@ void compressDataForType(const char * source, UInt32 source_size, char * dest) const char * const source_end = source + source_size; while (source < source_end) { - T curr_src = unalignedLoad(source); - unalignedStore(dest, curr_src - prev_src); + T curr_src = unalignedLoadLittleEndian(source); + unalignedStoreLittleEndian(dest, curr_src - prev_src); prev_src = curr_src; source += sizeof(T); @@ -94,10 +94,10 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest, const char * const source_end = source + source_size; while (source < source_end) { - accumulator += unalignedLoad(source); + accumulator += unalignedLoadLittleEndian(source); if (dest + sizeof(accumulator) > output_end) [[unlikely]] throw Exception(ErrorCodes::CANNOT_DECOMPRESS, "Cannot decompress the data"); - unalignedStore(dest, accumulator); + unalignedStoreLittleEndian(dest, accumulator); source += sizeof(T); dest += sizeof(T); From 27f3c5cfa27f9aef5df563a9fd90089963afe26c Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 26 Jul 2023 10:53:07 +0800 Subject: [PATCH 047/777] change as requested --- docs/en/sql-reference/functions/string-functions.md | 4 ++-- src/Functions/endsWithUTF8.cpp | 7 ++++++- src/Functions/startsWithUTF8.cpp | 7 ++++++- .../02415_all_new_functions_must_be_documented.reference | 2 -- 4 files changed, 14 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index f36de0dc0d8..b1ffb32e7cd 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -695,7 +695,7 @@ endsWith(str, suffix) ## endsWithUTF8 -Returns whether string `str` ends with `suffix`, assuming that both strings contain valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. +Returns whether string `str` ends with `suffix`, the difference between `endsWithUTF8` and `endsWith` is that `endsWithUTF8` match `str` and `suffix` by UTF-8 characters. **Syntax** @@ -727,7 +727,7 @@ SELECT startsWith('Spider-Man', 'Spi'); ## startsWithUTF8 -Returns whether string `str` starts with `prefix`, assuming that both string contains valid UTF-8 encoded text. If this assumption is violated, no exception is thrown and the result is undefined. +Returns whether string `str` starts with `prefix`, the difference between `startsWithUTF8` and `startsWith` is that `startsWithUTF8` match `str` and `suffix` by UTF-8 characters. **Example** diff --git a/src/Functions/endsWithUTF8.cpp b/src/Functions/endsWithUTF8.cpp index 377683ce47c..1b042452298 100644 --- a/src/Functions/endsWithUTF8.cpp +++ b/src/Functions/endsWithUTF8.cpp @@ -10,7 +10,12 @@ using FunctionEndsWithUTF8 = FunctionStartsEndsWith; REGISTER_FUNCTION(EndsWithUTF8) { - factory.registerFunction(); + factory.registerFunction(FunctionDocumentation{ + .description = R"( +Returns whether string `str` ends with `suffix`, the difference between `endsWithUTF8` and `endsWith` is that `endsWithUTF8` match `str` and `suffix` by UTF-8 characters. + )", + .examples{{"endsWithUTF8", "select endsWithUTF8('富强民主文明和谐', '富强');", ""}}, + .categories{"String"}}); } } diff --git a/src/Functions/startsWithUTF8.cpp b/src/Functions/startsWithUTF8.cpp index e623a55f4dc..3f46916d760 100644 --- a/src/Functions/startsWithUTF8.cpp +++ b/src/Functions/startsWithUTF8.cpp @@ -10,7 +10,12 @@ using FunctionStartsWithUTF8 = FunctionStartsEndsWith; REGISTER_FUNCTION(StartsWithUTF8) { - factory.registerFunction(); + factory.registerFunction(FunctionDocumentation{ + .description = R"( +Returns whether string `str` starts with `prefix`, the difference between `startsWithUTF8` and `startsWith` is that `startsWithUTF8` match `str` and `suffix` by UTF-8 characters. + )", + .examples{{"startsWithUTF8", "select startsWithUTF8('富强民主文明和谐', '富强');", ""}}, + .categories{"String"}}); } } diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index d70214e9930..247df64f741 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -263,7 +263,6 @@ encodeURLComponent encodeURLFormComponent encodeXMLComponent endsWith -endsWithUTF8 equals erf erfc @@ -666,7 +665,6 @@ splitByString splitByWhitespace sqrt startsWith -startsWithUTF8 subBitmap substring substringUTF8 From 21e12ea0842578043fd4686e16c935052480bb80 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 26 Jul 2023 00:53:45 -0400 Subject: [PATCH 048/777] Fix tests --- src/Access/IAccessStorage.cpp | 2 +- tests/integration/test_role/test.py | 2 +- tests/queries/0_stateless/01292_create_user.reference | 8 ++++---- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 11bc97f6cc3..db73db900aa 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -180,7 +180,7 @@ UUID IAccessStorage::insert(const AccessEntityPtr & entity) std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { - return *insert(entity, replace_if_exists, throw_if_exists, /* set_id = */ std::nullopt); + return insert(entity, replace_if_exists, throw_if_exists, /* set_id = */ std::nullopt); } std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index 43a402d9fda..0e1bbea9cea 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -301,7 +301,7 @@ def test_introspection(): ) assert instance.query( - "SELECT * from system.role_grants WHERE user_name IN ('A', 'B') OR role_name IN ('R1', 'R2') ORDER BY user_name, role_name, granted_role_name" + "SELECT user_name, role_name, granted_role_name, granted_role_is_default, with_admin_option from system.role_grants WHERE user_name IN ('A', 'B') OR role_name IN ('R1', 'R2') ORDER BY user_name, role_name, granted_role_name" ) == TSV([["A", "\\N", "R1", 1, 0], ["B", "\\N", "R2", 1, 1]]) assert instance.query( diff --git a/tests/queries/0_stateless/01292_create_user.reference b/tests/queries/0_stateless/01292_create_user.reference index eb89a5ed38c..b7c30b304bf 100644 --- a/tests/queries/0_stateless/01292_create_user.reference +++ b/tests/queries/0_stateless/01292_create_user.reference @@ -106,10 +106,10 @@ CREATE USER u2_01292 DEFAULT ROLE r1_01292, r2_01292 SETTINGS readonly = 1 CREATE USER u3_01292 HOST LIKE \'%.%.myhost.com\' DEFAULT ROLE r1_01292, r2_01292 CREATE USER u4_01292 HOST LIKE \'%.%.myhost.com\' DEFAULT ROLE r1_01292, r2_01292 -- system.users -u1_01292 local directory plaintext_password {} [] ['localhost'] [] [] 1 [] [] -u2_01292 local directory no_password {} [] [] [] ['%.%.myhost.com'] 0 [] [] -u3_01292 local directory sha256_password {} ['192.169.1.1','192.168.0.0/16'] ['localhost'] [] [] 0 ['r1_01292'] [] -u4_01292 local directory double_sha1_password {} ['::/0'] [] [] [] 1 [] ['r1_01292'] +u1_01292 local_directory plaintext_password {} [] ['localhost'] [] [] 1 [] [] +u2_01292 local_directory no_password {} [] [] [] ['%.%.myhost.com'] 0 [] [] +u3_01292 local_directory sha256_password {} ['192.169.1.1','192.168.0.0/16'] ['localhost'] [] [] 0 ['r1_01292'] [] +u4_01292 local_directory double_sha1_password {} ['::/0'] [] [] [] 1 [] ['r1_01292'] -- system.settings_profile_elements \N u1_01292 \N 0 readonly 1 \N \N \N \N \N u2_01292 \N 0 \N \N \N \N \N default From d83eabcf772d942c18c8d8b2dccb43aaad1bf235 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 26 Jul 2023 17:39:45 +0800 Subject: [PATCH 049/777] fix spell --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 3cc25108eb1..240238cab12 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1394,7 +1394,7 @@ encodings encryptions endian endsWith -endsWithUTF8 +endsWithUTF enum enum's enums @@ -2201,7 +2201,7 @@ src stacktrace stacktraces startsWith -startsWithUTF8 +startsWithUTF statbox stateful stddev From 13d9952227162b63ae5e4ae98b0a40677c169b97 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 27 Jul 2023 00:38:51 -0400 Subject: [PATCH 050/777] add id to insertImpl --- src/Access/AccessControl.cpp | 10 +++++--- src/Access/AccessControl.h | 2 +- src/Access/DiskAccessStorage.cpp | 16 +++--------- src/Access/DiskAccessStorage.h | 3 +-- src/Access/IAccessStorage.cpp | 25 ++++++++++++------- src/Access/IAccessStorage.h | 11 ++++++-- src/Access/MemoryAccessStorage.cpp | 14 ++--------- src/Access/MemoryAccessStorage.h | 7 +----- src/Access/MultipleAccessStorage.cpp | 11 ++++---- src/Access/MultipleAccessStorage.h | 2 +- src/Access/ReplicatedAccessStorage.cpp | 16 +++--------- src/Access/ReplicatedAccessStorage.h | 3 +-- src/Parsers/Access/ParserCreateQuotaQuery.cpp | 4 +-- src/Parsers/Access/ParserCreateRoleQuery.cpp | 3 ++- .../Access/ParserCreateRowPolicyQuery.cpp | 4 +-- .../ParserCreateSettingsProfileQuery.cpp | 4 +-- src/Parsers/Access/ParserCreateUserQuery.cpp | 3 ++- .../Access/ParserDropAccessEntityQuery.cpp | 3 ++- .../Access/ParserMoveAccessEntityQuery.cpp | 3 ++- src/Parsers/Access/parseUserName.h | 5 ---- 20 files changed, 64 insertions(+), 85 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 22b00d5dde7..3759b6c8919 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -525,12 +525,14 @@ scope_guard AccessControl::subscribeForChanges(const std::vector & ids, co return changes_notifier->subscribeForChanges(ids, handler); } -std::optional AccessControl::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) +bool AccessControl::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { - auto id = MultipleAccessStorage::insertImpl(entity, replace_if_exists, throw_if_exists, set_id); - if (id) + if (MultipleAccessStorage::insertImpl(id, entity, replace_if_exists, throw_if_exists)) + { changes_notifier->sendNotifications(); - return id; + return true; + } + return false; } bool AccessControl::removeImpl(const UUID & id, bool throw_if_not_exists) diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index 85feaa70462..60b40775a56 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -229,7 +229,7 @@ private: class CustomSettingsPrefixes; class PasswordComplexityRules; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index acbd7b12569..190c7567b85 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -498,20 +498,10 @@ std::optional> DiskAccessStorage::readNameWi } -std::optional DiskAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) -{ - UUID id = set_id ? *set_id : generateRandomID(); - if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk= */ true)) - return id; - - return std::nullopt; -} - - -bool DiskAccessStorage::insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk) +bool DiskAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) { std::lock_guard lock{mutex}; - return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, write_on_disk); + return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists, /* write_on_disk = */ true); } @@ -745,7 +735,7 @@ void DiskAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] { for (const auto & [id, entity] : my_entities) - insertWithID(id, entity, replace_if_exists, throw_if_exists, /* write_on_disk= */ true); + insert(id, entity, replace_if_exists, throw_if_exists); }); } diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index 3541c3228e3..5d94008b34f 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -39,7 +39,7 @@ private: std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; @@ -53,7 +53,6 @@ private: void listsWritingThreadFunc() TSA_NO_THREAD_SAFETY_ANALYSIS; void stopListsWritingThread(); - bool insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk); bool insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, bool write_on_disk) TSA_REQUIRES(mutex); bool updateNoLock(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex); bool removeNoLock(const UUID & id, bool throw_if_not_exists, bool write_on_disk) TSA_REQUIRES(mutex); diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index db73db900aa..468c93c1116 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -180,14 +180,21 @@ UUID IAccessStorage::insert(const AccessEntityPtr & entity) std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { - return insert(entity, replace_if_exists, throw_if_exists, /* set_id = */ std::nullopt); + auto id = generateRandomID(); + + if (insert(id, entity, replace_if_exists, throw_if_exists)) + return id; + + return std::nullopt; } -std::optional IAccessStorage::insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) + +bool IAccessStorage::insert(const DB::UUID & id, const DB::AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { - return insertImpl(entity, replace_if_exists, throw_if_exists, set_id); + return insertImpl(id, entity, replace_if_exists, throw_if_exists); } + std::vector IAccessStorage::insert(const std::vector & multiple_entities, bool replace_if_exists, bool throw_if_exists) { return insert(multiple_entities, /* ids = */ {}, replace_if_exists, throw_if_exists); @@ -216,16 +223,16 @@ std::vector IAccessStorage::insert(const std::vector & mu { const auto & entity = multiple_entities[i]; - std::optional id; + UUID id; if (!ids.empty()) id = ids[i]; + else + id = generateRandomID(); - auto new_id = insertImpl(entity, replace_if_exists, throw_if_exists, id); - - if (new_id) + if (insertImpl(id, entity, replace_if_exists, throw_if_exists)) { successfully_inserted.push_back(entity); - new_ids.push_back(*new_id); + new_ids.push_back(id); } } return new_ids; @@ -274,7 +281,7 @@ std::vector IAccessStorage::insertOrReplace(const std::vector IAccessStorage::insertImpl(const AccessEntityPtr & entity, bool, bool, std::optional) +bool IAccessStorage::insertImpl(const UUID &, const AccessEntityPtr & entity, bool, bool) { if (isReadOnly()) throwReadonlyCannotInsert(entity->getType(), entity->getName()); diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index 894220e95ad..b82537df639 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -132,7 +134,7 @@ public: /// Throws an exception if the specified name already exists. UUID insert(const AccessEntityPtr & entity); std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); - std::optional insert(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id); + bool insert(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); std::vector insert(const std::vector & multiple_entities, bool replace_if_exists = false, bool throw_if_exists = true); std::vector insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists = false, bool throw_if_exists = true); @@ -185,7 +187,7 @@ protected: virtual std::vector findAllImpl(AccessEntityType type) const = 0; virtual AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const = 0; virtual std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const; - virtual std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id); + virtual bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); virtual bool removeImpl(const UUID & id, bool throw_if_not_exists); virtual bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); virtual std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const; @@ -284,4 +286,9 @@ std::vector>> IAccessStorage return entities; } +inline bool parseAccessStorageName(IParser::Pos & pos, Expected & expected, String & storage_name) +{ + return parseIdentifierOrStringLiteral(pos, expected, storage_name); +} + } diff --git a/src/Access/MemoryAccessStorage.cpp b/src/Access/MemoryAccessStorage.cpp index 16f3b3793a0..999e2c41d0b 100644 --- a/src/Access/MemoryAccessStorage.cpp +++ b/src/Access/MemoryAccessStorage.cpp @@ -63,17 +63,7 @@ AccessEntityPtr MemoryAccessStorage::readImpl(const UUID & id, bool throw_if_not } -std::optional MemoryAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) -{ - UUID id = set_id ? *set_id : generateRandomID(); - if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists)) - return id; - - return std::nullopt; -} - - -bool MemoryAccessStorage::insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +bool MemoryAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) { std::lock_guard lock{mutex}; return insertNoLock(id, new_entity, replace_if_exists, throw_if_exists); @@ -300,7 +290,7 @@ void MemoryAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] { for (const auto & [id, entity] : my_entities) - insertWithID(id, entity, replace_if_exists, throw_if_exists); + insert(id, entity, replace_if_exists, throw_if_exists); }); } diff --git a/src/Access/MemoryAccessStorage.h b/src/Access/MemoryAccessStorage.h index 7c19ab9c4ab..f877860041a 100644 --- a/src/Access/MemoryAccessStorage.h +++ b/src/Access/MemoryAccessStorage.h @@ -22,11 +22,6 @@ public: const char * getStorageType() const override { return STORAGE_TYPE; } - /// Inserts an entity with a specified ID. - /// If `replace_if_exists == true` it can replace an existing entry with such ID and also remove an existing entry - /// with such name & type. - bool insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists); - /// Removes all entities except the specified list `ids_to_keep`. /// The function skips IDs not contained in the storage. void removeAllExcept(const std::vector & ids_to_keep); @@ -44,7 +39,7 @@ private: std::optional findImpl(AccessEntityType type, const String & name) const override; std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index 169dc889509..e6665349ae6 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -316,7 +316,7 @@ void MultipleAccessStorage::reload(ReloadMode reload_mode) } -std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) +bool MultipleAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) { std::shared_ptr storage_for_insertion; @@ -339,13 +339,14 @@ std::optional MultipleAccessStorage::insertImpl(const AccessEntityPtr & en getStorageName()); } - auto id = storage_for_insertion->insert(entity, replace_if_exists, throw_if_exists, set_id); - if (id) + if (storage_for_insertion->insert(id, entity, replace_if_exists, throw_if_exists)) { std::lock_guard lock{mutex}; - ids_cache.set(*id, storage_for_insertion); + ids_cache.set(id, storage_for_insertion); + return true; } - return id; + + return false; } diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index 1d95296b796..069d414f601 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -63,7 +63,7 @@ protected: std::vector findAllImpl(AccessEntityType type) const override; AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override; std::optional> readNameWithTypeImpl(const UUID & id, bool throw_if_not_exists) const override; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; std::optional authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override; diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index efa9dbbdfaa..275672da115 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -108,17 +108,7 @@ static void retryOnZooKeeperUserError(size_t attempts, Func && function) } } -std::optional ReplicatedAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) -{ - const UUID id = set_id ? *set_id : generateRandomID(); - if (insertWithID(id, new_entity, replace_if_exists, throw_if_exists)) - return id; - - return std::nullopt; -} - - -bool ReplicatedAccessStorage::insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) +bool ReplicatedAccessStorage::insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) { const AccessEntityTypeInfo type_info = AccessEntityTypeInfo::get(new_entity->getType()); const String & name = new_entity->getName(); @@ -619,7 +609,7 @@ AccessEntityPtr ReplicatedAccessStorage::tryReadEntityFromZooKeeper(const zkutil void ReplicatedAccessStorage::setEntityNoLock(const UUID & id, const AccessEntityPtr & entity) { LOG_DEBUG(getLogger(), "Setting id {} to entity named {}", toString(id), entity->getName()); - memory_storage.insertWithID(id, entity, /* replace_if_exists= */ true, /* throw_if_exists= */ false); + memory_storage.insert(id, entity, /* replace_if_exists= */ true, /* throw_if_exists= */ false); } @@ -711,7 +701,7 @@ void ReplicatedAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) restorer.addDataRestoreTask([this, my_entities = std::move(entities), replace_if_exists, throw_if_exists] { for (const auto & [id, entity] : my_entities) - insertWithID(id, entity, replace_if_exists, throw_if_exists); + insert(id, entity, replace_if_exists, throw_if_exists); }); } diff --git a/src/Access/ReplicatedAccessStorage.h b/src/Access/ReplicatedAccessStorage.h index 0a0c53d3bc4..cddb20860f7 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -46,11 +46,10 @@ private: std::unique_ptr watching_thread; std::shared_ptr> watched_queue; - std::optional insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists, std::optional set_id) override; + bool insertImpl(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override; - bool insertWithID(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, bool throw_if_exists); bool insertZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists); bool removeZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, bool throw_if_not_exists); bool updateZooKeeper(const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists); diff --git a/src/Parsers/Access/ParserCreateQuotaQuery.cpp b/src/Parsers/Access/ParserCreateQuotaQuery.cpp index 05c2820b012..718498ed1d1 100644 --- a/src/Parsers/Access/ParserCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ParserCreateQuotaQuery.cpp @@ -1,11 +1,11 @@ #include +#include #include #include #include #include #include #include -#include #include #include #include @@ -312,7 +312,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; - if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; break; diff --git a/src/Parsers/Access/ParserCreateRoleQuery.cpp b/src/Parsers/Access/ParserCreateRoleQuery.cpp index b548c07ec70..99a97f6901d 100644 --- a/src/Parsers/Access/ParserCreateRoleQuery.cpp +++ b/src/Parsers/Access/ParserCreateRoleQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -111,7 +112,7 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; - if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; break; diff --git a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp index b2177ca4ade..f9e9466e35d 100644 --- a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp @@ -1,10 +1,10 @@ +#include #include #include #include #include #include #include -#include #include #include #include @@ -273,7 +273,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; - if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; break; diff --git a/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp b/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp index 634ff68427a..fcb6ebd8806 100644 --- a/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp +++ b/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp @@ -1,10 +1,10 @@ +#include #include #include #include #include #include #include -#include #include #include #include @@ -132,7 +132,7 @@ bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expec if (cluster.empty() && parseOnCluster(pos, expected, cluster)) continue; - if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; break; diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index d95d41296dc..577b6772ab2 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -481,7 +482,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec } } - if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseStorageName(pos, expected, storage_name)) + if (storage_name.empty() && ParserKeyword{"IN"}.ignore(pos, expected) && parseAccessStorageName(pos, expected, storage_name)) continue; break; diff --git a/src/Parsers/Access/ParserDropAccessEntityQuery.cpp b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp index d5ecdf27449..7fd34a6f8e4 100644 --- a/src/Parsers/Access/ParserDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -78,7 +79,7 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & } if (ParserKeyword{"FROM"}.ignore(pos, expected)) - parseStorageName(pos, expected, storage_name); + parseAccessStorageName(pos, expected, storage_name); if (cluster.empty()) parseOnCluster(pos, expected, cluster); diff --git a/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp b/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp index 6ac41b77b84..798e200bfb3 100644 --- a/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp +++ b/src/Parsers/Access/ParserMoveAccessEntityQuery.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -73,7 +74,7 @@ bool ParserMoveAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & return false; } - if (!ParserKeyword{"TO"}.ignore(pos, expected) || !parseStorageName(pos, expected, storage_name)) + if (!ParserKeyword{"TO"}.ignore(pos, expected) || !parseAccessStorageName(pos, expected, storage_name)) return false; if (cluster.empty()) diff --git a/src/Parsers/Access/parseUserName.h b/src/Parsers/Access/parseUserName.h index 64fed2ce21c..10c548d8826 100644 --- a/src/Parsers/Access/parseUserName.h +++ b/src/Parsers/Access/parseUserName.h @@ -35,9 +35,4 @@ inline bool parseRoleNames(IParser::Pos & pos, Expected & expected, Strings & ro return parseUserNames(pos, expected, role_names); } -inline bool parseStorageName(IParser::Pos & pos, Expected & expected, String & storage_name) -{ - return parseIdentifierOrStringLiteral(pos, expected, storage_name); -} - } From 8c14c4b5e8925c6793c04cced5ebe64668ba3944 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 14:59:28 +0000 Subject: [PATCH 051/777] Remove unneeded readBinary() specializations + update docs The default instantiation template requires is_arithmetic_v inline void readBinary(T & x, ReadBuffer & buf) { readPODBinary(x, buf); } covers Int32, Int128, Int256, UInt32, UInt128 and UInt256 cases already. --- docs/en/sql-reference/data-types/uuid.md | 24 +++++++++++------------- src/IO/ReadHelpers.h | 6 ------ 2 files changed, 11 insertions(+), 19 deletions(-) diff --git a/docs/en/sql-reference/data-types/uuid.md b/docs/en/sql-reference/data-types/uuid.md index b0f19f0d8be..40f756b9588 100644 --- a/docs/en/sql-reference/data-types/uuid.md +++ b/docs/en/sql-reference/data-types/uuid.md @@ -6,42 +6,42 @@ sidebar_label: UUID # UUID -A universally unique identifier (UUID) is a 16-byte number used to identify records. For detailed information about the UUID, see [Wikipedia](https://en.wikipedia.org/wiki/Universally_unique_identifier). +A Universally Unique Identifier (UUID) is a 16-byte value used to identify records. For detailed information about UUIDs, see [Wikipedia](https://en.wikipedia.org/wiki/Universally_unique_identifier). -The example of UUID type value is represented below: +While different UUID variants exist (see [here](https://datatracker.ietf.org/doc/html/draft-ietf-uuidrev-rfc4122bis)), ClickHouse does not validate that inserted UUIDs conform to a particular variant. UUIDs are internally treated as a sequence of 16 random bytes with [8-4-4-4-12 representation](https://en.wikipedia.org/wiki/Universally_unique_identifier#Textual_representation) at SQL level. + +Example UUID value: ``` text 61f0c404-5cb3-11e7-907b-a6006ad3dba0 ``` -If you do not specify the UUID column value when inserting a new record, the UUID value is filled with zero: +The default UUID is all-zero. It is used, for example, when a new record is inserted but no value for a UUID column is specified: ``` text 00000000-0000-0000-0000-000000000000 ``` -## How to Generate +## Generating UUIDs -To generate the UUID value, ClickHouse provides the [generateUUIDv4](../../sql-reference/functions/uuid-functions.md) function. +ClickHouse provides the [generateUUIDv4](../../sql-reference/functions/uuid-functions.md) function to generate random UUID version 4 values. ## Usage Example **Example 1** -This example demonstrates creating a table with the UUID type column and inserting a value into the table. +This example demonstrates the creation of a table with a UUID column and the insertion of a value into the table. ``` sql CREATE TABLE t_uuid (x UUID, y String) ENGINE=TinyLog -``` -``` sql INSERT INTO t_uuid SELECT generateUUIDv4(), 'Example 1' -``` -``` sql SELECT * FROM t_uuid ``` +Result: + ``` text ┌────────────────────────────────────x─┬─y─────────┐ │ 417ddc5d-e556-4d27-95dd-a34d84e46a50 │ Example 1 │ @@ -50,13 +50,11 @@ SELECT * FROM t_uuid **Example 2** -In this example, the UUID column value is not specified when inserting a new record. +In this example, no UUID column value is specified when the record is inserted, i.e. the default UUID value is inserted: ``` sql INSERT INTO t_uuid (y) VALUES ('Example 2') -``` -``` sql SELECT * FROM t_uuid ``` diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 2636898c1b3..1fe08a738c2 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1096,12 +1096,6 @@ inline void readBinary(bool & x, ReadBuffer & buf) } inline void readBinary(String & x, ReadBuffer & buf) { readStringBinary(x, buf); } -inline void readBinary(Int32 & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(Int128 & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(Int256 & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(UInt32 & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(UInt128 & x, ReadBuffer & buf) { readPODBinary(x, buf); } -inline void readBinary(UInt256 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(Decimal32 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(Decimal64 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(Decimal128 & x, ReadBuffer & buf) { readPODBinary(x, buf); } From 07147ef88805ca09b5cd7120ecf60d71c1d94d55 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 15:24:28 +0000 Subject: [PATCH 052/777] Remove remainders of legacy setting 'allow_experimental_query_cache' --- src/Core/Settings.h | 1 - ...726_set_allow_experimental_query_cache_as_obsolete.reference | 0 .../02726_set_allow_experimental_query_cache_as_obsolete.sql | 2 -- 3 files changed, 3 deletions(-) delete mode 100644 tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.reference delete mode 100644 tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c69d132ea25..e7fd28476d9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -808,7 +808,6 @@ class IColumn; MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_parts_interval_seconds, 1) \ MAKE_OBSOLETE(M, UInt64, partial_merge_join_optimizations, 0) \ MAKE_OBSOLETE(M, MaxThreads, max_alter_threads, 0) \ - MAKE_OBSOLETE(M, Bool, allow_experimental_query_cache, true) \ /* moved to config.xml: see also src/Core/ServerSettings.h */ \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_buffer_flush_schedule_pool_size, 16) \ MAKE_DEPRECATED_BY_SERVER_CONFIG(M, UInt64, background_pool_size, 16) \ diff --git a/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.reference b/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql b/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql deleted file mode 100644 index 244ba4e959a..00000000000 --- a/tests/queries/0_stateless/02726_set_allow_experimental_query_cache_as_obsolete.sql +++ /dev/null @@ -1,2 +0,0 @@ -SET allow_experimental_query_cache = 0; -SET allow_experimental_query_cache = 1; From b98cb7fa145d1a92c2c78421be1eeb8fe8353d53 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 27 Jul 2023 17:50:39 +0000 Subject: [PATCH 053/777] Fix build --- src/IO/ReadHelpers.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 1fe08a738c2..bbb1c517d7c 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1101,6 +1101,9 @@ inline void readBinary(Decimal64 & x, ReadBuffer & buf) { readPODBinary(x, buf); inline void readBinary(Decimal128 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(Decimal256 & x, ReadBuffer & buf) { readPODBinary(x.value, buf); } inline void readBinary(LocalDate & x, ReadBuffer & buf) { readPODBinary(x, buf); } +inline void readBinary(UUID & x, ReadBuffer & buf) { readPODBinary(x, buf); } +inline void readBinary(IPv4 & x, ReadBuffer & buf) { readPODBinary(x, buf); } +inline void readBinary(IPv6 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(StackTrace::FramePointers & x, ReadBuffer & buf) { readPODBinary(x, buf); } From 6d77d52dfe034afe196fa1219ddc8897d1070146 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 27 Jul 2023 18:02:29 +0000 Subject: [PATCH 054/777] Allow variable number of columns in TSV/CuatomSeprarated/JSONCompactEachRow, make schema inference work with variable number of columns --- docs/en/interfaces/formats.md | 11 ++-- .../operations/settings/settings-formats.md | 25 +++++++- src/Core/Settings.h | 5 +- src/Formats/FormatFactory.cpp | 3 + src/Formats/FormatSettings.h | 3 + src/Processors/Formats/ISchemaReader.cpp | 46 +++++++++++--- src/Processors/Formats/ISchemaReader.h | 6 +- .../Formats/Impl/CSVRowInputFormat.cpp | 13 ++-- .../Formats/Impl/CSVRowInputFormat.h | 8 ++- .../Impl/CustomSeparatedRowInputFormat.cpp | 19 +++--- .../Impl/CustomSeparatedRowInputFormat.h | 10 +++- .../Impl/JSONCompactEachRowRowInputFormat.cpp | 8 ++- .../Impl/JSONCompactEachRowRowInputFormat.h | 7 ++- .../Formats/Impl/MsgPackRowInputFormat.cpp | 2 +- .../Formats/Impl/MsgPackRowInputFormat.h | 2 +- .../Formats/Impl/MySQLDumpRowInputFormat.cpp | 2 +- .../Formats/Impl/MySQLDumpRowInputFormat.h | 2 +- .../Formats/Impl/RegexpRowInputFormat.cpp | 2 +- .../Formats/Impl/RegexpRowInputFormat.h | 2 +- .../Impl/TabSeparatedRowInputFormat.cpp | 16 +++-- .../Formats/Impl/TabSeparatedRowInputFormat.h | 9 ++- .../Formats/Impl/TemplateRowInputFormat.cpp | 2 +- .../Formats/Impl/TemplateRowInputFormat.h | 2 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 2 +- .../Formats/Impl/ValuesBlockInputFormat.h | 2 +- .../RowInputFormatWithNamesAndTypes.cpp | 60 ++++++++++--------- .../Formats/RowInputFormatWithNamesAndTypes.h | 10 ++-- ..._with_variable_number_of_columns.reference | 52 ++++++++++++++++ ...ormats_with_variable_number_of_columns.sql | 18 ++++++ 29 files changed, 264 insertions(+), 85 deletions(-) create mode 100644 tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.reference create mode 100644 tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.sql diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 15f9d1f47bf..95483068cb2 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -195,6 +195,7 @@ SELECT * FROM nestedt FORMAT TSV - [input_format_tsv_skip_first_lines](/docs/en/operations/settings/settings-formats.md/#input_format_tsv_skip_first_lines) - skip specified number of lines at the beginning of data. Default value - `0`. - [input_format_tsv_detect_header](/docs/en/operations/settings/settings-formats.md/#input_format_tsv_detect_header) - automatically detect header with names and types in TSV format. Default value - `true`. - [input_format_tsv_skip_trailing_empty_lines](/docs/en/operations/settings/settings-formats.md/#input_format_tsv_skip_trailing_empty_lines) - skip trailing empty lines at the end of data. Default value - `false`. +- [input_format_tsv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_tsv_allow_variable_number_of_columns) - allow variable number of columns in TSV format, ignore extra columns and use default values on missing columns. Default value - `false`. ## TabSeparatedRaw {#tabseparatedraw} @@ -472,7 +473,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe - [input_format_csv_skip_trailing_empty_lines](/docs/en/operations/settings/settings-formats.md/#input_format_csv_skip_trailing_empty_lines) - skip trailing empty lines at the end of data. Default value - `false`. - [input_format_csv_trim_whitespaces](/docs/en/operations/settings/settings-formats.md/#input_format_csv_trim_whitespaces) - trim spaces and tabs in non-quoted CSV strings. Default value - `true`. - [input_format_csv_allow_whitespace_or_tab_as_delimiter](/docs/en/operations/settings/settings-formats.md/# input_format_csv_allow_whitespace_or_tab_as_delimiter) - Allow to use whitespace or tab as field delimiter in CSV strings. Default value - `false`. -- [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. Default value - `false`. +- [input_format_csv_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_csv_allow_variable_number_of_columns) - allow variable number of columns in CSV format, ignore extra columns and use default values on missing columns. Default value - `false`. - [input_format_csv_use_default_on_bad_values](/docs/en/operations/settings/settings-formats.md/#input_format_csv_use_default_on_bad_values) - Allow to set default value to column when CSV field deserialization failed on bad value. Default value - `false`. ## CSVWithNames {#csvwithnames} @@ -501,9 +502,10 @@ the types from input data will be compared with the types of the corresponding c Similar to [Template](#format-template), but it prints or reads all names and types of columns and uses escaping rule from [format_custom_escaping_rule](/docs/en/operations/settings/settings-formats.md/#format_custom_escaping_rule) setting and delimiters from [format_custom_field_delimiter](/docs/en/operations/settings/settings-formats.md/#format_custom_field_delimiter), [format_custom_row_before_delimiter](/docs/en/operations/settings/settings-formats.md/#format_custom_row_before_delimiter), [format_custom_row_after_delimiter](/docs/en/operations/settings/settings-formats.md/#format_custom_row_after_delimiter), [format_custom_row_between_delimiter](/docs/en/operations/settings/settings-formats.md/#format_custom_row_between_delimiter), [format_custom_result_before_delimiter](/docs/en/operations/settings/settings-formats.md/#format_custom_result_before_delimiter) and [format_custom_result_after_delimiter](/docs/en/operations/settings/settings-formats.md/#format_custom_result_after_delimiter) settings, not from format strings. -If setting [input_format_custom_detect_header](/docs/en/operations/settings/settings-formats.md/#input_format_custom_detect_header) is enabled, ClickHouse will automatically detect header with names and types if any. - -If setting [input_format_tsv_skip_trailing_empty_lines](/docs/en/operations/settings/settings-formats.md/#input_format_custom_detect_header) is enabled, trailing empty lines at the end of file will be skipped. +Additional settings: +- [input_format_custom_detect_header](/docs/en/operations/settings/settings-formats.md/#input_format_custom_detect_header) - enables automatic detection of header with names and types if any. Default value - `true`. +- [input_format_custom_skip_trailing_empty_lines](/docs/en/operations/settings/settings-formats.md/#input_format_custom_skip_trailing_empty_lines) - skip trailing empty lines at the end of file . Default value - `false`. +- [input_format_custom_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_custom_allow_variable_number_of_columns) - allow variable number of columns in CustomSeparated format, ignore extra columns and use default values on missing columns. Default value - `false`. There is also `CustomSeparatedIgnoreSpaces` format, which is similar to [TemplateIgnoreSpaces](#templateignorespaces). @@ -1261,6 +1263,7 @@ SELECT * FROM json_each_row_nested - [input_format_json_named_tuples_as_objects](/docs/en/operations/settings/settings-formats.md/#input_format_json_named_tuples_as_objects) - parse named tuple columns as JSON objects. Default value - `true`. - [input_format_json_defaults_for_missing_elements_in_named_tuple](/docs/en/operations/settings/settings-formats.md/#input_format_json_defaults_for_missing_elements_in_named_tuple) - insert default values for missing elements in JSON object while parsing named tuple. Default value - `true`. - [input_format_json_ignore_unknown_keys_in_named_tuple](/docs/en/operations/settings/settings-formats.md/#input_format_json_ignore_unknown_keys_in_named_tuple) - Ignore unknown keys in json object for named tuples. Default value - `false`. +- [input_format_json_compact_allow_variable_number_of_columns](/docs/en/operations/settings/settings-formats.md/#input_format_json_compact_allow_variable_number_of_columns) - allow variable number of columns in JSONCompact/JSONCompactEachRow format, ignore extra columns and use default values on missing columns. Default value - `false`. - [output_format_json_quote_64bit_integers](/docs/en/operations/settings/settings-formats.md/#output_format_json_quote_64bit_integers) - controls quoting of 64-bit integers in JSON output format. Default value - `true`. - [output_format_json_quote_64bit_floats](/docs/en/operations/settings/settings-formats.md/#output_format_json_quote_64bit_floats) - controls quoting of 64-bit floats in JSON output format. Default value - `false`. - [output_format_json_quote_denormals](/docs/en/operations/settings/settings-formats.md/#output_format_json_quote_denormals) - enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format. Default value - `false`. diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index ee8e0d547b8..8e3d6b74ffa 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -623,6 +623,13 @@ Column type should be String. If value is empty, default names `row_{i}`will be Default value: ''. +### input_format_json_compact_allow_variable_number_of_columns {#input_format_json_compact_allow_variable_number_of_columns} + +Allow variable number of columns in rows in JSONCompact/JSONCompactEachRow input formats. +Ignore extra columns in rows with more columns than expected and treat missing columns as default values. + +Disabled by default. + ## TSV format settings {#tsv-format-settings} ### input_format_tsv_empty_as_default {#input_format_tsv_empty_as_default} @@ -760,6 +767,13 @@ When enabled, trailing empty lines at the end of TSV file will be skipped. Disabled by default. +### input_format_tsv_allow_variable_number_of_columns {#input_format_tsv_allow_variable_number_of_columns} + +Allow variable number of columns in rows in TSV input format. +Ignore extra columns in rows with more columns than expected and treat missing columns as default values. + +Disabled by default. + ## CSV format settings {#csv-format-settings} ### format_csv_delimiter {#format_csv_delimiter} @@ -951,9 +965,11 @@ Result ```text " string " ``` + ### input_format_csv_allow_variable_number_of_columns {#input_format_csv_allow_variable_number_of_columns} -ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values. +Allow variable number of columns in rows in CSV input format. +Ignore extra columns in rows with more columns than expected and treat missing columns as default values. Disabled by default. @@ -1589,6 +1605,13 @@ When enabled, trailing empty lines at the end of file in CustomSeparated format Disabled by default. +### input_format_custom_allow_variable_number_of_columns {#input_format_custom_allow_variable_number_of_columns} + +Allow variable number of columns in rows in CustomSeparated input format. +Ignore extra columns in rows with more columns than expected and treat missing columns as default values. + +Disabled by default. + ## Regexp format settings {#regexp-format-settings} ### format_regexp_escaping_rule {#format_regexp_escaping_rule} diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c69d132ea25..86146bfad07 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -879,6 +879,10 @@ class IColumn; M(Bool, input_format_csv_allow_whitespace_or_tab_as_delimiter, false, "Allow to use spaces and tabs(\\t) as field delimiter in the CSV strings", 0) \ M(Bool, input_format_csv_trim_whitespaces, true, "Trims spaces and tabs (\\t) characters at the beginning and end in CSV strings", 0) \ M(Bool, input_format_csv_use_default_on_bad_values, false, "Allow to set default value to column when CSV field deserialization failed on bad value", 0) \ + M(Bool, input_format_csv_allow_variable_number_of_columns, false, "Ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values", 0) \ + M(Bool, input_format_tsv_allow_variable_number_of_columns, false, "Ignore extra columns in TSV input (if file has more columns than expected) and treat missing fields in TSV input as default values", 0) \ + M(Bool, input_format_custom_allow_variable_number_of_columns, false, "Ignore extra columns in CustomSeparated input (if file has more columns than expected) and treat missing fields in CustomSeparated input as default values", 0) \ + M(Bool, input_format_json_compact_allow_variable_number_of_columns, false, "Ignore extra columns in JSONCompact(EachRow) input (if file has more columns than expected) and treat missing fields in JSONCompact(EachRow) input as default values", 0) \ M(Bool, input_format_tsv_detect_header, true, "Automatically detect header with names and types in TSV format", 0) \ M(Bool, input_format_custom_detect_header, true, "Automatically detect header with names and types in CustomSeparated format", 0) \ M(Bool, input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Parquet", 0) \ @@ -1023,7 +1027,6 @@ class IColumn; M(Bool, regexp_dict_allow_hyperscan, true, "Allow regexp_tree dictionary using Hyperscan library.", 0) \ \ M(Bool, dictionary_use_async_executor, false, "Execute a pipeline for reading from a dictionary with several threads. It's supported only by DIRECT dictionary with CLICKHOUSE source.", 0) \ - M(Bool, input_format_csv_allow_variable_number_of_columns, false, "Ignore extra columns in CSV input (if file has more columns than expected) and treat missing fields in CSV input as default values", 0) \ // End of FORMAT_FACTORY_SETTINGS // Please add settings non-related to formats into the COMMON_SETTINGS above. diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 663b7f1ba95..dff480d1f79 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -86,6 +86,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter; format_settings.custom.try_detect_header = settings.input_format_custom_detect_header; format_settings.custom.skip_trailing_empty_lines = settings.input_format_custom_skip_trailing_empty_lines; + format_settings.custom.allow_variable_number_of_columns = settings.input_format_custom_allow_variable_number_of_columns; format_settings.date_time_input_format = settings.date_time_input_format; format_settings.date_time_output_format = settings.date_time_output_format; format_settings.interval.output_format = settings.interval_output_format; @@ -115,6 +116,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.json.validate_utf8 = settings.output_format_json_validate_utf8; format_settings.json_object_each_row.column_for_object_name = settings.format_json_object_each_row_column_for_object_name; format_settings.json.allow_object_type = context->getSettingsRef().allow_experimental_object_type; + format_settings.json.compact_allow_variable_number_of_columns = settings.input_format_json_compact_allow_variable_number_of_columns; format_settings.null_as_default = settings.input_format_null_as_default; format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros; format_settings.parquet.row_group_rows = settings.output_format_parquet_row_group_size; @@ -161,6 +163,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.tsv.skip_first_lines = settings.input_format_tsv_skip_first_lines; format_settings.tsv.try_detect_header = settings.input_format_tsv_detect_header; format_settings.tsv.skip_trailing_empty_lines = settings.input_format_tsv_skip_trailing_empty_lines; + format_settings.tsv.allow_variable_number_of_columns = settings.input_format_tsv_allow_variable_number_of_columns; format_settings.values.accurate_types_of_literals = settings.input_format_values_accurate_types_of_literals; format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions; format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 3259c46e5ff..68cf9ad817d 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -176,6 +176,7 @@ struct FormatSettings EscapingRule escaping_rule = EscapingRule::Escaped; bool try_detect_header = true; bool skip_trailing_empty_lines = false; + bool allow_variable_number_of_columns = false; } custom; struct @@ -198,6 +199,7 @@ struct FormatSettings bool validate_types_from_metadata = true; bool validate_utf8 = false; bool allow_object_type = false; + bool compact_allow_variable_number_of_columns = false; } json; struct @@ -316,6 +318,7 @@ struct FormatSettings UInt64 skip_first_lines = 0; bool try_detect_header = true; bool skip_trailing_empty_lines = false; + bool allow_variable_number_of_columns = false; } tsv; struct diff --git a/src/Processors/Formats/ISchemaReader.cpp b/src/Processors/Formats/ISchemaReader.cpp index 1fa520eaaee..15b53c2a499 100644 --- a/src/Processors/Formats/ISchemaReader.cpp +++ b/src/Processors/Formats/ISchemaReader.cpp @@ -115,21 +115,24 @@ NamesAndTypesList IRowSchemaReader::readSchema() "Cannot read rows to determine the schema, the maximum number of rows (or bytes) to read is set to 0. " "Most likely setting input_format_max_rows_to_read_for_schema_inference or input_format_max_bytes_to_read_for_schema_inference is set to 0"); - DataTypes data_types = readRowAndGetDataTypes(); + auto data_types_maybe = readRowAndGetDataTypes(); /// Check that we read at list one column. - if (data_types.empty()) + if (!data_types_maybe) throw Exception(ErrorCodes::EMPTY_DATA_PASSED, "Cannot read rows from the data"); + DataTypes data_types = std::move(*data_types_maybe); + /// If column names weren't set, use default names 'c1', 'c2', ... - if (column_names.empty()) + bool use_default_column_names = column_names.empty(); + if (use_default_column_names) { column_names.reserve(data_types.size()); for (size_t i = 0; i != data_types.size(); ++i) column_names.push_back("c" + std::to_string(i + 1)); } /// If column names were set, check that the number of names match the number of types. - else if (column_names.size() != data_types.size()) + else if (column_names.size() != data_types.size() && !allowVariableNumberOfColumns()) { throw Exception( ErrorCodes::INCORRECT_DATA, @@ -137,6 +140,9 @@ NamesAndTypesList IRowSchemaReader::readSchema() } else { + if (column_names.size() != data_types.size()) + data_types.resize(column_names.size()); + std::unordered_set names_set; for (const auto & name : column_names) { @@ -155,13 +161,39 @@ NamesAndTypesList IRowSchemaReader::readSchema() for (rows_read = 1; rows_read < max_rows_to_read && in.count() < max_bytes_to_read; ++rows_read) { - DataTypes new_data_types = readRowAndGetDataTypes(); - if (new_data_types.empty()) + auto new_data_types_maybe = readRowAndGetDataTypes(); + if (!new_data_types_maybe) /// We reached eof. break; + DataTypes new_data_types = std::move(*new_data_types_maybe); + if (new_data_types.size() != data_types.size()) - throw Exception(ErrorCodes::INCORRECT_DATA, "Rows have different amount of values"); + { + if (!allowVariableNumberOfColumns()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Rows have different amount of values"); + + if (use_default_column_names) + { + /// Current row contains new columns, add new default names. + if (new_data_types.size() > data_types.size()) + { + for (size_t i = data_types.size(); i < new_data_types.size(); ++i) + column_names.push_back("c" + std::to_string(i + 1)); + data_types.resize(new_data_types.size()); + } + /// Current row contain less columns than previous rows. + else + { + new_data_types.resize(data_types.size()); + } + } + /// If names were explicitly set, ignore all extra columns. + else + { + new_data_types.resize(column_names.size()); + } + } for (field_index = 0; field_index != data_types.size(); ++field_index) { diff --git a/src/Processors/Formats/ISchemaReader.h b/src/Processors/Formats/ISchemaReader.h index 40702198a57..0cc8b98f05e 100644 --- a/src/Processors/Formats/ISchemaReader.h +++ b/src/Processors/Formats/ISchemaReader.h @@ -93,11 +93,13 @@ protected: /// Read one row and determine types of columns in it. /// Return types in the same order in which the values were in the row. /// If it's impossible to determine the type for some column, return nullptr for it. - /// Return empty list if can't read more data. - virtual DataTypes readRowAndGetDataTypes() = 0; + /// Return std::nullopt if can't read more data. + virtual std::optional readRowAndGetDataTypes() = 0; void setColumnNames(const std::vector & names) { column_names = names; } + virtual bool allowVariableNumberOfColumns() const { return false; } + size_t field_index; private: diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 244b906549e..9092c7fceba 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -284,7 +284,7 @@ bool CSVFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out) return true; } -bool CSVFormatReader::allowVariableNumberOfColumns() +bool CSVFormatReader::allowVariableNumberOfColumns() const { return format_settings.csv.allow_variable_number_of_columns; } @@ -410,19 +410,22 @@ CSVSchemaReader::CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_t { } -std::pair, DataTypes> CSVSchemaReader::readRowAndGetFieldsAndDataTypes() +std::optional, DataTypes>> CSVSchemaReader::readRowAndGetFieldsAndDataTypes() { if (buf.eof()) return {}; auto fields = reader.readRow(); auto data_types = tryInferDataTypesByEscapingRule(fields, format_settings, FormatSettings::EscapingRule::CSV); - return {fields, data_types}; + return std::make_pair(fields, data_types); } -DataTypes CSVSchemaReader::readRowAndGetDataTypesImpl() +std::optional CSVSchemaReader::readRowAndGetDataTypesImpl() { - return std::move(readRowAndGetFieldsAndDataTypes().second); + auto fields_with_types = readRowAndGetFieldsAndDataTypes(); + if (!fields_with_types) + return {}; + return std::move(fields_with_types->second); } diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index 7b1a1fc433d..2444477b184 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -70,7 +70,7 @@ public: void skipPrefixBeforeHeader() override; bool checkForEndOfRow() override; - bool allowVariableNumberOfColumns() override; + bool allowVariableNumberOfColumns() const override; std::vector readNames() override { return readHeaderRow(); } std::vector readTypes() override { return readHeaderRow(); } @@ -102,8 +102,10 @@ public: CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_settings_); private: - DataTypes readRowAndGetDataTypesImpl() override; - std::pair, DataTypes> readRowAndGetFieldsAndDataTypes() override; + bool allowVariableNumberOfColumns() const override { return format_settings.csv.allow_variable_number_of_columns; } + + std::optional readRowAndGetDataTypesImpl() override; + std::optional, DataTypes>> readRowAndGetFieldsAndDataTypes() override; PeekableReadBuffer buf; CSVFormatReader reader; diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp index 1e67db79a2c..8f8e12e3c2a 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp @@ -142,7 +142,7 @@ void CustomSeparatedFormatReader::skipField() skipFieldByEscapingRule(*buf, format_settings.custom.escaping_rule, format_settings); } -bool CustomSeparatedFormatReader::checkEndOfRow() +bool CustomSeparatedFormatReader::checkForEndOfRow() { PeekableReadBufferCheckpoint checkpoint{*buf, true}; @@ -200,12 +200,12 @@ std::vector CustomSeparatedFormatReader::readRowImpl() std::vector values; skipRowStartDelimiter(); - if (columns == 0) + if (columns == 0 || allowVariableNumberOfColumns()) { do { values.push_back(readFieldIntoString(values.empty(), false, true)); - } while (!checkEndOfRow()); + } while (!checkForEndOfRow()); columns = values.size(); } else @@ -230,7 +230,7 @@ void CustomSeparatedFormatReader::skipHeaderRow() skipField(); } - while (!checkEndOfRow()); + while (!checkForEndOfRow()); skipRowEndDelimiter(); } @@ -369,7 +369,7 @@ CustomSeparatedSchemaReader::CustomSeparatedSchemaReader( { } -std::pair, DataTypes> CustomSeparatedSchemaReader::readRowAndGetFieldsAndDataTypes() +std::optional, DataTypes>> CustomSeparatedSchemaReader::readRowAndGetFieldsAndDataTypes() { if (no_more_data || reader.checkForSuffix()) { @@ -385,12 +385,15 @@ std::pair, DataTypes> CustomSeparatedSchemaReader::readRowAn auto fields = reader.readRow(); auto data_types = tryInferDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule(), &json_inference_info); - return {fields, data_types}; + return std::make_pair(fields, data_types); } -DataTypes CustomSeparatedSchemaReader::readRowAndGetDataTypesImpl() +std::optional CustomSeparatedSchemaReader::readRowAndGetDataTypesImpl() { - return readRowAndGetFieldsAndDataTypes().second; + auto fields_with_types = readRowAndGetFieldsAndDataTypes(); + if (!fields_with_types) + return {}; + return std::move(fields_with_types->second); } void CustomSeparatedSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h index 2acf35bd143..893f06409f6 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h @@ -74,7 +74,9 @@ public: std::vector readRowForHeaderDetection() override { return readRowImpl(); } - bool checkEndOfRow(); + bool checkForEndOfRow() override; + bool allowVariableNumberOfColumns() const override { return format_settings.custom.allow_variable_number_of_columns; } + bool checkForSuffixImpl(bool check_eof); inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(*buf, true); } @@ -109,9 +111,11 @@ public: CustomSeparatedSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_setting_); private: - DataTypes readRowAndGetDataTypesImpl() override; + bool allowVariableNumberOfColumns() const override { return format_settings.custom.allow_variable_number_of_columns; } - std::pair, DataTypes> readRowAndGetFieldsAndDataTypes() override; + std::optional readRowAndGetDataTypesImpl() override; + + std::optional, DataTypes>> readRowAndGetFieldsAndDataTypes() override; void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override; diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index b91345bebe3..e3583a3dff0 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -112,6 +112,12 @@ bool JSONCompactEachRowFormatReader::readField(IColumn & column, const DataTypeP return JSONUtils::readField(*in, column, type, serialization, column_name, format_settings, yield_strings); } +bool JSONCompactEachRowFormatReader::checkForEndOfRow() +{ + skipWhitespaceIfAny(*in); + return !in->eof() && *in->position() == ']'; +} + bool JSONCompactEachRowFormatReader::parseRowStartWithDiagnosticInfo(WriteBuffer & out) { skipWhitespaceIfAny(*in); @@ -187,7 +193,7 @@ JSONCompactEachRowRowSchemaReader::JSONCompactEachRowRowSchemaReader( { } -DataTypes JSONCompactEachRowRowSchemaReader::readRowAndGetDataTypesImpl() +std::optional JSONCompactEachRowRowSchemaReader::readRowAndGetDataTypesImpl() { if (first_row) first_row = false; diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index bb699f0ca2e..378a41e6471 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -68,6 +68,9 @@ public: std::vector readNames() override { return readHeaderRow(); } std::vector readTypes() override { return readHeaderRow(); } + bool checkForEndOfRow() override; + bool allowVariableNumberOfColumns() const override { return format_settings.json.compact_allow_variable_number_of_columns; } + bool yieldStrings() const { return yield_strings; } private: bool yield_strings; @@ -79,7 +82,9 @@ public: JSONCompactEachRowRowSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool yield_strings_, const FormatSettings & format_settings_); private: - DataTypes readRowAndGetDataTypesImpl() override; + bool allowVariableNumberOfColumns() const override { return format_settings.json.compact_allow_variable_number_of_columns; } + + std::optional readRowAndGetDataTypesImpl() override; void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override; void transformFinalTypeIfNeeded(DataTypePtr & type) override; diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index eeca14176cc..a46f0018611 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -634,7 +634,7 @@ DataTypePtr MsgPackSchemaReader::getDataType(const msgpack::object & object) UNREACHABLE(); } -DataTypes MsgPackSchemaReader::readRowAndGetDataTypes() +std::optional MsgPackSchemaReader::readRowAndGetDataTypes() { if (buf.eof()) return {}; diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h index 0b485d3b97c..028ab878ad0 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h @@ -91,7 +91,7 @@ public: private: msgpack::object_handle readObject(); DataTypePtr getDataType(const msgpack::object & object); - DataTypes readRowAndGetDataTypes() override; + std::optional readRowAndGetDataTypes() override; PeekableReadBuffer buf; UInt64 number_of_columns; diff --git a/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp b/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp index 90dd07bd5a8..6c754f141da 100644 --- a/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.cpp @@ -422,7 +422,7 @@ NamesAndTypesList MySQLDumpSchemaReader::readSchema() return IRowSchemaReader::readSchema(); } -DataTypes MySQLDumpSchemaReader::readRowAndGetDataTypes() +std::optional MySQLDumpSchemaReader::readRowAndGetDataTypes() { if (in.eof()) return {}; diff --git a/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.h b/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.h index c28355054d7..14a73bf83b0 100644 --- a/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.h +++ b/src/Processors/Formats/Impl/MySQLDumpRowInputFormat.h @@ -33,7 +33,7 @@ public: private: NamesAndTypesList readSchema() override; - DataTypes readRowAndGetDataTypes() override; + std::optional readRowAndGetDataTypes() override; String table_name; }; diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index d902a8be6a7..8e94a568b1e 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -143,7 +143,7 @@ RegexpSchemaReader::RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & { } -DataTypes RegexpSchemaReader::readRowAndGetDataTypes() +std::optional RegexpSchemaReader::readRowAndGetDataTypes() { if (buf.eof()) return {}; diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/src/Processors/Formats/Impl/RegexpRowInputFormat.h index 2469774aaf9..7417d48d8c1 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.h +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -79,7 +79,7 @@ public: RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings); private: - DataTypes readRowAndGetDataTypes() override; + std::optional readRowAndGetDataTypes() override; void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override; diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 2239c8539e3..7fbad583ced 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -300,6 +300,11 @@ bool TabSeparatedFormatReader::checkForSuffix() return false; } +bool TabSeparatedFormatReader::checkForEndOfRow() +{ + return buf->eof() || *buf->position() == '\n'; +} + TabSeparatedSchemaReader::TabSeparatedSchemaReader( ReadBuffer & in_, bool with_names_, bool with_types_, bool is_raw_, const FormatSettings & format_settings_) : FormatWithNamesAndTypesSchemaReader( @@ -315,19 +320,22 @@ TabSeparatedSchemaReader::TabSeparatedSchemaReader( { } -std::pair, DataTypes> TabSeparatedSchemaReader::readRowAndGetFieldsAndDataTypes() +std::optional, DataTypes>> TabSeparatedSchemaReader::readRowAndGetFieldsAndDataTypes() { if (buf.eof()) return {}; auto fields = reader.readRow(); auto data_types = tryInferDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule()); - return {fields, data_types}; + return std::make_pair(fields, data_types); } -DataTypes TabSeparatedSchemaReader::readRowAndGetDataTypesImpl() +std::optional TabSeparatedSchemaReader::readRowAndGetDataTypesImpl() { - return readRowAndGetFieldsAndDataTypes().second; + auto fields_with_types = readRowAndGetFieldsAndDataTypes(); + if (!fields_with_types) + return {}; + return std::move(fields_with_types->second); } void registerInputFormatTabSeparated(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h index 8df57675cf5..e0234761d61 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h @@ -76,6 +76,9 @@ public: void setReadBuffer(ReadBuffer & in_) override; bool checkForSuffix() override; + bool checkForEndOfRow() override; + + bool allowVariableNumberOfColumns() const override { return format_settings.tsv.allow_variable_number_of_columns; } private: template @@ -92,8 +95,10 @@ public: TabSeparatedSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool is_raw_, const FormatSettings & format_settings); private: - DataTypes readRowAndGetDataTypesImpl() override; - std::pair, DataTypes> readRowAndGetFieldsAndDataTypes() override; + bool allowVariableNumberOfColumns() const override { return format_settings.tsv.allow_variable_number_of_columns; } + + std::optional readRowAndGetDataTypesImpl() override; + std::optional, DataTypes>> readRowAndGetFieldsAndDataTypes() override; PeekableReadBuffer buf; TabSeparatedFormatReader reader; diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp index 8a09e800fa7..b065e00f5d1 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp @@ -490,7 +490,7 @@ TemplateSchemaReader::TemplateSchemaReader( setColumnNames(row_format.column_names); } -DataTypes TemplateSchemaReader::readRowAndGetDataTypes() +std::optional TemplateSchemaReader::readRowAndGetDataTypes() { if (first_row) format_reader.readPrefix(); diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.h b/src/Processors/Formats/Impl/TemplateRowInputFormat.h index 8f9088e2c47..2752cb13e50 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.h +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.h @@ -119,7 +119,7 @@ public: std::string row_between_delimiter, const FormatSettings & format_settings_); - DataTypes readRowAndGetDataTypes() override; + std::optional readRowAndGetDataTypes() override; private: void transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 3a65a6fe4ea..6cb469afca1 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -638,7 +638,7 @@ ValuesSchemaReader::ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & { } -DataTypes ValuesSchemaReader::readRowAndGetDataTypes() +std::optional ValuesSchemaReader::readRowAndGetDataTypes() { if (first_row) { diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index 8f8d44ec088..7f1dbc0da66 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -105,7 +105,7 @@ public: ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings); private: - DataTypes readRowAndGetDataTypes() override; + std::optional readRowAndGetDataTypes() override; PeekableReadBuffer buf; ParserExpression parser; diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index fb49779e0af..cb5c11e2d3b 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -212,8 +212,23 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE format_reader->skipRowStartDelimiter(); ext.read_columns.resize(data_types.size()); - for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column) + size_t file_column = 0; + for (; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column) { + if (format_reader->allowVariableNumberOfColumns() && format_reader->checkForEndOfRow()) + { + while (file_column < column_mapping->column_indexes_for_input_fields.size()) + { + const auto & rem_column_index = column_mapping->column_indexes_for_input_fields[file_column]; + columns[*rem_column_index]->insertDefault(); + ++file_column; + } + break; + } + + if (file_column != 0) + format_reader->skipFieldDelimiter(); + const auto & column_index = column_mapping->column_indexes_for_input_fields[file_column]; const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size(); if (column_index) @@ -225,22 +240,6 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE column_mapping->names_of_columns[file_column]); else format_reader->skipField(file_column); - - if (!is_last_file_column) - { - if (format_reader->allowVariableNumberOfColumns() && format_reader->checkForEndOfRow()) - { - ++file_column; - while (file_column < column_mapping->column_indexes_for_input_fields.size()) - { - const auto & rem_column_index = column_mapping->column_indexes_for_input_fields[file_column]; - columns[*rem_column_index]->insertDefault(); - ++file_column; - } - } - else - format_reader->skipFieldDelimiter(); - } } if (format_reader->allowVariableNumberOfColumns() && !format_reader->checkForEndOfRow()) @@ -248,7 +247,7 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE do { format_reader->skipFieldDelimiter(); - format_reader->skipField(1); + format_reader->skipField(file_column++); } while (!format_reader->checkForEndOfRow()); } @@ -419,12 +418,14 @@ namespace void FormatWithNamesAndTypesSchemaReader::tryDetectHeader(std::vector & column_names, std::vector & type_names) { - auto [first_row_values, first_row_types] = readRowAndGetFieldsAndDataTypes(); + auto first_row = readRowAndGetFieldsAndDataTypes(); /// No data. - if (first_row_values.empty()) + if (!first_row) return; + auto [first_row_values, first_row_types] = *first_row; + /// The first row contains non String elements, it cannot be a header. if (!checkIfAllTypesAreString(first_row_types)) { @@ -432,15 +433,17 @@ void FormatWithNamesAndTypesSchemaReader::tryDetectHeader(std::vector & return; } - auto [second_row_values, second_row_types] = readRowAndGetFieldsAndDataTypes(); + auto second_row = readRowAndGetFieldsAndDataTypes(); /// Data contains only 1 row, don't treat it as a header. - if (second_row_values.empty()) + if (!second_row) { buffered_types = first_row_types; return; } + auto [second_row_values, second_row_types] = *second_row; + DataTypes data_types; bool second_row_can_be_type_names = checkIfAllTypesAreString(second_row_types) && checkIfAllValuesAreTypeNames(readNamesFromFields(second_row_values)); size_t row = 2; @@ -450,15 +453,16 @@ void FormatWithNamesAndTypesSchemaReader::tryDetectHeader(std::vector & } else { - data_types = readRowAndGetDataTypes(); + auto data_types_maybe = readRowAndGetDataTypes(); /// Data contains only 2 rows. - if (data_types.empty()) + if (!data_types_maybe) { second_row_can_be_type_names = false; data_types = second_row_types; } else { + data_types = *data_types_maybe; ++row; } } @@ -490,10 +494,10 @@ void FormatWithNamesAndTypesSchemaReader::tryDetectHeader(std::vector & return; } - auto next_row_types = readRowAndGetDataTypes(); + auto next_row_types_maybe = readRowAndGetDataTypes(); /// Check if there are no more rows in data. It means that all rows contains only String values and Nulls, /// so, the first two rows with all String elements can be real data and we cannot use them as a header. - if (next_row_types.empty()) + if (!next_row_types_maybe) { /// Buffer first data types from the first row, because it doesn't contain Nulls. buffered_types = first_row_types; @@ -502,11 +506,11 @@ void FormatWithNamesAndTypesSchemaReader::tryDetectHeader(std::vector & ++row; /// Combine types from current row and from previous rows. - chooseResultColumnTypes(*this, data_types, next_row_types, getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule::CSV), default_colum_names, row); + chooseResultColumnTypes(*this, data_types, *next_row_types_maybe, getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule::CSV), default_colum_names, row); } } -DataTypes FormatWithNamesAndTypesSchemaReader::readRowAndGetDataTypes() +std::optional FormatWithNamesAndTypesSchemaReader::readRowAndGetDataTypes() { /// Check if we tried to detect a header and have buffered types from read rows. if (!buffered_types.empty()) diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index b5103d3db39..7b3e2cbea67 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -121,7 +121,7 @@ public: virtual bool checkForEndOfRow() { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method checkForEndOfRow is not implemented"); } - virtual bool allowVariableNumberOfColumns() { return false; } + virtual bool allowVariableNumberOfColumns() const { return false; } const FormatSettings & getFormatSettings() const { return format_settings; } @@ -160,15 +160,15 @@ public: NamesAndTypesList readSchema() override; protected: - virtual DataTypes readRowAndGetDataTypes() override; + virtual std::optional readRowAndGetDataTypes() override; - virtual DataTypes readRowAndGetDataTypesImpl() + virtual std::optional readRowAndGetDataTypesImpl() { throw Exception{ErrorCodes::NOT_IMPLEMENTED, "Method readRowAndGetDataTypesImpl is not implemented"}; } - /// Return column fields with inferred types. In case of no more rows, return empty vectors. - virtual std::pair, DataTypes> readRowAndGetFieldsAndDataTypes() + /// Return column fields with inferred types. In case of no more rows, return nullopt. + virtual std::optional, DataTypes>> readRowAndGetFieldsAndDataTypes() { throw Exception{ErrorCodes::NOT_IMPLEMENTED, "Method readRowAndGetFieldsAndDataTypes is not implemented"}; } diff --git a/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.reference b/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.reference new file mode 100644 index 00000000000..39d24f2cbd2 --- /dev/null +++ b/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.reference @@ -0,0 +1,52 @@ +CSV +1 1 +2 0 +0 0 +3 3 +1 1 \N \N +2 \N \N \N +\N \N \N \N +3 3 3 3 +1 1 +2 \N +\N \N +3 3 +TSV +1 1 +2 0 +0 0 +3 3 +1 1 \N \N +2 \N \N \N +\N \N \N \N +3 3 3 3 +1 1 +2 \N +\N \N +3 3 +JSONCompactEachRow +1 1 +2 0 +0 0 +3 3 +1 1 \N \N +2 \N \N \N +\N \N \N \N +3 3 3 3 +1 1 +2 \N +\N \N +3 3 +CustomSeparated +1 1 +2 0 +0 0 +3 3 +1 1 \N \N +2 \N \N \N +\N \N \N \N +3 3 3 3 +1 1 +2 \N +\N \N +3 3 diff --git a/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.sql b/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.sql new file mode 100644 index 00000000000..c0a80bf2114 --- /dev/null +++ b/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.sql @@ -0,0 +1,18 @@ +select 'CSV'; +select * from format(CSV, 'x UInt32, y UInt32', '1,1\n2\n\n3,3,3,3') settings input_format_csv_allow_variable_number_of_columns=1; +select * from format(CSV, '1,1\n2\n\n3,3,3,3') settings input_format_csv_allow_variable_number_of_columns=1; +select * from format(CSVWithNames, '"x","y"\n1,1\n2\n\n3,3,3,3') settings input_format_csv_allow_variable_number_of_columns=1; +select 'TSV'; +select * from format(TSV, 'x UInt32, y UInt32', '1\t1\n2\n\n3\t3\t3\t3') settings input_format_tsv_allow_variable_number_of_columns=1; +select * from format(TSV, '1\t1\n2\n\n3\t3\t3\t3') settings input_format_tsv_allow_variable_number_of_columns=1; +select * from format(TSVWithNames, 'x\ty\n1\t1\n2\n\n3\t3\t3\t3') settings input_format_tsv_allow_variable_number_of_columns=1; +select 'JSONCompactEachRow'; +select * from format(JSONCompactEachRow, 'x UInt32, y UInt32', '[1,1]\n[2]\n[]\n[3,3,3,3]') settings input_format_json_compact_allow_variable_number_of_columns=1; +select * from format(JSONCompactEachRow, '[1,1]\n[2]\n[]\n[3,3,3,3]') settings input_format_json_compact_allow_variable_number_of_columns=1; +select * from format(JSONCompactEachRowWithNames, '["x","y"]\n[1,1]\n[2]\n[]\n[3,3,3,3]') settings input_format_json_compact_allow_variable_number_of_columns=1; +select 'CustomSeparated'; +set format_custom_escaping_rule='CSV', format_custom_field_delimiter='', format_custom_row_before_delimiter='', format_custom_row_after_delimiter='', format_custom_row_between_delimiter='', format_custom_result_before_delimiter='', format_custom_result_after_delimiter=''; +select * from format(CustomSeparated, 'x UInt32, y UInt32', '1123333') settings input_format_custom_allow_variable_number_of_columns=1; +select * from format(CustomSeparated, '1123333') settings input_format_custom_allow_variable_number_of_columns=1; +select * from format(CustomSeparatedWithNames, '"x""y"1123333') settings input_format_custom_allow_variable_number_of_columns=1; + From 9cc87b642b5c9077d563ee006f85065a158927ae Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 27 Jul 2023 16:58:35 -0700 Subject: [PATCH 055/777] Fix compilation error --- src/Common/TransformEndianness.hpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index 0a9055dde15..fe43861f66f 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -3,6 +3,8 @@ #include #include +#include + #include namespace DB @@ -65,4 +67,11 @@ inline void transformEndianness(StrongTypedef & x) { transformEndianness(x.toUnderType()); } + +template +inline void transformEndianness(CityHash_v1_0_2::uint128 & x) +{ + transformEndianness(x.low64); + transformEndianness(x.high64); +} } From 9dbb106dc14a085013140427401a235ae2115e47 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 27 Jul 2023 23:37:09 -0400 Subject: [PATCH 056/777] Fix tests, docs --- .../sql-reference/statements/create/quota.md | 1 + .../sql-reference/statements/create/role.md | 1 + .../statements/create/row-policy.md | 1 + .../statements/create/settings-profile.md | 1 + .../sql-reference/statements/create/user.md | 1 + docs/en/sql-reference/statements/drop.md | 10 +++--- docs/en/sql-reference/statements/move.md | 32 +++++++++++++++++ src/Access/IAccessStorage.cpp | 15 +++++--- src/Access/MultipleAccessStorage.cpp | 4 +-- .../InterpreterMoveAccessEntityQuery.cpp | 7 ++-- .../test.py | 11 +++--- tests/integration/test_quota/test.py | 36 +++++++++---------- 12 files changed, 81 insertions(+), 39 deletions(-) create mode 100644 docs/en/sql-reference/statements/move.md diff --git a/docs/en/sql-reference/statements/create/quota.md b/docs/en/sql-reference/statements/create/quota.md index c69285171ab..a6ced870c18 100644 --- a/docs/en/sql-reference/statements/create/quota.md +++ b/docs/en/sql-reference/statements/create/quota.md @@ -11,6 +11,7 @@ Syntax: ``` sql CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] + [IN access_storage_type] [KEYED BY {user_name | ip_address | client_key | client_key,user_name | client_key,ip_address} | NOT KEYED] [FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day | week | month | quarter | year} {MAX { {queries | query_selects | query_inserts | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number } [,...] | diff --git a/docs/en/sql-reference/statements/create/role.md b/docs/en/sql-reference/statements/create/role.md index 9b14e220e1f..4b6fffe4f60 100644 --- a/docs/en/sql-reference/statements/create/role.md +++ b/docs/en/sql-reference/statements/create/role.md @@ -11,6 +11,7 @@ Syntax: ``` sql CREATE ROLE [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1] [, name2 [ON CLUSTER cluster_name2] ...] + [IN access_storage_type] [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [CONST|READONLY|WRITABLE|CHANGEABLE_IN_READONLY] | PROFILE 'profile_name'] [,...] ``` diff --git a/docs/en/sql-reference/statements/create/row-policy.md b/docs/en/sql-reference/statements/create/row-policy.md index 83bb2e6bb9a..cd7718793bd 100644 --- a/docs/en/sql-reference/statements/create/row-policy.md +++ b/docs/en/sql-reference/statements/create/row-policy.md @@ -16,6 +16,7 @@ Syntax: ``` sql CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] policy_name1 [ON CLUSTER cluster_name1] ON [db1.]table1|db1.* [, policy_name2 [ON CLUSTER cluster_name2] ON [db2.]table2|db2.* ...] + [IN access_storage_type] [FOR SELECT] USING condition [AS {PERMISSIVE | RESTRICTIVE}] [TO {role1 [, role2 ...] | ALL | ALL EXCEPT role1 [, role2 ...]}] diff --git a/docs/en/sql-reference/statements/create/settings-profile.md b/docs/en/sql-reference/statements/create/settings-profile.md index 8e221a4d82f..d8afce9d6de 100644 --- a/docs/en/sql-reference/statements/create/settings-profile.md +++ b/docs/en/sql-reference/statements/create/settings-profile.md @@ -12,6 +12,7 @@ Syntax: ``` sql CREATE SETTINGS PROFILE [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1] [, name2 [ON CLUSTER cluster_name2] ...] + [IN access_storage_type] [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [CONST|READONLY|WRITABLE|CHANGEABLE_IN_READONLY] | INHERIT 'profile_name'] [,...] ``` diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md index d168be63c36..11d4eae7bc8 100644 --- a/docs/en/sql-reference/statements/create/user.md +++ b/docs/en/sql-reference/statements/create/user.md @@ -14,6 +14,7 @@ CREATE USER [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1] [, name2 [ON CLUSTER cluster_name2] ...] [NOT IDENTIFIED | IDENTIFIED {[WITH {no_password | plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']} | {WITH ssl_certificate CN 'common_name'}] [HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] + [IN access_storage_type] [DEFAULT ROLE role [,...]] [DEFAULT DATABASE database | NONE] [GRANTEES {user | role | ANY | NONE} [,...] [EXCEPT {user | role} [,...]]] diff --git a/docs/en/sql-reference/statements/drop.md b/docs/en/sql-reference/statements/drop.md index b6208c2fd52..245fd22e57c 100644 --- a/docs/en/sql-reference/statements/drop.md +++ b/docs/en/sql-reference/statements/drop.md @@ -49,7 +49,7 @@ Deletes a user. Syntax: ``` sql -DROP USER [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +DROP USER [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type] ``` ## DROP ROLE @@ -59,7 +59,7 @@ Deletes a role. The deleted role is revoked from all the entities where it was a Syntax: ``` sql -DROP ROLE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +DROP ROLE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type] ``` ## DROP ROW POLICY @@ -69,7 +69,7 @@ Deletes a row policy. Deleted row policy is revoked from all the entities where Syntax: ``` sql -DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] [ON CLUSTER cluster_name] +DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] [ON CLUSTER cluster_name] [FROM access_storage_type] ``` ## DROP QUOTA @@ -79,7 +79,7 @@ Deletes a quota. The deleted quota is revoked from all the entities where it was Syntax: ``` sql -DROP QUOTA [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +DROP QUOTA [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type] ``` ## DROP SETTINGS PROFILE @@ -89,7 +89,7 @@ Deletes a settings profile. The deleted settings profile is revoked from all the Syntax: ``` sql -DROP [SETTINGS] PROFILE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] +DROP [SETTINGS] PROFILE [IF EXISTS] name [,...] [ON CLUSTER cluster_name] [FROM access_storage_type] ``` ## DROP VIEW diff --git a/docs/en/sql-reference/statements/move.md b/docs/en/sql-reference/statements/move.md new file mode 100644 index 00000000000..fac738ff711 --- /dev/null +++ b/docs/en/sql-reference/statements/move.md @@ -0,0 +1,32 @@ +--- +slug: /en/sql-reference/statements/move +sidebar_position: 54 +sidebar_label: MOVE +--- + +# MOVE access entity statement + +This statement allows to move an access entity from one access storage to another. + +Syntax: + +```sql +MOVE {USER, ROLE, QUOTA, SETTINGS PROFILE, ROW POLICY} name1 [, name2, ...] TO access_storage_type +``` + +Currently, there are five access storages in ClickHouse: + - `local_directory` + - `memory` + - `replicated` + - `users_xml` (ro) + - `ldap` (ro) + +Examples: + +```sql +MOVE USER test TO local_directory +``` + +```sql +MOVE ROLE test TO memory +``` diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 468c93c1116..ec25fd505f0 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -202,16 +202,21 @@ std::vector IAccessStorage::insert(const std::vector & mu std::vector IAccessStorage::insert(const std::vector & multiple_entities, const std::vector & ids, bool replace_if_exists, bool throw_if_exists) { - if (!ids.empty()) - assert(multiple_entities.size() == ids.size()); + assert(ids.empty() || (multiple_entities.size() == ids.size())); if (multiple_entities.empty()) return {}; if (multiple_entities.size() == 1) { - if (auto id = insert(multiple_entities[0], replace_if_exists, throw_if_exists)) - return {*id}; + UUID id; + if (!ids.empty()) + id = ids[0]; + else + id = generateRandomID(); + + if (insert(id, multiple_entities[0], replace_if_exists, throw_if_exists)) + return {id}; return {}; } @@ -229,7 +234,7 @@ std::vector IAccessStorage::insert(const std::vector & mu else id = generateRandomID(); - if (insertImpl(id, entity, replace_if_exists, throw_if_exists)) + if (insert(id, entity, replace_if_exists, throw_if_exists)) { successfully_inserted.push_back(entity); new_ids.push_back(id); diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index e6665349ae6..c6c9ede9e1c 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -230,8 +230,8 @@ StoragePtr MultipleAccessStorage::findExcludingStorage(AccessEntityType type, co void MultipleAccessStorage::moveAccessEntities(const std::vector & ids, const String & source_storage_name, const String & destination_storage_name) { - auto source_storage = findStorageByName(source_storage_name); - auto destination_storage = findStorageByName(destination_storage_name); + auto source_storage = getStorageByName(source_storage_name); + auto destination_storage = getStorageByName(destination_storage_name); auto to_move = source_storage->read(ids); source_storage->remove(ids); diff --git a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp index ac7ff4efcb6..49e90783a59 100644 --- a/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterMoveAccessEntityQuery.cpp @@ -28,12 +28,9 @@ BlockIO InterpreterMoveAccessEntityQuery::execute() std::vector ids; if (query.type == AccessEntityType::ROW_POLICY) - ids = access_control.find(query.type, query.row_policy_names->toStrings()); + ids = access_control.getIDs(query.type, query.row_policy_names->toStrings()); else - ids = access_control.find(query.type, query.names); - - if (ids.empty()) - return {}; + ids = access_control.getIDs(query.type, query.names); /// Validate that all entities are from the same storage. const auto source_storage = access_control.findStorage(ids.front()); diff --git a/tests/integration/test_multi_access_storage_role_management/test.py b/tests/integration/test_multi_access_storage_role_management/test.py index 4c2696158ec..77aea411394 100644 --- a/tests/integration/test_multi_access_storage_role_management/test.py +++ b/tests/integration/test_multi_access_storage_role_management/test.py @@ -76,6 +76,13 @@ def execute_test_for_access_type(access_type: str, system_table_name: str): with pytest.raises(QueryRuntimeException): node.query(f"MOVE {access_type} test6 TO users_xml") + node.query(f"DROP {access_type} test1") + node.query(f"DROP {access_type} test2") + node.query(f"DROP {access_type} test3") + node.query(f"DROP {access_type} test4") + node.query(f"DROP {access_type} test5") + node.query(f"DROP {access_type} test6") + def test_roles(): execute_test_for_access_type("ROLE", "roles") @@ -93,10 +100,6 @@ def test_quotas(): execute_test_for_access_type("QUOTA", "quotas") -def test_row_policies(): - execute_test_for_access_type("ROW POLICY", "row_policies") - - def test_role_from_different_storages(): node.query("CREATE ROLE default_role") node.query("GRANT SELECT ON system.* TO default_role") diff --git a/tests/integration/test_quota/test.py b/tests/integration/test_quota/test.py index 4f98b9a0d0d..cec14b0af73 100644 --- a/tests/integration/test_quota/test.py +++ b/tests/integration/test_quota/test.py @@ -105,7 +105,7 @@ def test_quota_from_users_xml(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, @@ -252,7 +252,7 @@ def test_simpliest_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[]", 0, @@ -326,7 +326,7 @@ def test_tracking_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -446,7 +446,7 @@ def test_exceed_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -523,7 +523,7 @@ def test_exceed_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -613,7 +613,7 @@ def test_add_remove_interval(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, @@ -675,7 +675,7 @@ def test_add_remove_interval(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952,63113904]", 0, @@ -824,7 +824,7 @@ def test_add_remove_interval(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, @@ -914,7 +914,7 @@ def test_add_remove_interval(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[]", 0, @@ -986,7 +986,7 @@ def test_add_remove_interval(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, @@ -1048,7 +1048,7 @@ def test_add_remove_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, @@ -1111,7 +1111,7 @@ def test_add_remove_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -1121,7 +1121,7 @@ def test_add_remove_quota(): [ "myQuota2", "4590510c-4d13-bf21-ec8a-c2187b092e73", - "users.xml", + "users_xml", "['client_key','user_name']", "[3600,2629746]", 0, @@ -1214,7 +1214,7 @@ def test_add_remove_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -1283,7 +1283,7 @@ def test_add_remove_quota(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -1346,7 +1346,7 @@ def test_reload_users_xml_by_timer(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", "[31556952]", 0, @@ -1385,7 +1385,7 @@ def test_reload_users_xml_by_timer(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", ["user_name"], "[31556952]", 0, @@ -1554,7 +1554,7 @@ def test_query_inserts(): [ "myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", - "users.xml", + "users_xml", "['user_name']", [31556952], 0, From c3c64a7dd50ee0f25dd94eb1d1b645e0352471ec Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 28 Jul 2023 11:40:05 +0000 Subject: [PATCH 057/777] Fix --- .../Impl/CustomSeparatedRowInputFormat.cpp | 5 ++++- .../Formats/RowInputFormatWithNamesAndTypes.cpp | 3 ++- ...ats_with_variable_number_of_columns.reference | 16 ++++++++++++++++ ...4_formats_with_variable_number_of_columns.sql | 4 ++++ 4 files changed, 26 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp index 8f8e12e3c2a..ff3d6d49199 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp @@ -139,7 +139,10 @@ void CustomSeparatedFormatReader::skipRowBetweenDelimiter() void CustomSeparatedFormatReader::skipField() { skipSpaces(); - skipFieldByEscapingRule(*buf, format_settings.custom.escaping_rule, format_settings); + if (format_settings.custom.escaping_rule == FormatSettings::EscapingRule::CSV) + readCSVFieldWithTwoPossibleDelimiters(*buf, format_settings.csv, format_settings.custom.field_delimiter, format_settings.custom.row_after_delimiter); + else + skipFieldByEscapingRule(*buf, format_settings.custom.escaping_rule, format_settings); } bool CustomSeparatedFormatReader::checkForEndOfRow() diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index cb5c11e2d3b..4000bd14ddc 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -220,7 +220,8 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE while (file_column < column_mapping->column_indexes_for_input_fields.size()) { const auto & rem_column_index = column_mapping->column_indexes_for_input_fields[file_column]; - columns[*rem_column_index]->insertDefault(); + if (rem_column_index) + columns[*rem_column_index]->insertDefault(); ++file_column; } break; diff --git a/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.reference b/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.reference index 39d24f2cbd2..e9ff548e05c 100644 --- a/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.reference +++ b/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.reference @@ -11,6 +11,10 @@ CSV 2 \N \N \N 3 3 +1 0 +2 0 +0 0 +3 0 TSV 1 1 2 0 @@ -24,6 +28,10 @@ TSV 2 \N \N \N 3 3 +1 0 +2 0 +0 0 +3 0 JSONCompactEachRow 1 1 2 0 @@ -37,6 +45,10 @@ JSONCompactEachRow 2 \N \N \N 3 3 +1 0 +2 0 +0 0 +3 0 CustomSeparated 1 1 2 0 @@ -50,3 +62,7 @@ CustomSeparated 2 \N \N \N 3 3 +1 0 +2 0 +0 0 +3 0 diff --git a/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.sql b/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.sql index c0a80bf2114..dea4c20db8a 100644 --- a/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.sql +++ b/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.sql @@ -2,17 +2,21 @@ select 'CSV'; select * from format(CSV, 'x UInt32, y UInt32', '1,1\n2\n\n3,3,3,3') settings input_format_csv_allow_variable_number_of_columns=1; select * from format(CSV, '1,1\n2\n\n3,3,3,3') settings input_format_csv_allow_variable_number_of_columns=1; select * from format(CSVWithNames, '"x","y"\n1,1\n2\n\n3,3,3,3') settings input_format_csv_allow_variable_number_of_columns=1; +select * from format(CSVWithNames, 'x UInt32, z UInt32', '"x","y"\n1,1\n2\n\n3,3,3,3') settings input_format_csv_allow_variable_number_of_columns=1; select 'TSV'; select * from format(TSV, 'x UInt32, y UInt32', '1\t1\n2\n\n3\t3\t3\t3') settings input_format_tsv_allow_variable_number_of_columns=1; select * from format(TSV, '1\t1\n2\n\n3\t3\t3\t3') settings input_format_tsv_allow_variable_number_of_columns=1; select * from format(TSVWithNames, 'x\ty\n1\t1\n2\n\n3\t3\t3\t3') settings input_format_tsv_allow_variable_number_of_columns=1; +select * from format(TSVWithNames, 'x UInt32, z UInt32', 'x\ty\n1\t1\n2\n\n3\t3\t3\t3') settings input_format_tsv_allow_variable_number_of_columns=1; select 'JSONCompactEachRow'; select * from format(JSONCompactEachRow, 'x UInt32, y UInt32', '[1,1]\n[2]\n[]\n[3,3,3,3]') settings input_format_json_compact_allow_variable_number_of_columns=1; select * from format(JSONCompactEachRow, '[1,1]\n[2]\n[]\n[3,3,3,3]') settings input_format_json_compact_allow_variable_number_of_columns=1; select * from format(JSONCompactEachRowWithNames, '["x","y"]\n[1,1]\n[2]\n[]\n[3,3,3,3]') settings input_format_json_compact_allow_variable_number_of_columns=1; +select * from format(JSONCompactEachRowWithNames, 'x UInt32, z UInt32', '["x","y"]\n[1,1]\n[2]\n[]\n[3,3,3,3]') settings input_format_json_compact_allow_variable_number_of_columns=1; select 'CustomSeparated'; set format_custom_escaping_rule='CSV', format_custom_field_delimiter='', format_custom_row_before_delimiter='', format_custom_row_after_delimiter='', format_custom_row_between_delimiter='', format_custom_result_before_delimiter='', format_custom_result_after_delimiter=''; select * from format(CustomSeparated, 'x UInt32, y UInt32', '1123333') settings input_format_custom_allow_variable_number_of_columns=1; select * from format(CustomSeparated, '1123333') settings input_format_custom_allow_variable_number_of_columns=1; select * from format(CustomSeparatedWithNames, '"x""y"1123333') settings input_format_custom_allow_variable_number_of_columns=1; +select * from format(CustomSeparatedWithNames, 'x UInt32, z UInt32', '"x""y"1123333') settings input_format_custom_allow_variable_number_of_columns=1; From f3a31c20fe8fd4a0ba57586bd36828af8581cf5d Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Jul 2023 10:53:02 -0400 Subject: [PATCH 058/777] Update test_row_policy.py --- .../test_row_policy.py | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_disabled_access_control_improvements/test_row_policy.py b/tests/integration/test_disabled_access_control_improvements/test_row_policy.py index 509b4de1a37..b620e88e7eb 100644 --- a/tests/integration/test_disabled_access_control_improvements/test_row_policy.py +++ b/tests/integration/test_disabled_access_control_improvements/test_row_policy.py @@ -76,7 +76,7 @@ def test_introspection(): "mydb", "filtered_table1", "6068883a-0e9d-f802-7e22-0144f8e66d3c", - "users.xml", + "users_xml", "1", 0, 0, @@ -89,7 +89,7 @@ def test_introspection(): "mydb", "filtered_table2", "c019e957-c60b-d54e-cc52-7c90dac5fb01", - "users.xml", + "users_xml", "1", 0, 0, @@ -102,7 +102,7 @@ def test_introspection(): "mydb", "filtered_table3", "4cb080d0-44e8-dbef-6026-346655143628", - "users.xml", + "users_xml", "1", 0, 0, @@ -115,7 +115,7 @@ def test_introspection(): "mydb", "local", "5b23c389-7e18-06bf-a6bc-dd1afbbc0a97", - "users.xml", + "users_xml", "a = 1", 0, 0, @@ -128,7 +128,7 @@ def test_introspection(): "mydb", "filtered_table1", "9e8a8f62-4965-2b5e-8599-57c7b99b3549", - "users.xml", + "users_xml", "a = 1", 0, 0, @@ -141,7 +141,7 @@ def test_introspection(): "mydb", "filtered_table2", "cffae79d-b9bf-a2ef-b798-019c18470b25", - "users.xml", + "users_xml", "a + b < 1 or c - d > 5", 0, 0, @@ -154,7 +154,7 @@ def test_introspection(): "mydb", "filtered_table3", "12fc5cef-e3da-3940-ec79-d8be3911f42b", - "users.xml", + "users_xml", "c = 1", 0, 0, @@ -167,7 +167,7 @@ def test_introspection(): "mydb", "local", "cdacaeb5-1d97-f99d-2bb0-4574f290629c", - "users.xml", + "users_xml", "1", 0, 0, From f42cfddee7966496836da44e26d27fe208d43ac9 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Jul 2023 20:28:56 -0400 Subject: [PATCH 059/777] Update test.py --- tests/integration/test_row_policy/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index 1933823f5d2..acf544c98aa 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -420,7 +420,7 @@ def test_introspection(): "mydb", "local", "5b23c389-7e18-06bf-a6bc-dd1afbbc0a97", - "users.xml", + "users_xml", "a = 1", 0, 0, @@ -433,7 +433,7 @@ def test_introspection(): "mydb", "filtered_table1", "9e8a8f62-4965-2b5e-8599-57c7b99b3549", - "users.xml", + "users_xml", "a = 1", 0, 0, @@ -446,7 +446,7 @@ def test_introspection(): "mydb", "filtered_table2", "cffae79d-b9bf-a2ef-b798-019c18470b25", - "users.xml", + "users_xml", "a + b < 1 or c - d > 5", 0, 0, @@ -459,7 +459,7 @@ def test_introspection(): "mydb", "filtered_table3", "12fc5cef-e3da-3940-ec79-d8be3911f42b", - "users.xml", + "users_xml", "c = 1", 0, 0, From 1ee1ae120e921a355a752b332aae25b18b1cf89d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 29 Jul 2023 02:03:21 +0000 Subject: [PATCH 060/777] add modulo, intDiv, intDivOrZero for tuple --- src/Functions/FunctionBinaryArithmetic.h | 24 ++++++++++++++++--- src/Functions/IsOperation.h | 2 +- src/Functions/vectorFunctions.cpp | 21 ++++++++++++++++ .../0_stateless/02841_tuple_modulo.reference | 4 ++++ .../0_stateless/02841_tuple_modulo.sql | 4 ++++ 5 files changed, 51 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02841_tuple_modulo.reference create mode 100644 tests/queries/0_stateless/02841_tuple_modulo.sql diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index bf3b33d13ff..408a16236e8 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -742,6 +742,9 @@ class FunctionBinaryArithmetic : public IFunction static constexpr bool is_multiply = IsOperation::multiply; static constexpr bool is_division = IsOperation::division; static constexpr bool is_bit_hamming_distance = IsOperation::bit_hamming_distance; + static constexpr bool is_modulo = IsOperation::modulo; + static constexpr bool is_div_int = IsOperation::div_int; + static constexpr bool is_div_int_or_zero = IsOperation::div_int_or_zero; ContextPtr context; bool check_decimal_overflow = true; @@ -951,13 +954,28 @@ class FunctionBinaryArithmetic : public IFunction "argument of numeric type cannot be first", name); std::string function_name; - if (is_multiply) + if constexpr (is_multiply) { function_name = "tupleMultiplyByNumber"; } - else + else // is_division { - function_name = "tupleDivideByNumber"; + if constexpr (is_modulo) + { + function_name = "tupleModuloByNumber"; + } + else if constexpr (is_div_int) + { + function_name = "tupleIntDivByNumber"; + } + else if constexpr (is_div_int_or_zero) + { + function_name = "tupleIntDivOrZeroByNumber"; + } + else + { + function_name = "tupleDivideByNumber"; + } } return FunctionFactory::instance().get(function_name, context); diff --git a/src/Functions/IsOperation.h b/src/Functions/IsOperation.h index 0c54901579e..8ea53c865ce 100644 --- a/src/Functions/IsOperation.h +++ b/src/Functions/IsOperation.h @@ -60,7 +60,7 @@ struct IsOperation static constexpr bool bit_hamming_distance = IsSameOperation::value; - static constexpr bool division = div_floating || div_int || div_int_or_zero; + static constexpr bool division = div_floating || div_int || div_int_or_zero || modulo; static constexpr bool allow_decimal = plus || minus || multiply || division || least || greatest; }; diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index d53d39e2f3b..35ba49e4545 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -23,6 +23,9 @@ struct PlusName { static constexpr auto name = "plus"; }; struct MinusName { static constexpr auto name = "minus"; }; struct MultiplyName { static constexpr auto name = "multiply"; }; struct DivideName { static constexpr auto name = "divide"; }; +struct ModuloName { static constexpr auto name = "modulo"; }; +struct IntDivName { static constexpr auto name = "intDiv"; }; +struct IntDivOrZeroName { static constexpr auto name = "intDivOrZero"; }; struct L1Label { static constexpr auto name = "1"; }; struct L2Label { static constexpr auto name = "2"; }; @@ -141,6 +144,12 @@ using FunctionTupleMultiply = FunctionTupleOperator; using FunctionTupleDivide = FunctionTupleOperator; +using FunctionTupleModulo = FunctionTupleOperator; + +using FunctionTupleIntDiv = FunctionTupleOperator; + +using FunctionTupleIntDivOrZero = FunctionTupleOperator; + class FunctionTupleNegate : public ITupleFunction { public: @@ -297,6 +306,12 @@ using FunctionTupleMultiplyByNumber = FunctionTupleOperatorByNumber; +using FunctionTupleModuloByNumber = FunctionTupleOperatorByNumber; + +using FunctionTupleIntDivByNumber = FunctionTupleOperatorByNumber; + +using FunctionTupleIntDivOrZeroByNumber = FunctionTupleOperatorByNumber; + class FunctionDotProduct : public ITupleFunction { public: @@ -1563,6 +1578,9 @@ REGISTER_FUNCTION(VectorFunctions) factory.registerAlias("vectorDifference", FunctionTupleMinus::name, FunctionFactory::CaseInsensitive); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(FunctionDocumentation @@ -1626,6 +1644,9 @@ If the types of the first interval (or the interval in the tuple) and the second factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerAlias("scalarProduct", TupleOrArrayFunctionDotProduct::name, FunctionFactory::CaseInsensitive); diff --git a/tests/queries/0_stateless/02841_tuple_modulo.reference b/tests/queries/0_stateless/02841_tuple_modulo.reference new file mode 100644 index 00000000000..6e6f07d0683 --- /dev/null +++ b/tests/queries/0_stateless/02841_tuple_modulo.reference @@ -0,0 +1,4 @@ +(1,0) +(2,2) +(2,2) +(0,0) diff --git a/tests/queries/0_stateless/02841_tuple_modulo.sql b/tests/queries/0_stateless/02841_tuple_modulo.sql new file mode 100644 index 00000000000..56bacf87967 --- /dev/null +++ b/tests/queries/0_stateless/02841_tuple_modulo.sql @@ -0,0 +1,4 @@ +SELECT (5,4) % 2; +SELECT intDiv((5,4), 2); +SELECT intDivOrZero((5,4), 2); +SELECT intDivOrZero((5,4), 0); From 6f2404b79209bf332e54e83b86eff2f677bbd2bc Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 28 Jul 2023 22:18:53 -0400 Subject: [PATCH 061/777] Additional info in exception --- src/Access/MultipleAccessStorage.cpp | 21 ++++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index c6c9ede9e1c..0550c140c17 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -234,17 +234,32 @@ void MultipleAccessStorage::moveAccessEntities(const std::vector & ids, co auto destination_storage = getStorageByName(destination_storage_name); auto to_move = source_storage->read(ids); - source_storage->remove(ids); + bool need_rollback = false; try { + source_storage->remove(ids); + need_rollback = true; destination_storage->insert(to_move, ids); } catch (Exception & e) { - e.addMessage("while moving access entities"); + String message; + + bool need_comma = false; + for (const auto & entity : to_move) + { + if (std::exchange(need_comma, true)) + message += ", "; + + message += entity->formatTypeWithName(); + } + + e.addMessage("while moving {} from {} to {}", message, source_storage_name, destination_storage_name); + + if (need_rollback) + source_storage->insert(to_move, ids); - source_storage->insert(to_move, ids); throw; } } From a64d6b11f29c3eac5eedf4e48fc118510d4e2a01 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 29 Jul 2023 04:55:19 +0000 Subject: [PATCH 062/777] add functions to undocumented --- .../02415_all_new_functions_must_be_documented.reference | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 61a2e4e9f02..b57372ffa8d 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -885,7 +885,13 @@ tupleDivide tupleDivideByNumber tupleElement tupleHammingDistance +tupleIntDiv +tupleIntDivByNumber +tupleIntDivOrZero +tupleIntDivOrZeroByNumber tupleMinus +tupleModulo +tupleModuloByNumber tupleMultiply tupleMultiplyByNumber tupleNegate From 12069b2cff6c4ae77cc674f02ce4e04987c9515b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Jul 2023 23:25:44 +0200 Subject: [PATCH 063/777] Allow yaml configs in clickhouse-client --- src/Common/Config/configReadClient.cpp | 51 ++++++++++++++++++-------- 1 file changed, 36 insertions(+), 15 deletions(-) diff --git a/src/Common/Config/configReadClient.cpp b/src/Common/Config/configReadClient.cpp index 44d338c07af..fea055618bb 100644 --- a/src/Common/Config/configReadClient.cpp +++ b/src/Common/Config/configReadClient.cpp @@ -6,36 +6,57 @@ #include #include + namespace fs = std::filesystem; namespace DB { -bool safeFsExists(const String & path) -{ - std::error_code ec; - return fs::exists(path, ec); -} - bool configReadClient(Poco::Util::LayeredConfiguration & config, const std::string & home_path) { std::string config_path; - if (config.has("config-file")) - config_path = config.getString("config-file"); - else if (safeFsExists("./clickhouse-client.xml")) - config_path = "./clickhouse-client.xml"; - else if (!home_path.empty() && safeFsExists(home_path + "/.clickhouse-client/config.xml")) - config_path = home_path + "/.clickhouse-client/config.xml"; - else if (safeFsExists("/etc/clickhouse-client/config.xml")) - config_path = "/etc/clickhouse-client/config.xml"; - if (!config_path.empty()) + bool found = false; + if (config.has("config-file")) + { + found = true; + config_path = config.getString("config-file"); + } + else + { + std::vector names; + names.emplace_back("./clickhouse-client"); + if (!home_path.empty()) + names.emplace_back(home_path + "/.clickhouse-client/config"); + names.emplace_back("/etc/clickhouse-client/config"); + + for (const auto & name : names) + { + for (const auto & extension : {".xml", ".yaml", ".yml"}) + { + config_path = name + extension; + + std::error_code ec; + if (fs::exists(config_path, ec)) + { + found = true; + break; + } + } + if (found) + break; + } + } + + if (found) { ConfigProcessor config_processor(config_path); auto loaded_config = config_processor.loadConfig(); config.add(loaded_config.configuration); return true; } + return false; } + } From fdfe64f1b0330522180811b83e8db8436839c829 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 31 Jul 2023 12:07:50 +0800 Subject: [PATCH 064/777] change as request --- .../functions/string-functions.md | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index b1ffb32e7cd..fff37257917 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -706,7 +706,15 @@ endsWithUTF8(str, suffix) **Example** ``` sql -SELECT endsWithUTF8('中国', '国'); +SELECT endsWithUTF8('中国', '\xbd'), endsWith('中国', '\xbd') +``` + +Result: + +```result +┌─endsWithUTF8('中国', '½')─┬─endsWith('中国', '½')─┐ +│ 0 │ 1 │ +└──────────────────────────┴──────────────────────┘ ``` ## startsWith @@ -733,9 +741,16 @@ Returns whether string `str` starts with `prefix`, the difference between `start **Example** ``` sql -SELECT startsWithUTF8('中国', '中'); +SELECT startsWithUTF8('中国', '\xe4'), startsWith('中国', '\xe4') ``` +Result: + +```result +┌─startsWithUTF8('中国', '⥩─┬─startsWith('中国', '⥩─┐ +│ 0 │ 1 │ +└────────────────────────────┴────────────────────────┘ +``` ## trim From fb8502ba7671f75cb420ab5937cced224f5be7c6 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 31 Jul 2023 12:20:27 +0000 Subject: [PATCH 065/777] do not test upper bounds for throttlers --- tests/integration/test_throttling/test.py | 45 ++++++++++++----------- 1 file changed, 23 insertions(+), 22 deletions(-) diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index 62640394a85..526c34ac916 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -117,7 +117,8 @@ def assert_took(took, should_took): # we need to decrease the lower limit because the server limits could # be enforced by throttling some server background IO instead of query IO # and we have no control over it - assert took >= should_took[0] * 0.85 and took < should_took[1] + # Note that throttler does not apply any restrictions on upper bound, so we can only tell how much time required "at least", not "at most" + assert took >= should_took * 0.85 @pytest.mark.parametrize( @@ -132,7 +133,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_local_throttling", ), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds @@ -142,7 +143,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (7, 14), + 7, id="user_local_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -152,7 +153,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_local_throttling", ), # @@ -164,7 +165,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_remote_to_local_throttling", ), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds @@ -184,7 +185,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_remote_to_local_throttling", ), # @@ -196,7 +197,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_remote_to_remote_throttling", ), # No throttling for S3-to-S3, uses native copy @@ -206,7 +207,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (0, 3), + 0, id="user_remote_to_remote_throttling", ), # No throttling for S3-to-S3, uses native copy @@ -216,7 +217,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (0, 3), + 0, id="server_remote_to_remote_throttling", ), # @@ -233,7 +234,7 @@ def assert_took(took, should_took): None, None, None, - (0, 3), + 0, id="no_local_to_remote_throttling", ), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds, but for S3Client it is 2x more @@ -243,7 +244,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (7 * 3, 7 * 4 - 1), + 7 * 3, id="user_local_to_remote_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds, but for S3Client it is 2x more @@ -253,7 +254,7 @@ def assert_took(took, should_took): "server", "max_backup_bandwidth_for_server", "2M", - (3 * 3, 3 * 5), + 3 * 3, id="server_local_to_remote_throttling", ), ], @@ -306,7 +307,7 @@ def test_backup_throttling_override(): "user", "max_local_read_bandwidth", "1M", - (7, 14), + 7, id="user_local_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -315,7 +316,7 @@ def test_backup_throttling_override(): "server", "max_local_read_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_local_throttling", ), # @@ -328,7 +329,7 @@ def test_backup_throttling_override(): "user", "max_remote_read_network_bandwidth", "1M", - (7, 14), + 7, id="user_remote_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -337,7 +338,7 @@ def test_backup_throttling_override(): "server", "max_remote_read_network_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_remote_throttling", ), ], @@ -368,7 +369,7 @@ def test_read_throttling(policy, mode, setting, value, should_took): "user", "max_local_write_bandwidth", "1M", - (7, 14), + 7, id="local_user_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -377,29 +378,29 @@ def test_read_throttling(policy, mode, setting, value, should_took): "server", "max_local_write_bandwidth_for_server", "2M", - (3, 7), + 3, id="local_server_throttling", ), # # Remote # pytest.param("s3", None, None, None, (0, 3), id="no_remote_throttling"), - # writeing 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds + # writing 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "s3", "user", "max_remote_write_network_bandwidth", "1M", - (7, 14), + 7, id="user_remote_throttling", ), - # writeing 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds + # writing 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds pytest.param( "s3", "server", "max_remote_write_network_bandwidth_for_server", "2M", - (3, 7), + 3, id="server_remote_throttling", ), ], From e869adf64573f8e2732c9aa65192ea1b745b766e Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 31 Jul 2023 06:48:50 -0700 Subject: [PATCH 066/777] Improve function naming --- src/Analyzer/IQueryTreeNode.cpp | 2 +- src/Client/QueryFuzzer.cpp | 2 +- src/Common/SipHash.h | 2 +- src/Parsers/IAST.cpp | 2 +- .../Formats/Impl/ConstantExpressionTemplate.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp | 2 +- 7 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Analyzer/IQueryTreeNode.cpp b/src/Analyzer/IQueryTreeNode.cpp index 7e5d778f1c1..babc76f29d6 100644 --- a/src/Analyzer/IQueryTreeNode.cpp +++ b/src/Analyzer/IQueryTreeNode.cpp @@ -229,7 +229,7 @@ IQueryTreeNode::Hash IQueryTreeNode::getTreeHash() const } } - return getSipHash128AsLoHi(hash_state); + return getSipHash128AsPair(hash_state); } QueryTreeNodePtr IQueryTreeNode::clone() const diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index f20c869e119..86cedf65345 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -521,7 +521,7 @@ void QueryFuzzer::fuzzCreateQuery(ASTCreateQuery & create) if (create.storage) create.storage->updateTreeHash(sip_hash); - const auto hash = getSipHash128AsLoHi(sip_hash); + const auto hash = getSipHash128AsPair(sip_hash); /// Save only tables with unique definition. if (created_tables_hashes.insert(hash).second) diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index 22c2c61abdc..43d620adde0 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -249,7 +249,7 @@ inline std::array getSipHash128AsArray(SipHash & sip_hash) return arr; } -inline CityHash_v1_0_2::uint128 getSipHash128AsLoHi(SipHash & sip_hash) +inline CityHash_v1_0_2::uint128 getSipHash128AsPair(SipHash & sip_hash) { CityHash_v1_0_2::uint128 result; sip_hash.get128(result.low64, result.high64); diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index b3cb5a65098..a494a528cd2 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -118,7 +118,7 @@ IAST::Hash IAST::getTreeHash() const { SipHash hash_state; updateTreeHash(hash_state); - return getSipHash128AsLoHi(hash_state); + return getSipHash128AsPair(hash_state); } diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 92bb1a15693..fe6fb42d0a0 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -402,7 +402,7 @@ size_t ConstantExpressionTemplate::TemplateStructure::getTemplateHash(const ASTP /// Allows distinguish expression in the last column in Values format hash_state.update(salt); - const auto res128 = getSipHash128AsLoHi(hash_state); + const auto res128 = getSipHash128AsPair(hash_state); size_t res = 0; boost::hash_combine(res, res128.low64); boost::hash_combine(res, res128.high64); diff --git a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp index d09cae7870f..55484a2c8cb 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartChecksum.cpp @@ -323,7 +323,7 @@ MergeTreeDataPartChecksums::Checksum::uint128 MergeTreeDataPartChecksums::getTot hash_of_all_files.update(checksum.file_hash); } - return getSipHash128AsLoHi(hash_of_all_files); + return getSipHash128AsPair(hash_of_all_files); } void MinimalisticDataPartChecksums::serialize(WriteBuffer & to) const @@ -413,9 +413,9 @@ void MinimalisticDataPartChecksums::computeTotalChecksums(const MergeTreeDataPar } } - hash_of_all_files = getSipHash128AsLoHi(hash_of_all_files_state); - hash_of_uncompressed_files = getSipHash128AsLoHi(hash_of_uncompressed_files_state); - uncompressed_hash_of_compressed_files = getSipHash128AsLoHi(uncompressed_hash_of_compressed_files_state); + hash_of_all_files = getSipHash128AsPair(hash_of_all_files_state); + hash_of_uncompressed_files = getSipHash128AsPair(hash_of_uncompressed_files_state); + uncompressed_hash_of_compressed_files = getSipHash128AsPair(uncompressed_hash_of_compressed_files_state); } String MinimalisticDataPartChecksums::getSerializedString(const MergeTreeDataPartChecksums & full_checksums, bool minimalistic) diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index a8d38a1bff8..179871c9ad9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -167,7 +167,7 @@ IMergeTreeDataPart::Checksum MergeTreeDataPartInMemory::calculateBlockChecksum() column.column->updateHashFast(hash); checksum.uncompressed_size = block.bytes(); - checksum.uncompressed_hash = getSipHash128AsLoHi(hash); + checksum.uncompressed_hash = getSipHash128AsPair(hash); return checksum; } From 8b0f52f4114099b8a3fc0e37caa8b3f9a99cc054 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 31 Jul 2023 15:04:30 +0000 Subject: [PATCH 067/777] 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 5570333c7c13773342c4190a6b829d00a7a3f9a3 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 31 Jul 2023 15:00:32 -0700 Subject: [PATCH 068/777] Add support for arbitrary endianness transformation --- src/Common/TransformEndianness.hpp | 36 +++++++++---------- .../Serializations/SerializationNumber.cpp | 2 +- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index fe43861f66f..05f7778a12e 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -9,19 +9,19 @@ namespace DB { -template +template requires std::is_integral_v inline void transformEndianness(T & value) { - if constexpr (endian != std::endian::native) + if constexpr (ToEndian != FromEndian) value = std::byteswap(value); } -template +template requires is_big_int_v inline void transformEndianness(T & x) { - if constexpr (std::endian::native != endian) + if constexpr (ToEndian != FromEndian) { auto & items = x.items; std::transform(std::begin(items), std::end(items), std::begin(items), [](auto & item) { return std::byteswap(item); }); @@ -29,49 +29,49 @@ inline void transformEndianness(T & x) } } -template +template requires is_decimal inline void transformEndianness(T & x) { - transformEndianness(x.value); + transformEndianness(x.value); } -template +template requires std::is_floating_point_v inline void transformEndianness(T & value) { - if constexpr (std::endian::native != endian) + if constexpr (ToEndian != FromEndian) { auto * start = reinterpret_cast(&value); std::reverse(start, start + sizeof(T)); } } -template +template requires std::is_scoped_enum_v inline void transformEndianness(T & x) { using UnderlyingType = std::underlying_type_t; - transformEndianness(reinterpret_cast(x)); + transformEndianness(reinterpret_cast(x)); } -template +template inline void transformEndianness(std::pair & pair) { - transformEndianness(pair.first); - transformEndianness(pair.second); + transformEndianness(pair.first); + transformEndianness(pair.second); } -template +template inline void transformEndianness(StrongTypedef & x) { - transformEndianness(x.toUnderType()); + transformEndianness(x.toUnderType()); } -template +template inline void transformEndianness(CityHash_v1_0_2::uint128 & x) { - transformEndianness(x.low64); - transformEndianness(x.high64); + transformEndianness(x.low64); + transformEndianness(x.high64); } } diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index ca2616f4582..6b9fd9c3982 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -169,7 +169,7 @@ void SerializationNumber::deserializeBinaryBulk(IColumn & column, ReadBuffer x.resize(initial_size + size / sizeof(typename ColumnVector::ValueType)); if constexpr (std::endian::native == std::endian::big && sizeof(T) >= 2) - std::ranges::for_each(x | std::views::drop(initial_size), [](auto & i) { transformEndianness(i); }); + std::ranges::for_each(x | std::views::drop(initial_size), [](auto & i) { transformEndianness(i); }); } template class SerializationNumber; From a0ff04e0e7cc7bd911d80a01aee34846ec26bb3c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 08:03:09 +0200 Subject: [PATCH 069/777] Add a test --- .../02834_client_yaml_configs.reference | 3 +++ .../0_stateless/02834_client_yaml_configs.sh | 21 +++++++++++++++++++ 2 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/02834_client_yaml_configs.reference create mode 100755 tests/queries/0_stateless/02834_client_yaml_configs.sh diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.reference b/tests/queries/0_stateless/02834_client_yaml_configs.reference new file mode 100644 index 00000000000..302360f2570 --- /dev/null +++ b/tests/queries/0_stateless/02834_client_yaml_configs.reference @@ -0,0 +1,3 @@ +31337 +31337 +31337 diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh new file mode 100755 index 00000000000..f17186328b4 --- /dev/null +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +pushd "${CLICKHOUSE_TMP}" > /dev/null + +echo "max_block_size: 31337" > clickhouse-client.yaml +${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" +rm clickhouse-client.yaml + +echo "max_block_size: 31337" > clickhouse-client.yml +${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" +rm clickhouse-client.yml + +echo "31337" > clickhouse-client.xml +${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" +rm clickhouse-client.xml + +popd > /dev/null From 628786e7eb6a55d5f5b2881a0fc34c3edcf7d0e8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 10:52:06 +0300 Subject: [PATCH 070/777] Update 02834_client_yaml_configs.sh --- tests/queries/0_stateless/02834_client_yaml_configs.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh index f17186328b4..5a6d26808a3 100755 --- a/tests/queries/0_stateless/02834_client_yaml_configs.sh +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 8b37abfa598b80bf9a1281d519f09d953e18523e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 11:16:00 +0300 Subject: [PATCH 071/777] Update 02834_client_yaml_configs.sh --- tests/queries/0_stateless/02834_client_yaml_configs.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh index 5a6d26808a3..dbb40d33e0a 100755 --- a/tests/queries/0_stateless/02834_client_yaml_configs.sh +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -pushd "${CLICKHOUSE_TMP}" > /dev/null +pushd "${CLICKHOUSE_TMP}" || exit > /dev/null echo "max_block_size: 31337" > clickhouse-client.yaml ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" @@ -19,4 +19,4 @@ echo "31337" > clickho ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" rm clickhouse-client.xml -popd > /dev/null +popd || exit > /dev/null From 513d3fd3c1c1484cc908987f135f153f59d36405 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 1 Aug 2023 09:15:35 +0000 Subject: [PATCH 072/777] fix more places --- tests/integration/test_throttling/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index 526c34ac916..91b472513c7 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -175,7 +175,7 @@ def assert_took(took, should_took): "user", "max_backup_bandwidth", "1M", - (7, 14), + 7, id="user_remote_to_local_throttling", ), # reading 1e6*8 bytes with 2M default bandwith should take (8-2)/2=3 seconds @@ -291,7 +291,7 @@ def test_backup_throttling_override(): }, ) # reading 1e6*8 bytes with 500Ki default bandwith should take (8-0.5)/0.5=15 seconds - assert_took(took, (15, 20)) + assert_took(took, 15) @pytest.mark.parametrize( From 135790c0d6caed1075399cab69a5036c71860cd8 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 1 Aug 2023 09:44:08 +0000 Subject: [PATCH 073/777] more fixes --- tests/integration/test_throttling/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_throttling/test.py b/tests/integration/test_throttling/test.py index 91b472513c7..a27bb472ea8 100644 --- a/tests/integration/test_throttling/test.py +++ b/tests/integration/test_throttling/test.py @@ -300,7 +300,7 @@ def test_backup_throttling_override(): # # Local # - pytest.param("default", None, None, None, (0, 3), id="no_local_throttling"), + pytest.param("default", None, None, None, 0, id="no_local_throttling"), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "default", @@ -322,7 +322,7 @@ def test_backup_throttling_override(): # # Remote # - pytest.param("s3", None, None, None, (0, 3), id="no_remote_throttling"), + pytest.param("s3", None, None, None, 0, id="no_remote_throttling"), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "s3", @@ -362,7 +362,7 @@ def test_read_throttling(policy, mode, setting, value, should_took): # # Local # - pytest.param("default", None, None, None, (0, 3), id="no_local_throttling"), + pytest.param("default", None, None, None, 0, id="no_local_throttling"), # reading 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "default", @@ -384,7 +384,7 @@ def test_read_throttling(policy, mode, setting, value, should_took): # # Remote # - pytest.param("s3", None, None, None, (0, 3), id="no_remote_throttling"), + pytest.param("s3", None, None, None, 0, id="no_remote_throttling"), # writing 1e6*8 bytes with 1M default bandwith should take (8-1)/1=7 seconds pytest.param( "s3", From 8646eefc9cfeedc840c14bc7571c0c6490679a4e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 17:05:24 +0300 Subject: [PATCH 074/777] Update docker_test.py --- tests/ci/docker_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 8542de412d2..bdbee92d2d3 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -43,7 +43,7 @@ class TestDockerImageCheck(unittest.TestCase): di.DockerImage( "docker/test/sqltest", "clickhouse/sqltest", - True, + False, "clickhouse/test-base", # type: ignore ), di.DockerImage( From 07790ab485012f844a7efafa1729ec901df79c0b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 17:40:55 +0300 Subject: [PATCH 075/777] Update tests/queries/0_stateless/02834_client_yaml_configs.sh MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- tests/queries/0_stateless/02834_client_yaml_configs.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh index dbb40d33e0a..d1c5a40c04f 100755 --- a/tests/queries/0_stateless/02834_client_yaml_configs.sh +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -11,11 +11,11 @@ echo "max_block_size: 31337" > clickhouse-client.yaml ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" rm clickhouse-client.yaml -echo "max_block_size: 31337" > clickhouse-client.yml +echo "max_block_size: 31338" > clickhouse-client.yml ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" rm clickhouse-client.yml -echo "31337" > clickhouse-client.xml +echo "31339" > clickhouse-client.xml ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" rm clickhouse-client.xml From 93d766ba74f22f7de2dbbc3f196be06af91614d3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 17:41:00 +0300 Subject: [PATCH 076/777] Update tests/queries/0_stateless/02834_client_yaml_configs.reference MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- tests/queries/0_stateless/02834_client_yaml_configs.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.reference b/tests/queries/0_stateless/02834_client_yaml_configs.reference index 302360f2570..b2eddb19e52 100644 --- a/tests/queries/0_stateless/02834_client_yaml_configs.reference +++ b/tests/queries/0_stateless/02834_client_yaml_configs.reference @@ -1,3 +1,3 @@ 31337 -31337 -31337 +31338 +31339 From 5cc9600bbf718374ea8ca51d697d03328ac1e921 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Jul 2023 18:10:56 +0200 Subject: [PATCH 077/777] 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 17c4abce10c03bafc5316cde3f80a163fa56a944 Mon Sep 17 00:00:00 2001 From: Kenji Noguchi Date: Tue, 1 Aug 2023 10:44:43 -0700 Subject: [PATCH 078/777] CVE-2016-2183: disable 3DES --- base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h | 6 +++--- base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h | 2 +- base/poco/NetSSL_OpenSSL/src/Context.cpp | 2 +- .../operations/server-configuration-parameters/settings.md | 2 +- .../operations/server-configuration-parameters/settings.md | 2 +- .../operations/server-configuration-parameters/settings.md | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h index 65917ac9dd4..c19eecf5c73 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/Context.h @@ -146,7 +146,7 @@ namespace Net std::string cipherList; /// Specifies the supported ciphers in OpenSSL notation. - /// Defaults to "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH". + /// Defaults to "ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH". std::string dhParamsFile; /// Specifies a file containing Diffie-Hellman parameters. @@ -172,7 +172,7 @@ namespace Net VerificationMode verificationMode = VERIFY_RELAXED, int verificationDepth = 9, bool loadDefaultCAs = false, - const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH"); + const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH"); /// Creates a Context. /// /// * usage specifies whether the context is used by a client or server. @@ -200,7 +200,7 @@ namespace Net VerificationMode verificationMode = VERIFY_RELAXED, int verificationDepth = 9, bool loadDefaultCAs = false, - const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH"); + const std::string & cipherList = "ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH"); /// Creates a Context. /// /// * usage specifies whether the context is used by a client or server. diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h index 21a1ed685e5..e4037c87927 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h @@ -76,7 +76,7 @@ namespace Net /// none|relaxed|strict|once /// 1..9 /// true|false - /// ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH + /// ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH /// true|false /// /// KeyFileHandler diff --git a/base/poco/NetSSL_OpenSSL/src/Context.cpp b/base/poco/NetSSL_OpenSSL/src/Context.cpp index ca220c40a33..d0bab902b89 100644 --- a/base/poco/NetSSL_OpenSSL/src/Context.cpp +++ b/base/poco/NetSSL_OpenSSL/src/Context.cpp @@ -41,7 +41,7 @@ Context::Params::Params(): verificationMode(VERIFY_RELAXED), verificationDepth(9), loadDefaultCAs(false), - cipherList("ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH") + cipherList("ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH") { } diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 5187ccce789..a7637082496 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1640,7 +1640,7 @@ Keys for server/client settings: - verificationMode (default: relaxed) – The method for checking the node’s certificates. Details are in the description of the [Context](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) class. Possible values: `none`, `relaxed`, `strict`, `once`. - verificationDepth (default: 9) – The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value. - loadDefaultCAFile (default: true) – Wether built-in CA certificates for OpenSSL will be used. ClickHouse assumes that builtin CA certificates are in the file `/etc/ssl/cert.pem` (resp. the directory `/etc/ssl/certs`) or in file (resp. directory) specified by the environment variable `SSL_CERT_FILE` (resp. `SSL_CERT_DIR`). -- cipherList (default: `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`) - Supported OpenSSL encryptions. +- cipherList (default: `ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH`) - Supported OpenSSL encryptions. - cacheSessions (default: false) – Enables or disables caching sessions. Must be used in combination with `sessionIdContext`. Acceptable values: `true`, `false`. - sessionIdContext (default: `${application.name}`) – A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed `SSL_MAX_SSL_SESSION_ID_LENGTH`. This parameter is always recommended since it helps avoid problems both if the server caches the session and if the client requested caching. Default value: `${application.name}`. - sessionCacheSize (default: [1024\*20](https://github.com/ClickHouse/boringssl/blob/master/include/openssl/ssl.h#L1978)) – The maximum number of sessions that the server caches. A value of 0 means unlimited sessions. diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 81a696bcfc1..7b026244624 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1106,7 +1106,7 @@ ClickHouse использует потоки из глобального пул - verificationMode - Способ проверки сертификатов узла. Подробности находятся в описании класса [Context](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h). Допустимые значения: `none`, `relaxed`, `strict`, `once`. - verificationDepth - Максимальная длина верификационной цепи. Верификация завершится ошибкой, если длина цепи сертификатов превысит установленное значение. - loadDefaultCAFile - Признак того, что будут использоваться встроенные CA-сертификаты для OpenSSL. Допустимые значения: `true`, `false`. \| -- cipherList - Поддерживаемые OpenSSL-шифры. Например, `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`. +- cipherList - Поддерживаемые OpenSSL-шифры. Например, `ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH`. - cacheSessions - Включение/выключение кеширования сессии. Использовать обязательно вместе с `sessionIdContext`. Допустимые значения: `true`, `false`. - sessionIdContext - Уникальный набор произвольных символов, которые сервер добавляет к каждому сгенерированному идентификатору. Длина строки не должна превышать `SSL_MAX_SSL_SESSION_ID_LENGTH`. Рекомендуется к использованию всегда, поскольку позволяет избежать проблем как в случае, если сервер кеширует сессию, так и если клиент затребовал кеширование. По умолчанию `${application.name}`. - sessionCacheSize - Максимальное количество сессий, которые кэширует сервер. По умолчанию - 1024\*20. 0 - неограниченное количество сессий. diff --git a/docs/zh/operations/server-configuration-parameters/settings.md b/docs/zh/operations/server-configuration-parameters/settings.md index f6106d8734e..8e2cb389f04 100644 --- a/docs/zh/operations/server-configuration-parameters/settings.md +++ b/docs/zh/operations/server-configuration-parameters/settings.md @@ -455,7 +455,7 @@ SSL客户端/服务器配置。 - verificationMode – The method for checking the node’s certificates. Details are in the description of the [A.背景](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) 同学们 可能的值: `none`, `relaxed`, `strict`, `once`. - verificationDepth – The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value. - loadDefaultCAFile – Indicates that built-in CA certificates for OpenSSL will be used. Acceptable values: `true`, `false`. \| -- cipherList – Supported OpenSSL encryptions. For example: `ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH`. +- cipherList – Supported OpenSSL encryptions. For example: `ALL:!ADH:!LOW:!EXP:!MD5:!3DES:@STRENGTH`. - cacheSessions – Enables or disables caching sessions. Must be used in combination with `sessionIdContext`. 可接受的值: `true`, `false`. - sessionIdContext – A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed `SSL_MAX_SSL_SESSION_ID_LENGTH`. 始终建议使用此参数,因为如果服务器缓存会话,以及客户端请求缓存,它有助于避免出现问题。 默认值: `${application.name}`. - sessionCacheSize – The maximum number of sessions that the server caches. Default value: 1024\*20. 0 – Unlimited sessions. From aaba48f0df13aa9c314d604affc1ac491c4db5dd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Jul 2023 17:19:33 +0000 Subject: [PATCH 079/777] 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 0641dfd7e640bfd08d8533d41cb8ada524dd7b3a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Aug 2023 22:33:04 +0200 Subject: [PATCH 080/777] Add missing modification --- tests/ci/ci_config.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 517e40fd2d6..be8364f8874 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -402,6 +402,9 @@ CI_CONFIG = { "Sqllogic test (release)": { "required_build": "package_release", }, + "SQLTest": { + "required_build": "package_release", + }, }, } # type: dict From 3eeaf7af22ba5ca5b530fd85b9dc08995b2964c2 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 2 Aug 2023 16:08:53 +0000 Subject: [PATCH 081/777] Fix build error --- src/Functions/FunctionsHashing.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index a2083d0a88e..f70e91033ea 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1377,8 +1377,8 @@ public: if constexpr (std::is_same_v) /// backward-compatible { - if (std::endian::native == std::endian::big) - std::ranges::for_each(col_to->getData(), transformEndianness); + if constexpr (std::endian::native == std::endian::big) + std::ranges::for_each(col_to->getData(), transformEndianness); auto col_to_fixed_string = ColumnFixedString::create(sizeof(UInt128)); const auto & data = col_to->getData(); From 469dd7f30033b4e7870d6999825f8a16fe106e59 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 19:06:31 +0200 Subject: [PATCH 082/777] Add the docs --- docs/en/interfaces/cli.md | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 8779dd1a544..36afb94433a 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -323,9 +323,9 @@ clickhouse-client clickhouse://192.168.1.15,192.168.1.25 `clickhouse-client` uses the first existing file of the following: - Defined in the `--config-file` parameter. -- `./clickhouse-client.xml` -- `~/.clickhouse-client/config.xml` -- `/etc/clickhouse-client/config.xml` +- `./clickhouse-client.xml`, `.yaml`, `.yml` +- `~/.clickhouse-client/config.xml`, `.yaml`, `.yml` +- `/etc/clickhouse-client/config.xml`, `.yaml`, `.yml` Example of a config file: @@ -342,6 +342,17 @@ Example of a config file: ``` +Or the same config in a YAML format: + +```yaml +user: username +password: 'password' +secure: true +openSSL: + client: + caConfig: '/etc/ssl/cert.pem' +``` + ### Query ID Format {#query-id-format} In interactive mode `clickhouse-client` shows query ID for every query. By default, the ID is formatted like this: From b4f0d0bc2edb5865128e16dd90db00d25df8747e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 21:05:39 +0300 Subject: [PATCH 083/777] Update pull_request.yml --- .github/workflows/pull_request.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 1d515633f61..6ec8c003491 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -5194,7 +5194,7 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/sqltest REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME="SQLTest" + CHECK_NAME=SQLTest REPO_COPY=${{runner.temp}}/sqltest/ClickHouse EOF - name: Download json reports From 5ec6a4695f2911758e8d64ee52e34149c4940112 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 23:22:15 +0200 Subject: [PATCH 084/777] Fix log paths --- tests/ci/sqltest.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index b752d4e4aee..7385716850f 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,11 +110,10 @@ def main(): s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" paths = { "run.log": run_log_path, - "main.log": os.path.join(workspace_path, "main.log"), - "server.log.zst": os.path.join(workspace_path, "server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "server.err.log.zst"), - "report.html": os.path.join(workspace_path, "report.html"), - "test.log": os.path.join(workspace_path, "test.log"), + "server.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.log.zst"), + "server.err.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.err.log.zst"), + "report.html": os.path.join(workspace_path, "/report.html"), + "test.log": os.path.join(workspace_path, "/test.log"), } s3_helper = S3Helper() From 619af0c2c66255e8c8aa4f8fd62b96c94fe35ee3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 2 Aug 2023 21:46:23 +0000 Subject: [PATCH 085/777] Automatic style fix --- tests/ci/sqltest.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index 7385716850f..498448a7928 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,8 +110,12 @@ def main(): s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" paths = { "run.log": run_log_path, - "server.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.err.log.zst"), + "server.log.zst": os.path.join( + workspace_path, "/var/log/clickhouse-server/server.log.zst" + ), + "server.err.log.zst": os.path.join( + workspace_path, "/var/log/clickhouse-server/server.err.log.zst" + ), "report.html": os.path.join(workspace_path, "/report.html"), "test.log": os.path.join(workspace_path, "/test.log"), } From 99c7f7b48ca61af6077fd99431194bde24f761b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Aug 2023 23:50:19 +0200 Subject: [PATCH 086/777] Fix the test --- tests/queries/0_stateless/02834_client_yaml_configs.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02834_client_yaml_configs.sh b/tests/queries/0_stateless/02834_client_yaml_configs.sh index d1c5a40c04f..66d3df8829e 100755 --- a/tests/queries/0_stateless/02834_client_yaml_configs.sh +++ b/tests/queries/0_stateless/02834_client_yaml_configs.sh @@ -1,11 +1,11 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-random-settings CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -pushd "${CLICKHOUSE_TMP}" || exit > /dev/null +pushd "${CLICKHOUSE_TMP}" > /dev/null || exit echo "max_block_size: 31337" > clickhouse-client.yaml ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" @@ -19,4 +19,4 @@ echo "31339" > clickho ${CLICKHOUSE_CLIENT} --query "SELECT getSetting('max_block_size')" rm clickhouse-client.xml -popd || exit > /dev/null +popd > /dev/null || exit From 48b1a0b44f85034d53245d9dd941929378f10d83 Mon Sep 17 00:00:00 2001 From: Jiyoung Yoo Date: Mon, 31 Jul 2023 08:09:57 -0700 Subject: [PATCH 087/777] Fix for issue #40220 improve error message for table function remote --- src/TableFunctions/TableFunctionRemote.cpp | 13 +++++++++++-- .../02841_remote_parameter_parsing_error.reference | 0 .../02841_remote_parameter_parsing_error.sql | 14 ++++++++++++++ 3 files changed, 25 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference create mode 100644 tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index e4442c565df..d88d4190191 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -159,7 +159,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr { if (arg_num >= args.size()) { - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name is missing"); } else { @@ -211,7 +211,16 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr } if (arg_num < args.size()) - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + { + if (is_cluster_function) + { + throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "user and password should be string literal (in single quotes)"); + } + } } if (!cluster_name.empty()) diff --git a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql new file mode 100644 index 00000000000..f2ac88b62fb --- /dev/null +++ b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql @@ -0,0 +1,14 @@ +-- Tags: shard + + +select * from remote('127.0.0.1'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', sys); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', system); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', system.o); -- { serverError UNKNOWN_TABLE } +select * from remote('127.0.0.1', system.one, default); -- { serverError UNKNOWN_IDENTIFIER } +select * from remote('127.0.0.1', system.one, default, ''); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, default, key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, 'default', '', key1); -- { serverError UNKNOWN_IDENTIFIER } +select * from remote('127.0.0.1', system.one, default, '', key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, 'default', pwd, key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system, one, 'default', '', key1, key2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From e507221b7c56db0d19850de81c596af0c2b47512 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 Aug 2023 09:35:45 +0000 Subject: [PATCH 088/777] 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 4734d416a54aaa87a968b9527b4782928b5f668a Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Thu, 3 Aug 2023 12:44:44 +0300 Subject: [PATCH 089/777] Add settings and processing code --- src/Access/MultipleAccessStorage.cpp | 11 ++++ src/Access/MultipleAccessStorage.h | 1 + src/Core/Settings.h | 3 + .../Access/InterpreterCreateQuotaQuery.cpp | 15 +++-- .../Access/InterpreterCreateRoleQuery.cpp | 10 +++- .../InterpreterCreateRowPolicyQuery.cpp | 15 +++-- .../InterpreterCreateSettingsProfileQuery.cpp | 14 +++-- .../Access/InterpreterCreateUserQuery.cpp | 18 ++++-- .../InterpreterDropAccessEntityQuery.cpp | 13 ++-- .../InterpreterCreateFunctionQuery.cpp | 8 ++- .../InterpreterDropFunctionQuery.cpp | 11 ++-- .../removeOnClusterClauseIfNeeded.cpp | 59 +++++++++++++++++++ .../removeOnClusterClauseIfNeeded.h | 12 ++++ 13 files changed, 152 insertions(+), 38 deletions(-) create mode 100644 src/Interpreters/removeOnClusterClauseIfNeeded.cpp create mode 100644 src/Interpreters/removeOnClusterClauseIfNeeded.h diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index f36a670ddbf..e6a89ca0c34 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -415,4 +415,15 @@ void MultipleAccessStorage::restoreFromBackup(RestorerFromBackup & restorer) throwBackupNotAllowed(); } +bool MultipleAccessStorage::containsStorage(std::string_view storage_type) const +{ + auto storages = getStoragesInternal(); + + for (const auto & storage : *storages) + { + if (storage->getStorageType() == storage_type) + return true; + } + return false; +} } diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index 6a0c1bdfc02..5f12448bff6 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -47,6 +47,7 @@ public: bool isRestoreAllowed() const override; void backup(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, AccessEntityType type) const override; void restoreFromBackup(RestorerFromBackup & restorer) override; + bool containsStorage(std::string_view storage_type) const; protected: std::optional findImpl(AccessEntityType type, const String & name) const override; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 26e0808aaa3..4294ded5c23 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -305,6 +305,9 @@ class IColumn; M(Bool, final, false, "Query with the FINAL modifier by default. If the engine does not support final, it does not have any effect. On queries with multiple tables final is applied only on those that support it. It also works on distributed tables", 0) \ \ M(Bool, partial_result_on_first_cancel, false, "Allows query to return a partial result after cancel.", 0) \ + \ + M(Bool, ignore_on_cluster_for_replicated_udf_queries, false, "Ignore ON CLUSTER clause for replicated UDF management queries.", 0) \ + M(Bool, ignore_on_cluster_for_replicated_access_entities_queries, false, "Ignore ON CLUSTER clause for replicated access entities management queries.", 0) \ /** Settings for testing hedged requests */ \ M(Milliseconds, sleep_in_send_tables_status_ms, 0, "Time to sleep in sending tables status response in TCPHandler", 0) \ M(Milliseconds, sleep_in_send_data_ms, 0, "Time to sleep in sending data in TCPHandler", 0) \ diff --git a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp index 83a620d11c6..8bec0b131ca 100644 --- a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp @@ -1,16 +1,17 @@ #include -#include -#include + #include #include #include #include #include +#include +#include +#include #include #include -#include #include - +#include namespace DB { @@ -76,14 +77,16 @@ namespace BlockIO InterpreterCreateQuotaQuery::execute() { - auto & query = query_ptr->as(); + const auto updated_query_ptr = removeOnClusterClauseIfNeeded(query_ptr, getContext()); + auto & query = updated_query_ptr->as(); + auto & access_control = getContext()->getAccessControl(); getContext()->checkAccess(query.alter ? AccessType::ALTER_QUOTA : AccessType::CREATE_QUOTA); if (!query.cluster.empty()) { query.replaceCurrentUserTag(getContext()->getUserName()); - return executeDDLQueryOnCluster(query_ptr, getContext()); + return executeDDLQueryOnCluster(updated_query_ptr, getContext()); } std::optional roles_from_query; diff --git a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp index 3386dfb8792..990b4a5769d 100644 --- a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp @@ -1,9 +1,11 @@ #include -#include + #include #include #include #include +#include +#include namespace DB @@ -33,7 +35,9 @@ namespace BlockIO InterpreterCreateRoleQuery::execute() { - const auto & query = query_ptr->as(); + const auto updated_query_ptr = removeOnClusterClauseIfNeeded(query_ptr, getContext()); + const auto & query = updated_query_ptr->as(); + auto & access_control = getContext()->getAccessControl(); if (query.alter) getContext()->checkAccess(AccessType::ALTER_ROLE); @@ -50,7 +54,7 @@ BlockIO InterpreterCreateRoleQuery::execute() } if (!query.cluster.empty()) - return executeDDLQueryOnCluster(query_ptr, getContext()); + return executeDDLQueryOnCluster(updated_query_ptr, getContext()); if (query.alter) { diff --git a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp index 87dc9039c17..9ab3de0fd6d 100644 --- a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp @@ -1,14 +1,16 @@ #include -#include -#include -#include -#include + #include #include #include #include #include #include +#include +#include +#include +#include +#include #include @@ -45,7 +47,8 @@ namespace BlockIO InterpreterCreateRowPolicyQuery::execute() { - auto & query = query_ptr->as(); + const auto updated_query_ptr = removeOnClusterClauseIfNeeded(query_ptr, getContext()); + auto & query = updated_query_ptr->as(); auto required_access = getRequiredAccess(); if (!query.cluster.empty()) @@ -53,7 +56,7 @@ BlockIO InterpreterCreateRowPolicyQuery::execute() query.replaceCurrentUserTag(getContext()->getUserName()); DDLQueryOnClusterParams params; params.access_to_check = std::move(required_access); - return executeDDLQueryOnCluster(query_ptr, getContext(), params); + return executeDDLQueryOnCluster(updated_query_ptr, getContext(), params); } assert(query.names->cluster.empty()); diff --git a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp index 0727b6f2182..e81599c0b23 100644 --- a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp @@ -1,11 +1,13 @@ #include -#include -#include + #include -#include #include +#include #include #include +#include +#include +#include namespace DB @@ -41,7 +43,9 @@ namespace BlockIO InterpreterCreateSettingsProfileQuery::execute() { - auto & query = query_ptr->as(); + const auto updated_query_ptr = removeOnClusterClauseIfNeeded(query_ptr, getContext()); + auto & query = updated_query_ptr->as(); + auto & access_control = getContext()->getAccessControl(); if (query.alter) getContext()->checkAccess(AccessType::ALTER_SETTINGS_PROFILE); @@ -60,7 +64,7 @@ BlockIO InterpreterCreateSettingsProfileQuery::execute() if (!query.cluster.empty()) { query.replaceCurrentUserTag(getContext()->getUserName()); - return executeDDLQueryOnCluster(query_ptr, getContext()); + return executeDDLQueryOnCluster(updated_query_ptr, getContext()); } std::optional roles_from_query; diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 165937560cc..c7d6604d811 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -1,14 +1,18 @@ #include -#include -#include -#include -#include + #include #include +#include #include +#include #include #include #include +#include +#include +#include +#include +#include #include @@ -104,7 +108,9 @@ namespace BlockIO InterpreterCreateUserQuery::execute() { - const auto & query = query_ptr->as(); + const auto updated_query_ptr = removeOnClusterClauseIfNeeded(query_ptr, getContext()); + const auto & query = updated_query_ptr->as(); + auto & access_control = getContext()->getAccessControl(); auto access = getContext()->getAccess(); access->checkAccess(query.alter ? AccessType::ALTER_USER : AccessType::CREATE_USER); @@ -137,7 +143,7 @@ BlockIO InterpreterCreateUserQuery::execute() } if (!query.cluster.empty()) - return executeDDLQueryOnCluster(query_ptr, getContext()); + return executeDDLQueryOnCluster(updated_query_ptr, getContext()); if (query.alter) { diff --git a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp index d4c37064065..2bfd7ae3d92 100644 --- a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp @@ -1,11 +1,12 @@ #include -#include -#include + #include #include #include #include - +#include +#include +#include namespace DB { @@ -17,12 +18,14 @@ namespace ErrorCodes BlockIO InterpreterDropAccessEntityQuery::execute() { - auto & query = query_ptr->as(); + const auto updated_query_ptr = removeOnClusterClauseIfNeeded(query_ptr, getContext()); + auto & query = updated_query_ptr->as(); + auto & access_control = getContext()->getAccessControl(); getContext()->checkAccess(getRequiredAccess()); if (!query.cluster.empty()) - return executeDDLQueryOnCluster(query_ptr, getContext()); + return executeDDLQueryOnCluster(updated_query_ptr, getContext()); query.replaceEmptyDatabase(getContext()->getCurrentDatabase()); diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.cpp b/src/Interpreters/InterpreterCreateFunctionQuery.cpp index d56b5029e41..3e87f4fe440 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.cpp +++ b/src/Interpreters/InterpreterCreateFunctionQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -18,7 +19,8 @@ namespace ErrorCodes BlockIO InterpreterCreateFunctionQuery::execute() { - ASTCreateFunctionQuery & create_function_query = query_ptr->as(); + const auto updated_query_ptr = removeOnClusterClauseIfNeeded(query_ptr, getContext()); + ASTCreateFunctionQuery & create_function_query = updated_query_ptr->as(); AccessRightsElements access_rights_elements; access_rights_elements.emplace_back(AccessType::CREATE_FUNCTION); @@ -35,7 +37,7 @@ BlockIO InterpreterCreateFunctionQuery::execute() DDLQueryOnClusterParams params; params.access_to_check = std::move(access_rights_elements); - return executeDDLQueryOnCluster(query_ptr, current_context, params); + return executeDDLQueryOnCluster(updated_query_ptr, current_context, params); } current_context->checkAccess(access_rights_elements); @@ -44,7 +46,7 @@ BlockIO InterpreterCreateFunctionQuery::execute() bool throw_if_exists = !create_function_query.if_not_exists && !create_function_query.or_replace; bool replace_if_exists = create_function_query.or_replace; - UserDefinedSQLFunctionFactory::instance().registerFunction(current_context, function_name, query_ptr, throw_if_exists, replace_if_exists); + UserDefinedSQLFunctionFactory::instance().registerFunction(current_context, function_name, updated_query_ptr, throw_if_exists, replace_if_exists); return {}; } diff --git a/src/Interpreters/InterpreterDropFunctionQuery.cpp b/src/Interpreters/InterpreterDropFunctionQuery.cpp index df81ae661c7..af60d9c5df7 100644 --- a/src/Interpreters/InterpreterDropFunctionQuery.cpp +++ b/src/Interpreters/InterpreterDropFunctionQuery.cpp @@ -1,12 +1,13 @@ -#include +#include #include #include #include #include #include -#include #include +#include +#include namespace DB @@ -20,7 +21,9 @@ namespace ErrorCodes BlockIO InterpreterDropFunctionQuery::execute() { FunctionNameNormalizer().visit(query_ptr.get()); - ASTDropFunctionQuery & drop_function_query = query_ptr->as(); + + const auto updated_query_ptr = removeOnClusterClauseIfNeeded(query_ptr, getContext()); + ASTDropFunctionQuery & drop_function_query = updated_query_ptr->as(); AccessRightsElements access_rights_elements; access_rights_elements.emplace_back(AccessType::DROP_FUNCTION); @@ -34,7 +37,7 @@ BlockIO InterpreterDropFunctionQuery::execute() DDLQueryOnClusterParams params; params.access_to_check = std::move(access_rights_elements); - return executeDDLQueryOnCluster(query_ptr, current_context, params); + return executeDDLQueryOnCluster(updated_query_ptr, current_context, params); } current_context->checkAccess(access_rights_elements); diff --git a/src/Interpreters/removeOnClusterClauseIfNeeded.cpp b/src/Interpreters/removeOnClusterClauseIfNeeded.cpp new file mode 100644 index 00000000000..7dc452a0fcb --- /dev/null +++ b/src/Interpreters/removeOnClusterClauseIfNeeded.cpp @@ -0,0 +1,59 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + + +static bool isUserDefinedFunctionQuery(const ASTPtr & query) +{ + return query->as() + || query->as(); +} + +static bool isAccessControlQuery(const ASTPtr & query) +{ + return query->as() + || query->as() + || query->as() + || query->as() + || query->as() + || query->as(); +} + +ASTPtr removeOnClusterClauseIfNeeded(const ASTPtr & query, ContextPtr context, const WithoutOnClusterASTRewriteParams & params) +{ + auto * query_on_cluster = dynamic_cast(query.get()); + + if (!query_on_cluster || query_on_cluster->cluster.empty()) + return query; + + if ((isUserDefinedFunctionQuery(query) + && context->getSettings().ignore_on_cluster_for_replicated_udf_queries + && context->getUserDefinedSQLObjectsLoader().isReplicated()) + || (isAccessControlQuery(query) + && context->getSettings().ignore_on_cluster_for_replicated_access_entities_queries + && context->getAccessControl().containsStorage(ReplicatedAccessStorage::STORAGE_TYPE))) + { + LOG_DEBUG(&Poco::Logger::get("removeOnClusterClauseIfNeeded"), "ON CLUSTER clause was ignored for query {}", query->getID()); + return query_on_cluster->getRewrittenASTWithoutOnCluster(params); + } + + return query; +} +} diff --git a/src/Interpreters/removeOnClusterClauseIfNeeded.h b/src/Interpreters/removeOnClusterClauseIfNeeded.h new file mode 100644 index 00000000000..0cbc196c9f2 --- /dev/null +++ b/src/Interpreters/removeOnClusterClauseIfNeeded.h @@ -0,0 +1,12 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +ASTPtr removeOnClusterClauseIfNeeded(const ASTPtr & query_ptr, ContextPtr context, const WithoutOnClusterASTRewriteParams & params = {}); + +} From 0fc0a89f8068538096888d5237665717abe9546e Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Thu, 3 Aug 2023 12:45:07 +0300 Subject: [PATCH 090/777] Add tests --- .../test.py | 29 +++++++++++++++++ .../integration/test_replicated_users/test.py | 32 +++++++++++++++++++ 2 files changed, 61 insertions(+) diff --git a/tests/integration/test_replicated_user_defined_functions/test.py b/tests/integration/test_replicated_user_defined_functions/test.py index c0990819bf4..f54be21c4c0 100644 --- a/tests/integration/test_replicated_user_defined_functions/test.py +++ b/tests/integration/test_replicated_user_defined_functions/test.py @@ -1,10 +1,12 @@ import inspect +from contextlib import nullcontext as does_not_raise import pytest import time import os.path from helpers.cluster import ClickHouseCluster +from helpers.client import QueryRuntimeException from helpers.test_tools import assert_eq_with_retry, TSV SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -83,6 +85,33 @@ def test_create_and_drop(): node1.query("DROP FUNCTION f1") +@pytest.mark.parametrize( + "ignore, expected_raise", + [("true", does_not_raise()), ("false", pytest.raises(QueryRuntimeException))], +) +def test_create_and_drop_udf_on_cluster(ignore, expected_raise): + node1.replace_config( + "/etc/clickhouse-server/users.d/users.xml", + inspect.cleandoc( + f""" + + + + {ignore} + + + + """ + ), + ) + node1.query("SYSTEM RELOAD CONFIG") + + with expected_raise: + node1.query("CREATE FUNCTION f1 ON CLUSTER default AS (x, y) -> x + y") + assert node1.query("SELECT f1(12, 3)") == "15\n" + node1.query("DROP FUNCTION f1 ON CLUSTER default") + + def test_create_and_replace(): node1.query("CREATE FUNCTION f1 AS (x, y) -> x + y") assert node1.query("SELECT f1(12, 3)") == "15\n" diff --git a/tests/integration/test_replicated_users/test.py b/tests/integration/test_replicated_users/test.py index a7dbaf6ed30..489724ed4fb 100644 --- a/tests/integration/test_replicated_users/test.py +++ b/tests/integration/test_replicated_users/test.py @@ -1,3 +1,4 @@ +import inspect import pytest import time @@ -82,6 +83,37 @@ def test_create_replicated_on_cluster(started_cluster, entity): node1.query(f"DROP {entity.keyword} {entity.name} {entity.options}") +@pytest.mark.parametrize("entity", entities, ids=get_entity_id) +def test_create_replicated_on_cluster_ignore(started_cluster, entity): + node1.replace_config( + "/etc/clickhouse-server/users.d/users.xml", + inspect.cleandoc( + f""" + + + + true + + + + """ + ), + ) + node1.query("SYSTEM RELOAD CONFIG") + + node1.query( + f"CREATE {entity.keyword} {entity.name} ON CLUSTER default {entity.options}" + ) + assert ( + f"cannot insert because {entity.keyword.lower()} `{entity.name}{entity.options}` already exists in replicated" + in node2.query_and_get_error_with_retry( + f"CREATE {entity.keyword} {entity.name} {entity.options}" + ) + ) + + node1.query(f"DROP {entity.keyword} {entity.name} {entity.options}") + + @pytest.mark.parametrize("entity", entities, ids=get_entity_id) def test_create_replicated_if_not_exists_on_cluster(started_cluster, entity): node1.query( From bfc4c0ebe3ac0369be48e0dfc3203a5fb7c5569e Mon Sep 17 00:00:00 2001 From: Aleksei Filatov Date: Thu, 3 Aug 2023 13:18:49 +0300 Subject: [PATCH 091/777] Fix style --- src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp | 2 +- src/Interpreters/Access/InterpreterCreateRoleQuery.cpp | 2 +- src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp index 8bec0b131ca..00397fdcd8f 100644 --- a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp @@ -79,7 +79,7 @@ BlockIO InterpreterCreateQuotaQuery::execute() { const auto updated_query_ptr = removeOnClusterClauseIfNeeded(query_ptr, getContext()); auto & query = updated_query_ptr->as(); - + auto & access_control = getContext()->getAccessControl(); getContext()->checkAccess(query.alter ? AccessType::ALTER_QUOTA : AccessType::CREATE_QUOTA); diff --git a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp index 990b4a5769d..b9624e1c9e0 100644 --- a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp @@ -37,7 +37,7 @@ BlockIO InterpreterCreateRoleQuery::execute() { const auto updated_query_ptr = removeOnClusterClauseIfNeeded(query_ptr, getContext()); const auto & query = updated_query_ptr->as(); - + auto & access_control = getContext()->getAccessControl(); if (query.alter) getContext()->checkAccess(AccessType::ALTER_ROLE); diff --git a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp index 9ab3de0fd6d..bb317c8883d 100644 --- a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp @@ -48,7 +48,7 @@ namespace BlockIO InterpreterCreateRowPolicyQuery::execute() { const auto updated_query_ptr = removeOnClusterClauseIfNeeded(query_ptr, getContext()); - auto & query = updated_query_ptr->as(); + auto & query = updated_query_ptr->as(); auto required_access = getRequiredAccess(); if (!query.cluster.empty()) From dadf4678ee9d678459c96cd13b8231e33dc02cc3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 3 Aug 2023 12:50:40 +0000 Subject: [PATCH 092/777] Automatic style fix --- tests/integration/test_dictionary_custom_settings/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_dictionary_custom_settings/test.py b/tests/integration/test_dictionary_custom_settings/test.py index 6b5ea643998..eb394da8bb6 100644 --- a/tests/integration/test_dictionary_custom_settings/test.py +++ b/tests/integration/test_dictionary_custom_settings/test.py @@ -82,5 +82,9 @@ def test_work(start_cluster): # It is possible that the HTTP server takes long time to start accepting connections - assert_eq_with_retry(instance, "SELECT dictGetString('test_http', 'first', toUInt64(1))", "\\'a\n") - assert_eq_with_retry(instance, "SELECT dictGetString('test_http', 'second', toUInt64(1))", '"b\n') + assert_eq_with_retry( + instance, "SELECT dictGetString('test_http', 'first', toUInt64(1))", "\\'a\n" + ) + assert_eq_with_retry( + instance, "SELECT dictGetString('test_http', 'second', toUInt64(1))", '"b\n' + ) From 65e8ee8fb7d30b0765c8bfd3d58d9def64e002bb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Aug 2023 15:28:18 +0200 Subject: [PATCH 093/777] Fix paths --- docker/test/sqltest/run.sh | 4 +++- tests/ci/sqltest.py | 8 ++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/docker/test/sqltest/run.sh b/docker/test/sqltest/run.sh index 42aeef9df15..cba1c1dab1f 100755 --- a/docker/test/sqltest/run.sh +++ b/docker/test/sqltest/run.sh @@ -42,8 +42,10 @@ done # Run the test pushd sqltest/standards/2016/ /test.py -mv report.html test.log / +mv report.html test.log /workspace popd zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.log zstd --threads=0 /var/log/clickhouse-server/clickhouse-server.err.log + +mv /var/log/clickhouse-server/clickhouse-server.log.zst /var/log/clickhouse-server/clickhouse-server.err.log.zst /workspace diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index 7385716850f..d76baf22bb3 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,10 +110,10 @@ def main(): s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" paths = { "run.log": run_log_path, - "server.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/var/log/clickhouse-server/server.err.log.zst"), - "report.html": os.path.join(workspace_path, "/report.html"), - "test.log": os.path.join(workspace_path, "/test.log"), + "server.log.zst": os.path.join(workspace_path, "/workspace/server.log.zst"), + "server.err.log.zst": os.path.join(workspace_path, "/workspace/server.err.log.zst"), + "report.html": os.path.join(workspace_path, "/workspace/report.html"), + "test.log": os.path.join(workspace_path, "/workspace/test.log"), } s3_helper = S3Helper() From a96b0457e04543876b35ca1a5223ea6007bbf9bb Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 3 Aug 2023 13:43:45 +0000 Subject: [PATCH 094/777] Automatic style fix --- tests/ci/sqltest.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index d76baf22bb3..69af21f1761 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -111,7 +111,9 @@ def main(): paths = { "run.log": run_log_path, "server.log.zst": os.path.join(workspace_path, "/workspace/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/workspace/server.err.log.zst"), + "server.err.log.zst": os.path.join( + workspace_path, "/workspace/server.err.log.zst" + ), "report.html": os.path.join(workspace_path, "/workspace/report.html"), "test.log": os.path.join(workspace_path, "/workspace/test.log"), } From 6a0a0ff30d3855be655142140002bb652f8227f8 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 3 Aug 2023 15:40:24 +0000 Subject: [PATCH 095/777] One line fix for the specific query --- src/Processors/QueryPlan/ReadFromRemote.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 7a99c363232..4a40d4343fd 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -240,7 +240,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact shard.shard_info.pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage); remote_query_executor->setLogger(log); - remote_query_executor->setPoolMode(PoolMode::GET_MANY); + remote_query_executor->setPoolMode(PoolMode::GET_ONE); if (!table_func_ptr) remote_query_executor->setMainTable(shard.main_table ? shard.main_table : main_table); From a0fa3cc73bbbec2879873608607fc18f51a6556a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 3 Aug 2023 18:55:15 +0300 Subject: [PATCH 096/777] EXPLAIN actions for JOIN step --- src/Interpreters/ConcurrentHashJoin.h | 1 + src/Interpreters/DirectJoin.h | 1 + src/Interpreters/FullSortingMergeJoin.h | 1 + src/Interpreters/GraceHashJoin.h | 1 + src/Interpreters/HashJoin.h | 1 + src/Interpreters/IJoin.h | 2 + src/Interpreters/JoinSwitcher.h | 1 + src/Interpreters/MergeJoin.h | 1 + src/Interpreters/TableJoin.h | 2 +- src/Processors/QueryPlan/JoinStep.cpp | 33 +++++ src/Processors/QueryPlan/JoinStep.h | 3 + .../02835_join_step_explain.reference | 116 ++++++++++++++++++ .../0_stateless/02835_join_step_explain.sql | 31 +++++ 13 files changed, 193 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02835_join_step_explain.reference create mode 100644 tests/queries/0_stateless/02835_join_step_explain.sql diff --git a/src/Interpreters/ConcurrentHashJoin.h b/src/Interpreters/ConcurrentHashJoin.h index 1283879971d..85e0c5a0ae7 100644 --- a/src/Interpreters/ConcurrentHashJoin.h +++ b/src/Interpreters/ConcurrentHashJoin.h @@ -36,6 +36,7 @@ public: explicit ConcurrentHashJoin(ContextPtr context_, std::shared_ptr table_join_, size_t slots_, const Block & right_sample_block, bool any_take_last_row_ = false); ~ConcurrentHashJoin() override = default; + std::string getName() const override { return "ConcurrentHashJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } bool addBlockToJoin(const Block & block, bool check_limits) override; void checkTypesOfKeys(const Block & block) const override; diff --git a/src/Interpreters/DirectJoin.h b/src/Interpreters/DirectJoin.h index e55ac278705..5f664314818 100644 --- a/src/Interpreters/DirectJoin.h +++ b/src/Interpreters/DirectJoin.h @@ -30,6 +30,7 @@ public: std::shared_ptr storage_, const Block & right_sample_block_with_storage_column_names_); + std::string getName() const override { return "DirectKeyValueJoin"; } virtual const TableJoin & getTableJoin() const override { return *table_join; } virtual bool addBlockToJoin(const Block &, bool) override; diff --git a/src/Interpreters/FullSortingMergeJoin.h b/src/Interpreters/FullSortingMergeJoin.h index a6b53a51c04..3fc9f8920ed 100644 --- a/src/Interpreters/FullSortingMergeJoin.h +++ b/src/Interpreters/FullSortingMergeJoin.h @@ -28,6 +28,7 @@ public: LOG_TRACE(&Poco::Logger::get("FullSortingMergeJoin"), "Will use full sorting merge join"); } + std::string getName() const override { return "FullSortingMergeJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } bool addBlockToJoin(const Block & /* block */, bool /* check_limits */) override diff --git a/src/Interpreters/GraceHashJoin.h b/src/Interpreters/GraceHashJoin.h index ce519892b0e..44949440467 100644 --- a/src/Interpreters/GraceHashJoin.h +++ b/src/Interpreters/GraceHashJoin.h @@ -60,6 +60,7 @@ public: ~GraceHashJoin() override; + std::string getName() const override { return "GraceHashJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } void initialize(const Block & sample_block) override; diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 56dea98c1f1..9f55945816c 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -151,6 +151,7 @@ public: ~HashJoin() override; + std::string getName() const override { return "HashJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } /** Add block of data from right hand of JOIN to the map. diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 97b119bd795..493a5dd2126 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -48,6 +48,8 @@ class IJoin public: virtual ~IJoin() = default; + virtual std::string getName() const = 0; + virtual const TableJoin & getTableJoin() const = 0; /// Add block of data from right hand of JOIN. diff --git a/src/Interpreters/JoinSwitcher.h b/src/Interpreters/JoinSwitcher.h index fb5066b2d04..1d2ebc6b456 100644 --- a/src/Interpreters/JoinSwitcher.h +++ b/src/Interpreters/JoinSwitcher.h @@ -18,6 +18,7 @@ class JoinSwitcher : public IJoin public: JoinSwitcher(std::shared_ptr table_join_, const Block & right_sample_block_); + std::string getName() const override { return "JoinSwitcher"; } const TableJoin & getTableJoin() const override { return *table_join; } /// Add block of data from right hand of JOIN into current join object. diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index 03a661c5b8a..98fae1d419f 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -22,6 +22,7 @@ class MergeJoin : public IJoin public: MergeJoin(std::shared_ptr table_join_, const Block & right_sample_block); + std::string getName() const override { return "PartialMergeJoin"; } const TableJoin & getTableJoin() const override { return *table_join; } bool addBlockToJoin(const Block & block, bool check_limits) override; void checkTypesOfKeys(const Block & block) const override; diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 5d14a57759f..75626764bda 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -331,7 +331,7 @@ public: const ColumnsWithTypeAndName & right_sample_columns); void setAsofInequality(ASOFJoinInequality inequality) { asof_inequality = inequality; } - ASOFJoinInequality getAsofInequality() { return asof_inequality; } + ASOFJoinInequality getAsofInequality() const { return asof_inequality; } ASTPtr leftKeysList() const; ASTPtr rightKeysList() const; /// For ON syntax only diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 33fa7955e0d..858a01a437d 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -2,6 +2,9 @@ #include #include #include +#include +#include +#include #include namespace DB @@ -62,6 +65,36 @@ void JoinStep::describePipeline(FormatSettings & settings) const IQueryPlanStep::describePipeline(processors, settings); } +void JoinStep::describeActions(FormatSettings & settings) const +{ + String prefix(settings.offset, ' '); + + const auto & table_join = join->getTableJoin(); + settings.out << prefix << "Kind: " << toString(table_join.kind()) << '\n'; + settings.out << prefix << "Strictness: " << toString(table_join.strictness()) << '\n'; + settings.out << prefix << "Type: " << join->getName() << '\n'; + + if (table_join.strictness() == JoinStrictness::Asof) + settings.out << prefix << "ASOF inequality: " << toString(table_join.getAsofInequality()) << '\n'; + + if (!table_join.getClauses().empty()) + settings.out << prefix << "Clauses: " << table_join.formatClauses(table_join.getClauses(), true /*short_format*/) << '\n'; +} + +void JoinStep::describeActions(JSONBuilder::JSONMap & map) const +{ + const auto & table_join = join->getTableJoin(); + map.add("Kind", toString(table_join.kind())); + map.add("Strictness", toString(table_join.strictness())); + map.add("Type", join->getName()); + + if (table_join.strictness() == JoinStrictness::Asof) + map.add("ASOF inequality", toString(table_join.getAsofInequality())); + + if (!table_join.getClauses().empty()) + map.add("Clauses", table_join.formatClauses(table_join.getClauses(), true /*short_format*/)); +} + void JoinStep::updateInputStream(const DataStream & new_input_stream_, size_t idx) { if (idx == 0) diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index e7185f36588..369ee9bec8b 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -27,6 +27,9 @@ public: void describePipeline(FormatSettings & settings) const override; + void describeActions(JSONBuilder::JSONMap & map) const override; + void describeActions(FormatSettings & settings) const override; + const JoinPtr & getJoin() const { return join; } bool allowPushDownToRight() const; diff --git a/tests/queries/0_stateless/02835_join_step_explain.reference b/tests/queries/0_stateless/02835_join_step_explain.reference new file mode 100644 index 00000000000..77c91b3d7e9 --- /dev/null +++ b/tests/queries/0_stateless/02835_join_step_explain.reference @@ -0,0 +1,116 @@ +Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Header: id UInt64 + value_1 String + rhs.id UInt64 + rhs.value_1 String +Actions: INPUT : 0 -> id_0 UInt64 : 0 + INPUT : 1 -> value_1_1 String : 1 + INPUT : 2 -> value_1_3 String : 2 + INPUT : 3 -> id_2 UInt64 : 3 + ALIAS id_0 :: 0 -> id UInt64 : 4 + ALIAS value_1_1 :: 1 -> value_1 String : 0 + ALIAS value_1_3 :: 2 -> rhs.value_1 String : 1 + ALIAS id_2 :: 3 -> rhs.id UInt64 : 2 +Positions: 4 0 2 1 + Join (JOIN FillRightFirst) + Header: id_0 UInt64 + value_1_1 String + value_1_3 String + id_2 UInt64 + Kind: INNER + Strictness: ALL + Type: HashJoin + Clauses: [(id_0) = (id_2)] + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_0 UInt64 + value_1_1 String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + ALIAS id :: 0 -> id_0 UInt64 : 2 + ALIAS value_1 :: 1 -> value_1_1 String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value_1 String + ReadType: Default + Parts: 1 + Granules: 1 + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_2 UInt64 + value_1_3 String + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + ALIAS id :: 0 -> id_2 UInt64 : 2 + ALIAS value_1 :: 1 -> value_1_3 String : 0 + Positions: 2 0 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value_1 String + ReadType: Default + Parts: 1 + Granules: 1 +-- +Expression ((Project names + (Projection + DROP unused columns after JOIN))) +Header: id UInt64 + value_1 String + rhs.id UInt64 + rhs.value_1 String +Actions: INPUT : 0 -> id_0 UInt64 : 0 + INPUT : 1 -> value_1_1 String : 1 + INPUT :: 2 -> value_2_4 UInt64 : 2 + INPUT : 3 -> value_1_3 String : 3 + INPUT :: 4 -> value_2_5 UInt64 : 4 + INPUT : 5 -> id_2 UInt64 : 5 + ALIAS id_0 :: 0 -> id UInt64 : 6 + ALIAS value_1_1 :: 1 -> value_1 String : 0 + ALIAS value_1_3 :: 3 -> rhs.value_1 String : 1 + ALIAS id_2 :: 5 -> rhs.id UInt64 : 3 +Positions: 6 0 3 1 + Join (JOIN FillRightFirst) + Header: id_0 UInt64 + value_1_1 String + value_2_4 UInt64 + value_1_3 String + value_2_5 UInt64 + id_2 UInt64 + Kind: INNER + Strictness: ASOF + Type: HashJoin + ASOF inequality: LESS + Clauses: [(id_0, value_2_4) = (id_2, value_2_5)] + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_0 UInt64 + value_1_1 String + value_2_4 UInt64 + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + INPUT : 2 -> value_2 UInt64 : 2 + ALIAS id :: 0 -> id_0 UInt64 : 3 + ALIAS value_1 :: 1 -> value_1_1 String : 0 + ALIAS value_2 :: 2 -> value_2_4 UInt64 : 1 + Positions: 3 0 1 + ReadFromMergeTree (default.test_table_1) + Header: id UInt64 + value_1 String + value_2 UInt64 + ReadType: Default + Parts: 1 + Granules: 1 + Expression ((JOIN actions + Change column names to column identifiers)) + Header: id_2 UInt64 + value_1_3 String + value_2_5 UInt64 + Actions: INPUT : 0 -> id UInt64 : 0 + INPUT : 1 -> value_1 String : 1 + INPUT : 2 -> value_2 UInt64 : 2 + ALIAS id :: 0 -> id_2 UInt64 : 3 + ALIAS value_1 :: 1 -> value_1_3 String : 0 + ALIAS value_2 :: 2 -> value_2_5 UInt64 : 1 + Positions: 3 0 1 + ReadFromMergeTree (default.test_table_2) + Header: id UInt64 + value_1 String + value_2 UInt64 + ReadType: Default + Parts: 1 + Granules: 1 diff --git a/tests/queries/0_stateless/02835_join_step_explain.sql b/tests/queries/0_stateless/02835_join_step_explain.sql new file mode 100644 index 00000000000..d0475fa14b6 --- /dev/null +++ b/tests/queries/0_stateless/02835_join_step_explain.sql @@ -0,0 +1,31 @@ +SET allow_experimental_analyzer = 1; + +DROP TABLE IF EXISTS test_table_1; +CREATE TABLE test_table_1 +( + id UInt64, + value_1 String, + value_2 UInt64 +) ENGINE=MergeTree ORDER BY id; + +DROP TABLE IF EXISTS test_table_2; +CREATE TABLE test_table_2 +( + id UInt64, + value_1 String, + value_2 UInt64 +) ENGINE=MergeTree ORDER BY id; + +INSERT INTO test_table_1 VALUES (0, 'Value', 0); +INSERT INTO test_table_2 VALUES (0, 'Value', 0); + +EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1 +FROM test_table_1 AS lhs INNER JOIN test_table_2 AS rhs ON lhs.id = rhs.id; + +SELECT '--'; + +EXPLAIN header = 1, actions = 1 SELECT lhs.id, lhs.value_1, rhs.id, rhs.value_1 +FROM test_table_1 AS lhs ASOF JOIN test_table_2 AS rhs ON lhs.id = rhs.id AND lhs.value_2 < rhs.value_2; + +DROP TABLE test_table_1; +DROP TABLE test_table_2; From 78a59b213c8a366ba9840764a358edd025bfc7d6 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 3 Aug 2023 15:58:48 +0000 Subject: [PATCH 097/777] Trivial test with MergeTree --- ...835_parallel_replicas_over_distributed.reference | 1 + .../02835_parallel_replicas_over_distributed.sql | 13 +++++++++++++ 2 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/02835_parallel_replicas_over_distributed.reference create mode 100644 tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql diff --git a/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.reference b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.reference new file mode 100644 index 00000000000..ea4483ec305 --- /dev/null +++ b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.reference @@ -0,0 +1 @@ +100 4950 diff --git a/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql new file mode 100644 index 00000000000..199be3771c7 --- /dev/null +++ b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS test_d; +DROP TABLE IF EXISTS test; +CREATE TABLE test (id UInt64, date Date) +ENGINE = MergeTree +ORDER BY id +as select *, today() from numbers(100); + +CREATE TABLE IF NOT EXISTS test_d as test +ENGINE = Distributed(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), test); + +SELECT count(), sum(id) +FROM test_d +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; From 388feb953d97b244b162c433e2983cbe4e2d73f3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Aug 2023 19:28:59 +0200 Subject: [PATCH 098/777] Fix paths --- tests/ci/sqltest.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index d76baf22bb3..be22a1c9312 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -110,10 +110,10 @@ def main(): s3_prefix = f"{pr_info.number}/{pr_info.sha}/sqltest_{check_name_lower}/" paths = { "run.log": run_log_path, - "server.log.zst": os.path.join(workspace_path, "/workspace/server.log.zst"), - "server.err.log.zst": os.path.join(workspace_path, "/workspace/server.err.log.zst"), - "report.html": os.path.join(workspace_path, "/workspace/report.html"), - "test.log": os.path.join(workspace_path, "/workspace/test.log"), + "server.log.zst": os.path.join(workspace_path, "server.log.zst"), + "server.err.log.zst": os.path.join(workspace_path, "server.err.log.zst"), + "report.html": os.path.join(workspace_path, "report.html"), + "test.log": os.path.join(workspace_path, "test.log"), } s3_helper = S3Helper() From 6dc74fb5ccf0c6ef6538e60835667b57b6d4f060 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Thu, 3 Aug 2023 12:38:43 -0700 Subject: [PATCH 099/777] Implement big-endian support for transform --- .../Serializations/SerializationArray.cpp | 4 ++-- .../Serializations/SerializationNumber.cpp | 8 ++++---- src/Functions/transform.cpp | 14 ++++++++------ 3 files changed, 14 insertions(+), 12 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index cedcca870dd..e01c1aea0e9 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -129,7 +129,7 @@ namespace for (size_t i = offset; i < end; ++i) { ColumnArray::Offset current_offset = offset_values[i]; - writeIntBinary(current_offset - prev_offset, ostr); + writeBinaryLittleEndian(current_offset - prev_offset, ostr); prev_offset = current_offset; } } @@ -145,7 +145,7 @@ namespace while (i < initial_size + limit && !istr.eof()) { ColumnArray::Offset current_size = 0; - readIntBinary(current_size, istr); + readBinaryLittleEndian(current_size, istr); if (unlikely(current_size > MAX_ARRAY_SIZE)) throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Array size is too large: {}", current_size); diff --git a/src/DataTypes/Serializations/SerializationNumber.cpp b/src/DataTypes/Serializations/SerializationNumber.cpp index 8cabaec753d..0294a1c8a67 100644 --- a/src/DataTypes/Serializations/SerializationNumber.cpp +++ b/src/DataTypes/Serializations/SerializationNumber.cpp @@ -106,28 +106,28 @@ void SerializationNumber::serializeBinary(const Field & field, WriteBuffer & { /// ColumnVector::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64 typename ColumnVector::ValueType x = static_cast::ValueType>(field.get()); - writeBinary(x, ostr); + writeBinaryLittleEndian(x, ostr); } template void SerializationNumber::deserializeBinary(Field & field, ReadBuffer & istr, const FormatSettings &) const { typename ColumnVector::ValueType x; - readBinary(x, istr); + readBinaryLittleEndian(x, istr); field = NearestFieldType(x); } template void SerializationNumber::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeBinary(assert_cast &>(column).getData()[row_num], ostr); + writeBinaryLittleEndian(assert_cast &>(column).getData()[row_num], ostr); } template void SerializationNumber::deserializeBinary(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { typename ColumnVector::ValueType x; - readBinary(x, istr); + readBinaryLittleEndian(x, istr); assert_cast &>(column).getData().push_back(x); } diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index e03701327b1..16326dd5a44 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -764,9 +764,8 @@ namespace } /// Note: Doesn't check the duplicates in the `from` array. - - WhichDataType which(from_type); - if (isNativeNumber(which) || which.isDecimal32() || which.isDecimal64()) + /// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64 + if (WhichDataType which(from_type); isNativeNumber(which) || which.isDecimal32() || which.isDecimal64()) { cache.table_num_to_idx = std::make_unique(); auto & table = *cache.table_num_to_idx; @@ -774,10 +773,13 @@ namespace { if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i])) { - /// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64 - StringRef ref = cache.from_column->getDataAt(i); UInt64 key = 0; - memcpy(&key, ref.data, ref.size); + auto * dst = reinterpret_cast(&key); + const auto ref = cache.from_column->getDataAt(i); + if constexpr (std::endian::native == std::endian::big) + dst += sizeof(key) - ref.size; + + memcpy(dst, ref.data, ref.size); table[key] = i; } } From 0032eb9ab7c0d2815bb6e5fc9e3f31ad14a2346a Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 13 Apr 2023 15:03:27 +0000 Subject: [PATCH 100/777] 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 101/777] 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 102/777] 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 103/777] 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 a16c764b26b4c9aa2a146a15e3d87e76d5b959b2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 4 Aug 2023 10:07:43 +0000 Subject: [PATCH 104/777] 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 105/777] 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 02ed17dfa54b46b6439401fffd1591d77b07e2b7 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 4 Aug 2023 14:16:33 +0000 Subject: [PATCH 106/777] 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 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 107/777] Reduce the execution time of a single query --- .../01414_mutations_and_errors_zookeeper.sh | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh b/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh index f92ab265eb8..dc88808d73b 100755 --- a/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh +++ b/tests/queries/0_stateless/01414_mutations_and_errors_zookeeper.sh @@ -18,19 +18,19 @@ $CLICKHOUSE_CLIENT --query " PARTITION BY date " -$CLICKHOUSE_CLIENT --query "INSERT INTO replicated_mutation_table SELECT toDate('2019-10-02'), number, '42' FROM numbers(4)" +$CLICKHOUSE_CLIENT --query "INSERT INTO replicated_mutation_table SELECT toDate('2019-10-02'), number, '42' FROM numbers(10)" -$CLICKHOUSE_CLIENT --query "INSERT INTO replicated_mutation_table SELECT toDate('2019-10-02'), number, 'Hello' FROM numbers(4)" +$CLICKHOUSE_CLIENT --query "INSERT INTO replicated_mutation_table SELECT toDate('2019-10-02'), number, 'Hello' FROM numbers(10)" $CLICKHOUSE_CLIENT --query "ALTER TABLE replicated_mutation_table UPDATE key = key + 1 WHERE sleepEachRow(1) == 0 SETTINGS mutations_sync = 2" 2>&1 | grep -o 'Mutation 0000000000 was killed' | head -n 1 & check_query="SELECT count() FROM system.mutations WHERE table='replicated_mutation_table' and database='$CLICKHOUSE_DATABASE' and mutation_id='0000000000'" -query_result=$($CLICKHOUSE_CLIENT --query="$check_query" 2>&1) +query_result=$(curl $CLICKHOUSE_URL --silent --fail --data "$check_query") while [ "$query_result" != "1" ] do - query_result=$($CLICKHOUSE_CLIENT --query="$check_query" 2>&1) + query_result=$(curl $CLICKHOUSE_URL --silent --fail --data "$check_query") sleep 0.1 done @@ -38,7 +38,7 @@ $CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE table='replicated_mutation_table while [ "$query_result" != "0" ] do - query_result=$($CLICKHOUSE_CLIENT --query="$check_query" 2>&1) + query_result=$(curl $CLICKHOUSE_URL --silent --fail --data "$check_query") sleep 0.5 done @@ -49,11 +49,11 @@ $CLICKHOUSE_CLIENT --query "ALTER TABLE replicated_mutation_table MODIFY COLUMN check_query="SELECT type = 'UInt64' FROM system.columns WHERE table='replicated_mutation_table' and database='$CLICKHOUSE_DATABASE' and name='value'" -query_result=$($CLICKHOUSE_CLIENT --query="$check_query" 2>&1) +query_result=$(curl $CLICKHOUSE_URL --silent --fail --data "$check_query") while [ "$query_result" != "1" ] do - query_result=$($CLICKHOUSE_CLIENT --query="$check_query" 2>&1) + query_result=$(curl $CLICKHOUSE_URL --silent --fail --data "$check_query") sleep 0.5 done @@ -66,7 +66,7 @@ $CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE table='replicated_mutation_table while [ "$query_result" != "0" ] do - query_result=$($CLICKHOUSE_CLIENT --query="$check_query" 2>&1) + query_result=$(curl $CLICKHOUSE_URL --silent --fail --data "$check_query") sleep 0.5 done From 06631f6a86a2e099ef69f807592ec2eb70b5aa35 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 4 Aug 2023 09:27:15 -0700 Subject: [PATCH 108/777] Make hasTokenOrNull return null on empty needle --- src/Functions/HasTokenImpl.h | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Functions/HasTokenImpl.h b/src/Functions/HasTokenImpl.h index ab6b6399486..661b8ae9753 100644 --- a/src/Functions/HasTokenImpl.h +++ b/src/Functions/HasTokenImpl.h @@ -39,9 +39,6 @@ struct HasTokenImpl if (start_pos != nullptr) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' does not support start_pos argument", name); - if (pattern.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle cannot be empty, because empty string isn't a token"); - if (haystack_offsets.empty()) return; @@ -49,7 +46,7 @@ struct HasTokenImpl const UInt8 * const end = haystack_data.data() + haystack_data.size(); const UInt8 * pos = begin; - if (!std::none_of(pattern.begin(), pattern.end(), isTokenSeparator)) + if (const auto has_separator = std::any_of(pattern.cbegin(), pattern.cend(), isTokenSeparator); has_separator || pattern.empty()) { if (res_null) { @@ -57,8 +54,12 @@ struct HasTokenImpl std::ranges::fill(res_null->getData(), true); return; } - else + else if (has_separator) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle must not contain whitespace or separator characters"); + else if (pattern.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle cannot be empty, because empty string isn't a token"); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected internal state"); } size_t pattern_size = pattern.size(); From 054562233143594b1ebd83d248567151bfa9adb2 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 4 Aug 2023 13:03:15 -0700 Subject: [PATCH 109/777] Add functional tests for hasTokenOrNull --- tests/queries/0_stateless/02816_has_token_empty.reference | 4 ++++ tests/queries/0_stateless/02816_has_token_empty.sql | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/tests/queries/0_stateless/02816_has_token_empty.reference b/tests/queries/0_stateless/02816_has_token_empty.reference index aa47d0d46d4..8435d77c5fe 100644 --- a/tests/queries/0_stateless/02816_has_token_empty.reference +++ b/tests/queries/0_stateless/02816_has_token_empty.reference @@ -1,2 +1,6 @@ 0 +\N +\N 0 +\N +\N diff --git a/tests/queries/0_stateless/02816_has_token_empty.sql b/tests/queries/0_stateless/02816_has_token_empty.sql index e5d6156debd..3e00959126b 100644 --- a/tests/queries/0_stateless/02816_has_token_empty.sql +++ b/tests/queries/0_stateless/02816_has_token_empty.sql @@ -2,6 +2,10 @@ SELECT hasTokenCaseInsensitive('K(G', ''); -- { serverError BAD_ARGUMENTS } SELECT hasTokenCaseInsensitive('Hello', ''); -- { serverError BAD_ARGUMENTS } SELECT hasTokenCaseInsensitive('', ''); -- { serverError BAD_ARGUMENTS } SELECT hasTokenCaseInsensitive('', 'Hello'); +SELECT hasTokenCaseInsensitiveOrNull('Hello', ''); +SELECT hasTokenCaseInsensitiveOrNull('', ''); SELECT hasToken('Hello', ''); -- { serverError BAD_ARGUMENTS } SELECT hasToken('', 'Hello'); SELECT hasToken('', ''); -- { serverError BAD_ARGUMENTS } +SELECT hasTokenOrNull('', ''); +SELECT hasTokenOrNull('Hello', ''); From 264bff8c9fcf25e206b86c88e1c3a6f2a5caf8b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 01:43:54 +0200 Subject: [PATCH 110/777] Fix a comment --- base/poco/Data/include/Poco/Data/TypeHandler.h | 2 +- programs/format/Format.cpp | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/base/poco/Data/include/Poco/Data/TypeHandler.h b/base/poco/Data/include/Poco/Data/TypeHandler.h index 34f88e986f7..e7633de7018 100644 --- a/base/poco/Data/include/Poco/Data/TypeHandler.h +++ b/base/poco/Data/include/Poco/Data/TypeHandler.h @@ -97,7 +97,7 @@ namespace Data /// /// static void extract(std::size_t pos, Person& obj, const Person& defVal, AbstractExtractor::Ptr pExt) /// { - /// // defVal is the default person we should use if we encunter NULL entries, so we take the individual fields + /// // defVal is the default person we should use if we encounter NULL entries, so we take the individual fields /// // as defaults. You can do more complex checking, ie return defVal if only one single entry of the fields is null etc... /// poco_assert_dbg (!pExt.isNull()); /// std::string lastName; diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 43c66a32302..d7d61bbcd3b 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -163,13 +163,15 @@ int mainEntryClickHouseFormat(int argc, char ** argv) { ASTPtr res = parseQueryAndMovePosition( parser, pos, end, "query", multiple, cmd_settings.max_query_size, cmd_settings.max_parser_depth); - /// For insert query with data(INSERT INTO ... VALUES ...), will lead to format fail, - /// should throw exception early and make exception message more readable. + + /// For insert query with data(INSERT INTO ... VALUES ...), that will lead to the formatting failure, + /// we should throw an exception early, and make exception message more readable. if (const auto * insert_query = res->as(); insert_query && insert_query->data) { throw Exception(DB::ErrorCodes::INVALID_FORMAT_INSERT_QUERY_WITH_DATA, "Can't format ASTInsertQuery with data, since data will be lost"); } + if (!quiet) { if (!backslash) From 2bda97d8f7e327aee0dcda56f94097ebba9b4e85 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 5 Aug 2023 00:05:50 +0000 Subject: [PATCH 111/777] Potential fix for custom key tests --- src/Processors/QueryPlan/ReadFromRemote.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 4a40d4343fd..5bc082b06ee 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -240,7 +240,8 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact shard.shard_info.pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage); remote_query_executor->setLogger(log); - remote_query_executor->setPoolMode(PoolMode::GET_ONE); + if (context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS) + remote_query_executor->setPoolMode(PoolMode::GET_ONE); if (!table_func_ptr) remote_query_executor->setMainTable(shard.main_table ? shard.main_table : main_table); From f67b51f6703954c8e2b345192318ed651d7336d0 Mon Sep 17 00:00:00 2001 From: jiyoungyoooo <140271086+jiyoungyoooo@users.noreply.github.com> Date: Fri, 4 Aug 2023 22:46:22 -0700 Subject: [PATCH 112/777] Update src/TableFunctions/TableFunctionRemote.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/TableFunctions/TableFunctionRemote.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index d88d4190191..13a24e96e4b 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -159,7 +159,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr { if (arg_num >= args.size()) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name is missing"); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name was not found in function arguments. {}", help_message); } else { From d479c125784010ed516188e948fa6554e72078c7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 5 Aug 2023 14:11:22 +0300 Subject: [PATCH 113/777] Fixed tests --- tests/queries/0_stateless/01655_plan_optimizations.reference | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index be42a656c66..54ca55d2068 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -168,19 +168,23 @@ Filter 3 > one condition of filter is pushed down before LEFT JOIN Join +Join Filter column: notEquals(number, 1) Join > (analyzer) one condition of filter is pushed down before LEFT JOIN Join +Join Filter column: notEquals(number_0, 1_UInt8) 0 0 3 3 > one condition of filter is pushed down before INNER JOIN Join +Join Filter column: notEquals(number, 1) Join > (analyzer) one condition of filter is pushed down before INNER JOIN Join +Join Filter column: notEquals(number_0, 1_UInt8) 3 3 > filter is pushed down before UNION From 9de95879655cb0ebbd895b0e95198e6ad759f09a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 13:54:36 +0200 Subject: [PATCH 114/777] Something with tests --- tests/integration/test_zookeeper_config/test.py | 7 +++---- tests/queries/0_stateless/02263_format_insert_settings.sh | 2 ++ 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 5e36a97b3ca..34ac2cfa572 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -58,9 +58,7 @@ def test_chroot_with_same_root(started_cluster): node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) # Replication might take time - assert_eq_with_retry(node1, "select count() from simple", "2\n") - assert_eq_with_retry(node2, "select count() from simple", "2\n") @@ -78,5 +76,6 @@ def test_chroot_with_different_root(started_cluster): for j in range(2): # Second insert to test deduplication node.query("INSERT INTO simple_different VALUES ({0}, {0})".format(i)) - assert node1.query("select count() from simple_different").strip() == "1" - assert node3.query("select count() from simple_different").strip() == "1" + # Replication might take time + assert_eq_with_retry(node1, "select count() from simple_different", "1\n") + assert_eq_with_retry(node3, "select count() from simple_different", "1\n") diff --git a/tests/queries/0_stateless/02263_format_insert_settings.sh b/tests/queries/0_stateless/02263_format_insert_settings.sh index 3d5f780a38c..efb3d39ab6c 100755 --- a/tests/queries/0_stateless/02263_format_insert_settings.sh +++ b/tests/queries/0_stateless/02263_format_insert_settings.sh @@ -23,10 +23,12 @@ function run_format_both() # NOTE: that those queries may work slow, due to stack trace obtaining run_format 'insert into foo settings max_threads=1' 2> >(grep -m1 -o "Syntax error (query): failed at position .* (end of query):") + # compatibility run_format 'insert into foo format tsv settings max_threads=1' 2> >(grep -m1 -F -o "Can't format ASTInsertQuery with data, since data will be lost.") run_format_both 'insert into foo format tsv settings max_threads=1' --allow_settings_after_format_in_insert run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT") + # and via server (since this is a separate code path) $CLICKHOUSE_CLIENT -q 'drop table if exists data_02263' $CLICKHOUSE_CLIENT -q 'create table data_02263 (key Int) engine=Memory()' From 10af8fcca4fedf6347fe724ed68f1e60b53b2b1a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 14:05:06 +0200 Subject: [PATCH 115/777] Fix Python --- tests/integration/test_zookeeper_config/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 34ac2cfa572..3b5ae7186fb 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -58,8 +58,8 @@ def test_chroot_with_same_root(started_cluster): node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) # Replication might take time - assert_eq_with_retry(node1, "select count() from simple", "2\n") - assert_eq_with_retry(node2, "select count() from simple", "2\n") + assert_eq_with_retry(node1, "select count() from simple", "2") + assert_eq_with_retry(node2, "select count() from simple", "2") def test_chroot_with_different_root(started_cluster): @@ -77,5 +77,5 @@ def test_chroot_with_different_root(started_cluster): node.query("INSERT INTO simple_different VALUES ({0}, {0})".format(i)) # Replication might take time - assert_eq_with_retry(node1, "select count() from simple_different", "1\n") - assert_eq_with_retry(node3, "select count() from simple_different", "1\n") + assert_eq_with_retry(node1, "select count() from simple_different", "1") + assert_eq_with_retry(node3, "select count() from simple_different", "1") From aa57fb6602bd0a815fe512b82df8b18893d268eb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 14:07:55 +0200 Subject: [PATCH 116/777] Maybe better, but not sure --- tests/integration/test_zookeeper_config/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 3b5ae7186fb..4008df3f84f 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -57,8 +57,9 @@ def test_chroot_with_same_root(started_cluster): for j in range(2): # Second insert to test deduplication node.query("INSERT INTO simple VALUES ({0}, {0})".format(i)) - # Replication might take time + node1.query("SYSTEM SYNC REPLICA simple") assert_eq_with_retry(node1, "select count() from simple", "2") + node2.query("SYSTEM SYNC REPLICA simple") assert_eq_with_retry(node2, "select count() from simple", "2") @@ -76,6 +77,7 @@ def test_chroot_with_different_root(started_cluster): for j in range(2): # Second insert to test deduplication node.query("INSERT INTO simple_different VALUES ({0}, {0})".format(i)) - # Replication might take time + node1.query("SYSTEM SYNC REPLICA simple_different") assert_eq_with_retry(node1, "select count() from simple_different", "1") + node2.query("SYSTEM SYNC REPLICA simple_different") assert_eq_with_retry(node3, "select count() from simple_different", "1") From 0a6d9881a72c9476ffeb6263f924150bf3116f29 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 5 Aug 2023 16:52:44 +0200 Subject: [PATCH 117/777] Maybe better, but not sure --- tests/integration/test_zookeeper_config/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 4008df3f84f..0c0f77ec597 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -79,5 +79,5 @@ def test_chroot_with_different_root(started_cluster): node1.query("SYSTEM SYNC REPLICA simple_different") assert_eq_with_retry(node1, "select count() from simple_different", "1") - node2.query("SYSTEM SYNC REPLICA simple_different") + node3.query("SYSTEM SYNC REPLICA simple_different") assert_eq_with_retry(node3, "select count() from simple_different", "1") From da2cb99c126c6610150d71c736d1e7e671eb2659 Mon Sep 17 00:00:00 2001 From: Jiyoung Yoo Date: Mon, 31 Jul 2023 08:09:57 -0700 Subject: [PATCH 118/777] Fix for issue #40220 improve error message for table function remote --- src/TableFunctions/TableFunctionRemote.cpp | 13 +++++++++++-- .../02841_remote_parameter_parsing_error.reference | 0 .../02841_remote_parameter_parsing_error.sql | 14 ++++++++++++++ 3 files changed, 25 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference create mode 100644 tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 78618d4928f..0d015196b3c 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -175,7 +175,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr { if (arg_num >= args.size()) { - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name was not found in function arguments. {}", help_message); } else { @@ -228,7 +228,16 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr } if (arg_num < args.size()) - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + { + if (is_cluster_function) + { + throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "user and password should be string literal (in single quotes)"); + } + } } if (!cluster_name.empty()) diff --git a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql new file mode 100644 index 00000000000..f2ac88b62fb --- /dev/null +++ b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql @@ -0,0 +1,14 @@ +-- Tags: shard + + +select * from remote('127.0.0.1'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', sys); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', system); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +select * from remote('127.0.0.1', system.o); -- { serverError UNKNOWN_TABLE } +select * from remote('127.0.0.1', system.one, default); -- { serverError UNKNOWN_IDENTIFIER } +select * from remote('127.0.0.1', system.one, default, ''); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, default, key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, 'default', '', key1); -- { serverError UNKNOWN_IDENTIFIER } +select * from remote('127.0.0.1', system.one, default, '', key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, 'default', pwd, key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system, one, 'default', '', key1, key2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From 608b5ace3610cf970e898e34b450f54b693faca5 Mon Sep 17 00:00:00 2001 From: Jiyoung Yoo Date: Sat, 5 Aug 2023 00:26:32 -0700 Subject: [PATCH 119/777] Additional fix for issue#40220 --- src/TableFunctions/TableFunctionRemote.cpp | 15 +++++---------- .../02841_remote_parameter_parsing_error.sql | 2 +- 2 files changed, 6 insertions(+), 11 deletions(-) diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 0d015196b3c..d00cda2462f 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -175,7 +175,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr { if (arg_num >= args.size()) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name was not found in function arguments. {}", help_message); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table name was not found in function arguments. {}", static_cast(help_message)); } else { @@ -220,8 +220,10 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr ++arg_num; } - if (arg_num < args.size() && !sharding_key) + if (arg_num < args.size()) { + if (sharding_key) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "user and password should be string literal (in single quotes)"); sharding_key = args[arg_num]; ++arg_num; } @@ -229,14 +231,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr if (arg_num < args.size()) { - if (is_cluster_function) - { - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "user and password should be string literal (in single quotes)"); - } + throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } } diff --git a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql index f2ac88b62fb..9e467a1f69a 100644 --- a/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql +++ b/tests/queries/0_stateless/02841_remote_parameter_parsing_error.sql @@ -1,7 +1,6 @@ -- Tags: shard -select * from remote('127.0.0.1'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select * from remote('127.0.0.1', sys); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select * from remote('127.0.0.1', system); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select * from remote('127.0.0.1', system.o); -- { serverError UNKNOWN_TABLE } @@ -11,4 +10,5 @@ select * from remote('127.0.0.1', system.one, default, key1); -- { serverError B select * from remote('127.0.0.1', system.one, 'default', '', key1); -- { serverError UNKNOWN_IDENTIFIER } select * from remote('127.0.0.1', system.one, default, '', key1); -- { serverError BAD_ARGUMENTS } select * from remote('127.0.0.1', system.one, 'default', pwd, key1); -- { serverError BAD_ARGUMENTS } +select * from remote('127.0.0.1', system.one, 'default', '', key1, key2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } select * from remote('127.0.0.1', system, one, 'default', '', key1, key2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From afb1a55bf3789196e2b9f96aea5318ea0a9f9219 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Aug 2023 00:56:33 +0200 Subject: [PATCH 120/777] Remove whitespaces --- src/Interpreters/SystemLog.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 437b1b2a6bb..c4c2b0b2499 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -131,8 +131,6 @@ protected: using Base::queue; private: - - /* Saving thread data */ const StorageID table_id; const String storage_def; From 7e90df67a746fb12a9d4ecb6944beeab198d869d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Aug 2023 00:58:26 +0200 Subject: [PATCH 121/777] Remove whitespaces --- src/Interpreters/SystemLog.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index c4c2b0b2499..cf287ad7775 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -111,7 +111,7 @@ public: * and new table get created - as if previous table was not exist. */ SystemLog(ContextPtr context_, - const SystemLogSettings& settings_, + const SystemLogSettings & settings_, std::shared_ptr> queue_ = nullptr); /** Append a record into log. From a8b782ca5a2a3a916001397a0a200d8de50c9fa6 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 5 Aug 2023 23:19:09 +0000 Subject: [PATCH 122/777] Parallel reading from replicas over disributed with several shards --- .../ClusterProxy/executeQuery.cpp | 5 --- src/Processors/QueryPlan/ReadFromRemote.cpp | 10 +++++ tests/config/config.d/clusters.xml | 32 ++++++++++++++ ...rallel_replicas_over_distributed.reference | 7 +++- ...835_parallel_replicas_over_distributed.sql | 42 +++++++++++++++++-- 5 files changed, 86 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 2fed626ffb7..de3150aa69e 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -270,11 +270,6 @@ void executeQueryWithParallelReplicas( const ASTPtr & query_ast, ContextPtr context, const SelectQueryInfo & query_info, const ClusterPtr & not_optimized_cluster) { - if (not_optimized_cluster->getShardsInfo().size() != 1) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Cluster for parallel replicas should consist only from one shard"); - - auto shard_info = not_optimized_cluster->getShardsInfo().front(); - const auto & settings = context->getSettingsRef(); ClusterPtr new_cluster = not_optimized_cluster->getClusterWithReplicasAsShards(settings); diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 5bc082b06ee..287ceece5bd 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -240,8 +240,18 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact shard.shard_info.pool, query_string, output_stream->header, context, throttler, scalars, external_tables, stage); remote_query_executor->setLogger(log); + if (context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS) + // when doing parallel reading from replicas (ParallelReplicasMode::READ_TASKS) on a shard: + // establish a connection to a replica on the shard, the replica will instantiate coordinator to manage parallel reading from replicas on the shard. + // The coordinator will return query result from the shard. + // Only one coordinator per shard is necessary. Therefore using PoolMode::GET_ONE to establish only one connection per shard. + // Using PoolMode::GET_MANY for this mode will(can) lead to instatiation of several coordinators (depends on max_parallel_replicas setting) + // each will execute parallel reading from replicas, + // so the query result will be multiplied by the number of created coordinator remote_query_executor->setPoolMode(PoolMode::GET_ONE); + else + remote_query_executor->setPoolMode(PoolMode::GET_MANY); if (!table_func_ptr) remote_query_executor->setMainTable(shard.main_table ? shard.main_table : main_table); diff --git a/tests/config/config.d/clusters.xml b/tests/config/config.d/clusters.xml index 031d6e64bc9..cfd4868f1dc 100644 --- a/tests/config/config.d/clusters.xml +++ b/tests/config/config.d/clusters.xml @@ -176,6 +176,38 @@ + + + false + + 127.0.0.1 + 9000 + + + 127.0.0.2 + 9000 + + + 127.0.0.3 + 9000 + + + + false + + 127.0.0.4 + 9000 + + + 127.0.0.5 + 9000 + + + 127.0.0.6 + 9000 + + + diff --git a/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.reference b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.reference index ea4483ec305..e6a24987c0d 100644 --- a/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.reference +++ b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.reference @@ -1 +1,6 @@ -100 4950 +-- 1 shard, 3 replicas +100 0 99 49.5 +200 0 99 49.5 +-- 2 shards, 3 replicas each +200 0 99 49.5 +400 0 99 49.5 diff --git a/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql index 199be3771c7..4e7c72ebfd8 100644 --- a/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql +++ b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql @@ -1,13 +1,47 @@ +-- 1 shard + +SELECT '-- 1 shard, 3 replicas'; DROP TABLE IF EXISTS test_d; DROP TABLE IF EXISTS test; CREATE TABLE test (id UInt64, date Date) ENGINE = MergeTree -ORDER BY id -as select *, today() from numbers(100); +ORDER BY id; CREATE TABLE IF NOT EXISTS test_d as test ENGINE = Distributed(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), test); -SELECT count(), sum(id) +insert into test select *, today() from numbers(100); + +SELECT count(), min(id), max(id), avg(id) FROM test_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; + +insert into test select *, today() from numbers(100); + +SELECT count(), min(id), max(id), avg(id) +FROM test_d +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; + +-- 2 shards + +SELECT '-- 2 shards, 3 replicas each'; +DROP TABLE IF EXISTS test2_d; +DROP TABLE IF EXISTS test2; +CREATE TABLE test2 (id UInt64, date Date) +ENGINE = MergeTree +ORDER BY id; + +CREATE TABLE IF NOT EXISTS test2_d as test2 +ENGINE = Distributed(test_cluster_two_shard_three_replicas_localhost, currentDatabase(), test2, id); + +insert into test2 select *, today() from numbers(100); + +SELECT count(), min(id), max(id), avg(id) +FROM test2_d +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0, cluster_for_parallel_replicas = 'test_cluster_two_shard_three_replicas_localhost'; + +insert into test2 select *, today() from numbers(100); + +SELECT count(), min(id), max(id), avg(id) +FROM test2_d +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0, cluster_for_parallel_replicas = 'test_cluster_two_shard_three_replicas_localhost'; From 06d45cfa81c2e21963a1a82bb7fabc1bcb9b115b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 6 Aug 2023 01:29:31 +0200 Subject: [PATCH 123/777] Allow creating system logs at startup --- programs/server/config.xml | 3 +++ src/Interpreters/SystemLog.cpp | 6 ++++++ src/Interpreters/SystemLog.h | 3 +++ 3 files changed, 12 insertions(+) diff --git a/programs/server/config.xml b/programs/server/config.xml index 14b8954fc39..3d1c92b135c 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1037,6 +1037,9 @@ + + + false +**Example** ``` sql SELECT arrayConcat([1, 2], [3, 4], [5, 6]) AS res diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 7ed2deaeda6..88e4ac03fdb 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -559,6 +559,29 @@ Result: └────────────────────────────┘ ``` +## tupleConcat + +Combines tuples passed as arguments. + +``` sql +tupleConcat(tuples) +``` + +**Arguments** + +- `tuples` – Arbitrary number of arguments of [Tuple](../../sql-reference/data-types/tuple.md) type. + +**Example** + +``` sql +SELECT tupleConcat((1, 2), (3, 4), (true, false)) AS res +``` + +``` text +┌─res──────────────────┐ +│ (1,2,3,4,true,false) │ +└──────────────────────┘ +``` ## Distance functions diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index 8288d872f18..9eb222d8c09 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -208,6 +208,10 @@ public: { return FunctionFactory::instance().getImpl("mapConcat", context)->build(arguments); } + else if (isTuple(arguments.at(0).type)) + { + return FunctionFactory::instance().getImpl("tupleConcat", context)->build(arguments); + } else return std::make_unique( FunctionConcat::create(context), collections::map(arguments, [](const auto & elem) { return elem.type; }), return_type); diff --git a/src/Functions/tupleConcat.cpp b/src/Functions/tupleConcat.cpp new file mode 100644 index 00000000000..0556f4181e6 --- /dev/null +++ b/src/Functions/tupleConcat.cpp @@ -0,0 +1,102 @@ +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; +} + +/// tupleConcat(tup1, ...) - concatenate tuples. +class FunctionTupleConcat : public IFunction +{ +public: + static constexpr auto name = "tupleConcat"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + + size_t getNumberOfArguments() const override { return 0; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} requires at least one argument.", + getName()); + + DataTypes tuple_arg_types; + + for (const auto arg_idx : collections::range(0, arguments.size())) + { + const auto * arg = arguments[arg_idx].get(); + if (!isTuple(arg)) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument {} of function {}", + arg->getName(), + arg_idx + 1, + getName()); + + const auto * type = checkAndGetDataType(arg); + for (const auto & elem : type->getElements()) + tuple_arg_types.push_back(elem); + } + + return std::make_shared(tuple_arg_types); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + { + const size_t num_arguments = arguments.size(); + Columns columns; + + for (size_t i = 0; i < num_arguments; i++) + { + const DataTypeTuple * arg_type = checkAndGetDataType(arguments[i].type.get()); + + if (!arg_type) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument {} of function {}", + arguments[i].type->getName(), + i + 1, + getName()); + + ColumnPtr arg_col = arguments[i].column->convertToFullColumnIfConst(); + const ColumnTuple * tuple_col = checkAndGetColumn(arg_col.get()); + + if (!tuple_col) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of argument of function {}", + arguments[i].column->getName(), + getName()); + + for (const auto & inner_col : tuple_col->getColumns()) + columns.push_back(inner_col); + } + + return ColumnTuple::create(columns); + } +}; + +REGISTER_FUNCTION(TupleConcat) +{ + factory.registerFunction(); +} + +} diff --git a/tests/queries/0_stateless/02833_tuple_concat.reference b/tests/queries/0_stateless/02833_tuple_concat.reference new file mode 100644 index 00000000000..2c865f13ffc --- /dev/null +++ b/tests/queries/0_stateless/02833_tuple_concat.reference @@ -0,0 +1,6 @@ +(1,'y',2,'n') +(1,'y',2,'n',3,'n') +(1,2,3,'a','b','c','2020-10-08','2020-11-08') 1 2 3 a b c 2020-10-08 2020-11-08 +(1,2,1,2) 1 2 1 2 +(1,2,3,4) 1 2 3 4 +(3,4,1,2) 3 4 1 2 diff --git a/tests/queries/0_stateless/02833_tuple_concat.sql b/tests/queries/0_stateless/02833_tuple_concat.sql new file mode 100644 index 00000000000..df43e08d595 --- /dev/null +++ b/tests/queries/0_stateless/02833_tuple_concat.sql @@ -0,0 +1,23 @@ +SELECT tupleConcat(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT tupleConcat((1, 'y'), 1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT tupleConcat((1, 'y'), (2, 'n')); +SELECT tupleConcat((1, 'y'), (2, 'n'), (3, 'n')); + +WITH (1,2,3) || ('a','b','c') || ('2020-10-08'::Date, '2020-11-08'::Date) AS t +SELECT t, t.1, t.2, t.3, t.4, t.5, t.6, t.7, t.8; + +DROP TABLE IF EXISTS t_02833; +CREATE TABLE t_02833 (tup Tuple(a UInt64, b UInt64)) ENGINE=Log; +INSERT INTO t_02833 VALUES ((1, 2)); + +WITH (tup || tup) AS res +SELECT res, res.1, res.2, res.3, res.4 FROM t_02833; + +WITH (tup || (3, 4)) AS res +SELECT res, res.1, res.2, res.3, res.4 FROM t_02833; + +WITH ((3, 4) || tup) AS res +SELECT res, res.1, res.2, res.3, res.4 FROM t_02833; + +DROP TABLE t_02833; From 0498b48c70aa60b49ae5a2f6ee702f564921062d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 9 Aug 2023 21:01:58 +0000 Subject: [PATCH 283/777] different linker --- cmake/linux/toolchain-s390x.cmake | 2 ++ 1 file changed, 2 insertions(+) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index c1cbb849bd3..f5e1cf660c9 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -22,6 +22,8 @@ set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") +set (CMAKE_STATIC_LINKER_FLAGS "${CMAKE_STATIC_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From 481792181e6ac9515db966817d123afb99d9971f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 9 Aug 2023 23:12:39 +0200 Subject: [PATCH 284/777] Remove garbage --- .../test_cluster_copier/test_two_nodes.py | 21 - .../test_cassandra.py | 99 - .../test.py | 68 - .../test_default_reading.py | 1 - .../test_default_string.py | 1 - .../test_dict_get.py | 1 - .../test_dict_get_or_default.py | 1 - .../test_extreme_deduplication/test.py | 70 - tests/integration/test_hive_query/__init__.py | 0 .../test_hive_query/configs/config.xml | 32 - .../test_hive_query/configs/hdfs-site.xml | 6 - .../test_hive_query/data/prepare_hive_data.sh | 15 - tests/integration/test_hive_query/test.py | 498 ----- .../__init__.py | 0 .../configs/host_regexp.xml | 11 - .../configs/listen_host.xml | 5 - .../coredns_config/Corefile | 8 - .../coredns_config/example.com | 1 - .../test.py | 115 - .../test.py | 68 - .../test.py | 78 - .../test_mutations_with_merge_tree/test.py | 194 -- .../test_polymorphic_parts/test.py | 48 - .../__init__.py | 0 .../configs/config.d/storage_conf.xml | 93 - .../test.py | 322 --- .../test_reverse_dns_query/__init__.py | 0 .../test_reverse_dns_query/configs/config.xml | 3 - .../configs/listen_host.xml | 5 - .../configs/reverse_dns_function.xml | 3 - .../coredns_config/Corefile | 4 - .../test_reverse_dns_query/test.py | 56 - .../test_s3_zero_copy_replication/test.py | 90 - .../test_s3_zero_copy_ttl/__init__.py | 0 .../configs/max_delayed_streams.xml | 9 - .../test_s3_zero_copy_ttl/configs/s3.xml | 39 - .../integration/test_s3_zero_copy_ttl/test.py | 94 - .../test_ttl_move_memory_usage.py | 81 - .../test_vertical_merge_memory_usage.py | 46 - .../integration/test_storage_nats/__init__.py | 0 .../clickhouse_path/format_schemas/nats.proto | 6 - .../test_storage_nats/configs/macros.xml | 7 - .../configs/named_collection.xml | 13 - .../test_storage_nats/configs/nats.xml | 6 - .../test_storage_nats/configs/users.xml | 7 - .../test_storage_nats/nats_certs.sh | 13 - .../integration/test_storage_nats/nats_pb2.py | 37 - tests/integration/test_storage_nats/test.py | 1875 ----------------- .../test_nats_json.reference | 50 - .../__init__.py | 0 .../configs/log_conf.xml | 11 - .../test_storage_postgresql_replica/test.py | 780 ------- .../test_postgresql_replica.reference | 50 - .../integration/test_storage_rabbitmq/test.py | 63 - tests/integration/test_ttl_move/test.py | 100 - 55 files changed, 5204 deletions(-) delete mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py delete mode 100644 tests/integration/test_hive_query/__init__.py delete mode 100644 tests/integration/test_hive_query/configs/config.xml delete mode 100644 tests/integration/test_hive_query/configs/hdfs-site.xml delete mode 100755 tests/integration/test_hive_query/data/prepare_hive_data.sh delete mode 100644 tests/integration/test_hive_query/test.py delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/__init__.py delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com delete mode 100644 tests/integration/test_host_regexp_multiple_ptr_records/test.py delete mode 100644 tests/integration/test_replicated_merge_tree_hdfs_zero_copy/__init__.py delete mode 100644 tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml delete mode 100644 tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py delete mode 100644 tests/integration/test_reverse_dns_query/__init__.py delete mode 100644 tests/integration/test_reverse_dns_query/configs/config.xml delete mode 100644 tests/integration/test_reverse_dns_query/configs/listen_host.xml delete mode 100644 tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml delete mode 100644 tests/integration/test_reverse_dns_query/coredns_config/Corefile delete mode 100644 tests/integration/test_reverse_dns_query/test.py delete mode 100644 tests/integration/test_s3_zero_copy_ttl/__init__.py delete mode 100644 tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml delete mode 100644 tests/integration/test_s3_zero_copy_ttl/configs/s3.xml delete mode 100644 tests/integration/test_s3_zero_copy_ttl/test.py delete mode 100644 tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py delete mode 100644 tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py delete mode 100644 tests/integration/test_storage_nats/__init__.py delete mode 100755 tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto delete mode 100644 tests/integration/test_storage_nats/configs/macros.xml delete mode 100644 tests/integration/test_storage_nats/configs/named_collection.xml delete mode 100644 tests/integration/test_storage_nats/configs/nats.xml delete mode 100644 tests/integration/test_storage_nats/configs/users.xml delete mode 100755 tests/integration/test_storage_nats/nats_certs.sh delete mode 100644 tests/integration/test_storage_nats/nats_pb2.py delete mode 100644 tests/integration/test_storage_nats/test.py delete mode 100644 tests/integration/test_storage_nats/test_nats_json.reference delete mode 100644 tests/integration/test_storage_postgresql_replica/__init__.py delete mode 100644 tests/integration/test_storage_postgresql_replica/configs/log_conf.xml delete mode 100644 tests/integration/test_storage_postgresql_replica/test.py delete mode 100644 tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference diff --git a/tests/integration/test_cluster_copier/test_two_nodes.py b/tests/integration/test_cluster_copier/test_two_nodes.py index 2b6fcf6cac2..1bd3561f24f 100644 --- a/tests/integration/test_cluster_copier/test_two_nodes.py +++ b/tests/integration/test_cluster_copier/test_two_nodes.py @@ -595,24 +595,3 @@ def execute_task(started_cluster, task, cmd_options): task.check() finally: zk.delete(task.zk_task_path, recursive=True) - - -# Tests -@pytest.mark.skip(reason="Too flaky :(") -def test_different_schema(started_cluster): - execute_task(started_cluster, TaskWithDifferentSchema(started_cluster), []) - - -@pytest.mark.skip(reason="Too flaky :(") -def test_ttl_columns(started_cluster): - execute_task(started_cluster, TaskTTL(started_cluster), []) - - -@pytest.mark.skip(reason="Too flaky :(") -def test_skip_index(started_cluster): - execute_task(started_cluster, TaskSkipIndex(started_cluster), []) - - -@pytest.mark.skip(reason="Too flaky :(") -def test_ttl_move_to_volume(started_cluster): - execute_task(started_cluster, TaskTTLMoveToVolume(started_cluster), []) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py deleted file mode 100644 index 90287e19bd0..00000000000 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py +++ /dev/null @@ -1,99 +0,0 @@ -import os -import math -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/33006 - -pytestmark = pytest.mark.skip - -from .common import * - -from helpers.cluster import ClickHouseCluster -from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout -from helpers.external_sources import SourceCassandra - -SOURCE = None -cluster = None -node = None -simple_tester = None -complex_tester = None -ranged_tester = None -test_name = "cassandra" - - -def setup_module(module): - global cluster - global node - global simple_tester - global complex_tester - global ranged_tester - - cluster = ClickHouseCluster(__file__) - - SOURCE = SourceCassandra( - "Cassandra", - None, - cluster.cassandra_port, - cluster.cassandra_host, - cluster.cassandra_port, - "", - "", - ) - - simple_tester = SimpleLayoutTester(test_name) - simple_tester.cleanup() - simple_tester.create_dictionaries(SOURCE) - - complex_tester = ComplexLayoutTester(test_name) - complex_tester.create_dictionaries(SOURCE) - - ranged_tester = RangedLayoutTester(test_name) - ranged_tester.create_dictionaries(SOURCE) - # Since that all .xml configs were created - - main_configs = [] - main_configs.append(os.path.join("configs", "disable_ssl_verification.xml")) - - dictionaries = simple_tester.list_dictionaries() - - node = cluster.add_instance( - "cass_node", - main_configs=main_configs, - dictionaries=dictionaries, - with_cassandra=True, - ) - - -def teardown_module(module): - simple_tester.cleanup() - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - simple_tester.prepare(cluster) - complex_tester.prepare(cluster) - ranged_tester.prepare(cluster) - - yield cluster - - finally: - cluster.shutdown() - - -@pytest.mark.parametrize("layout_name", sorted(LAYOUTS_SIMPLE)) -def test_simple(started_cluster, layout_name): - simple_tester.execute(layout_name, node) - - -@pytest.mark.parametrize("layout_name", sorted(LAYOUTS_COMPLEX)) -def test_complex(started_cluster, layout_name): - complex_tester.execute(layout_name, node) - - -@pytest.mark.parametrize("layout_name", sorted(LAYOUTS_RANGED)) -def test_ranged(started_cluster, layout_name): - ranged_tester.execute(layout_name, node) diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/test.py b/tests/integration/test_dictionaries_complex_key_cache_string/test.py index ae9cd4e7c91..bca7122a2a8 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/test.py +++ b/tests/integration/test_dictionaries_complex_key_cache_string/test.py @@ -27,71 +27,3 @@ def started_cluster(): yield cluster finally: cluster.shutdown() - - -@pytest.mark.skip(reason="SSD cache test can run on disk only") -@pytest.mark.parametrize("type", ["memory", "ssd"]) -def test_memory_consumption(started_cluster, type): - node = started_cluster.instances[f"node_{type}"] - node.query( - "insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format( - "w" * 8 - ) - ) - node.query( - "insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format( - "x" * 16 - ) - ) - node.query( - "insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format( - "y" * 32 - ) - ) - node.query( - "insert into radars_table select toString(rand() % 5000), '{0}', '{0}' from numbers(1000)".format( - "z" * 64 - ) - ) - - # Fill dictionary - node.query( - "select dictGetString('radars', 'client_id', tuple(toString(number))) from numbers(0, 5000)" - ) - - allocated_first = int( - node.query( - "select bytes_allocated from system.dictionaries where name = 'radars'" - ).strip() - ) - - alloc_array = [] - for i in range(5): - node.query( - "select dictGetString('radars', 'client_id', tuple(toString(number))) from numbers(0, 5000)" - ) - - allocated = int( - node.query( - "select bytes_allocated from system.dictionaries where name = 'radars'" - ).strip() - ) - alloc_array.append(allocated) - - # size doesn't grow - assert all(allocated_first >= a for a in alloc_array) - - for i in range(5): - node.query( - "select dictGetString('radars', 'client_id', tuple(toString(number))) from numbers(0, 5000)" - ) - - allocated = int( - node.query( - "select bytes_allocated from system.dictionaries where name = 'radars'" - ).strip() - ) - alloc_array.append(allocated) - - # size doesn't grow - assert all(allocated_first >= a for a in alloc_array) diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py index 85c45d5df3c..921fb4e4154 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py @@ -38,7 +38,6 @@ def started_cluster(): cluster.shutdown() -# @pytest.mark.skip(reason="debugging") def test_default_reading(started_cluster): assert None != dictionary_node.get_process_pid( "clickhouse" diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py index 92d681698bc..6eedf63f95c 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py @@ -53,7 +53,6 @@ def started_cluster(): cluster.shutdown() -# @pytest.mark.skip(reason="debugging") def test_return_real_values(started_cluster): assert None != dictionary_node.get_process_pid( "clickhouse" diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py index 05f638ec337..8e45af44640 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py @@ -38,7 +38,6 @@ def started_cluster(): cluster.shutdown() -# @pytest.mark.skip(reason="debugging") def test_simple_dict_get(started_cluster): assert None != dictionary_node.get_process_pid( "clickhouse" diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py index 1da8fd3325a..8d6242f4711 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py @@ -38,7 +38,6 @@ def started_cluster(): cluster.shutdown() -# @pytest.mark.skip(reason="debugging") def test_simple_dict_get_or_default(started_cluster): assert None != dictionary_node.get_process_pid( "clickhouse" diff --git a/tests/integration/test_extreme_deduplication/test.py b/tests/integration/test_extreme_deduplication/test.py index 03451933171..3632369154a 100644 --- a/tests/integration/test_extreme_deduplication/test.py +++ b/tests/integration/test_extreme_deduplication/test.py @@ -71,73 +71,3 @@ def test_deduplication_window_in_seconds(started_cluster): assert TSV.toMat(node.query("SELECT count() FROM simple"))[0][0] == "3" node1.query("""DROP TABLE simple ON CLUSTER test_cluster""") - - -# Currently this test just reproduce incorrect behavior that sould be fixed -@pytest.mark.skip(reason="Flapping test") -def test_deduplication_works_in_case_of_intensive_inserts(started_cluster): - inserters = [] - fetchers = [] - - node1.query( - """ - CREATE TABLE simple ON CLUSTER test_cluster (date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/{shard}/simple', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id""" - ) - - node1.query("INSERT INTO simple VALUES (0, 0)") - - for node in nodes: - host = node.ip_address - - inserters.append( - CommandRequest( - ["/bin/bash"], - timeout=10, - stdin=""" -set -e -for i in `seq 1000`; do - {} --host {} -q "INSERT INTO simple VALUES (0, 0)" -done -""".format( - cluster.get_client_cmd(), host - ), - ) - ) - - fetchers.append( - CommandRequest( - ["/bin/bash"], - timeout=10, - stdin=""" -set -e -for i in `seq 1000`; do - res=`{} --host {} -q "SELECT count() FROM simple"` - if [[ $? -ne 0 || $res -ne 1 ]]; then - echo "Selected $res elements! Host: {}" 1>&2 - exit -1 - fi; -done -""".format( - cluster.get_client_cmd(), host, node.name - ), - ) - ) - - # There were not errors during INSERTs - for inserter in inserters: - try: - inserter.get_answer() - except QueryTimeoutExceedException: - # Only timeout is accepted - pass - - # There were not errors during SELECTs - for fetcher in fetchers: - try: - fetcher.get_answer() - except QueryTimeoutExceedException: - # Only timeout is accepted - pass - - node1.query("""DROP TABLE simple ON CLUSTER test_cluster""") diff --git a/tests/integration/test_hive_query/__init__.py b/tests/integration/test_hive_query/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_hive_query/configs/config.xml b/tests/integration/test_hive_query/configs/config.xml deleted file mode 100644 index 7de1391e56c..00000000000 --- a/tests/integration/test_hive_query/configs/config.xml +++ /dev/null @@ -1,32 +0,0 @@ - - - - - - - localhost - 9000 - - - - - - - true - /tmp/clickhouse_local_cache,/tmp/clickhouse_local_cache1 - 207374182400 - 1048576 - - - - /etc/clickhouse-server/extra_conf.d/hdfs-site.xml - - - - system - metric_log
- 1000 - 1000 -
- -
diff --git a/tests/integration/test_hive_query/configs/hdfs-site.xml b/tests/integration/test_hive_query/configs/hdfs-site.xml deleted file mode 100644 index 82c525ea414..00000000000 --- a/tests/integration/test_hive_query/configs/hdfs-site.xml +++ /dev/null @@ -1,6 +0,0 @@ - - - dfs.replication - 1 - - diff --git a/tests/integration/test_hive_query/data/prepare_hive_data.sh b/tests/integration/test_hive_query/data/prepare_hive_data.sh deleted file mode 100755 index 495ea201870..00000000000 --- a/tests/integration/test_hive_query/data/prepare_hive_data.sh +++ /dev/null @@ -1,15 +0,0 @@ -#!/bin/bash -hive -e "create database test" - -hive -e "drop table if exists test.demo; create table test.demo(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'; create table test.demo_orc(id string, score int) PARTITIONED BY(day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'; " -hive -e "drop table if exists test.parquet_demo; create table test.parquet_demo(id string, score int) PARTITIONED BY(day string, hour string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'" -hive -e "drop table if exists test.demo_text; create table test.demo_text(id string, score int, day string)row format delimited fields terminated by ','; load data local inpath '/demo_data.txt' into table test.demo_text " -hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.demo partition(day) select * from test.demo_text; insert into test.demo_orc partition(day) select * from test.demo_text" - -hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '00' as hour from test.demo;" -hive -e "set hive.exec.dynamic.partition.mode=nonstrict;insert into test.parquet_demo partition(day, hour) select id, score, day, '01' as hour from test.demo;" - -hive -e "drop table if exists test.test_hive_types; CREATE TABLE test.test_hive_types( f_tinyint tinyint, f_smallint smallint, f_int int, f_integer int, f_bigint bigint, f_float float, f_double double, f_decimal decimal(10,0), f_timestamp timestamp, f_date date, f_string string, f_varchar varchar(100), f_char char(100), f_bool boolean, f_array_int array, f_array_string array, f_array_float array, f_map_int map, f_map_string map, f_map_float map, f_struct struct>) PARTITIONED BY( day string) ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat';" - -hive -e "insert into test.test_hive_types partition(day='2022-02-20') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-20 14:47:04', '2022-02-20', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3), 'd', named_struct('x', 10, 'y', 'xyz')); insert into test.test_hive_types partition(day='2022-02-19') select 1, 2, 3, 4, 5, 6.11, 7.22, 8.333, '2022-02-19 14:47:04', '2022-02-19', 'hello world', 'hello world', 'hello world', true, array(1,2,3), array('hello world', 'hello world'), array(float(1.1),float(1.2)), map('a', 100, 'b', 200, 'c', 300), map('a', 'aa', 'b', 'bb', 'c', 'cc'), map('a', float(111.1), 'b', float(222.2), 'c', float(333.3)), named_struct('a', 'aaa', 'b', 200, 'c', float(333.3), 'd', named_struct('x', 11, 'y', 'abc'));" - diff --git a/tests/integration/test_hive_query/test.py b/tests/integration/test_hive_query/test.py deleted file mode 100644 index 791ae03f9f6..00000000000 --- a/tests/integration/test_hive_query/test.py +++ /dev/null @@ -1,498 +0,0 @@ -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/43541 - -pytestmark = pytest.mark.skip - -import logging -import os - -import time -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import TSV - -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "h0_0_0", - main_configs=["configs/config.xml"], - extra_configs=["configs/hdfs-site.xml", "data/prepare_hive_data.sh"], - with_hive=True, - ) - - logging.info("Starting cluster ...") - cluster.start() - cluster.copy_file_to_container( - "roottesthivequery_hdfs1_1", - "/ClickHouse/tests/integration/test_hive_query/data/prepare_hive_data.sh", - "/prepare_hive_data.sh", - ) - cluster.exec_in_container( - "roottesthivequery_hdfs1_1", ["bash", "-c", "bash /prepare_hive_data.sh"] - ) - yield cluster - finally: - cluster.shutdown() - - -def test_create_parquet_table(started_cluster): - logging.info("Start testing creating hive table ...") - node = started_cluster.instances["h0_0_0"] - test_passed = False - for i in range(10): - node.query("set input_format_parquet_allow_missing_columns = true") - result = node.query( - """ -DROP TABLE IF EXISTS default.demo_parquet; -CREATE TABLE default.demo_parquet (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - """ - ) - logging.info("create result {}".format(result)) - if result.strip() == "": - test_passed = True - break - time.sleep(60) - assert test_passed - - -def test_create_parquet_table_1(started_cluster): - logging.info("Start testing creating hive table ...") - node = started_cluster.instances["h0_0_0"] - for i in range(10): - node.query("set input_format_parquet_allow_missing_columns = true") - result = node.query( - """ -DROP TABLE IF EXISTS default.demo_parquet_parts; -CREATE TABLE default.demo_parquet_parts (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String), `hour` String) ENGINE = Hive('thrift://hivetest:9083', 'test', 'parquet_demo') PARTITION BY(day, hour); - """ - ) - logging.info("create result {}".format(result)) - if result.strip() == "": - test_passed = True - break - time.sleep(60) - assert test_passed - - -def test_create_orc_table(started_cluster): - logging.info("Start testing creating hive table ...") - node = started_cluster.instances["h0_0_0"] - test_passed = False - for i in range(10): - result = node.query( - """ - DROP TABLE IF EXISTS default.demo_orc; - CREATE TABLE default.demo_orc (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day) - """ - ) - logging.info("create result {}".format(result)) - if result.strip() == "": - test_passed = True - break - time.sleep(60) - - assert test_passed - - -def test_create_text_table(started_cluster): - logging.info("Start testing creating hive table ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - DROP TABLE IF EXISTS default.demo_text; - CREATE TABLE default.demo_text (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_text') PARTITION BY (tuple()) - """ - ) - logging.info("create result {}".format(result)) - - assert result.strip() == "" - - -def test_parquet_groupby(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - SELECT day, count(*) FROM default.demo_parquet group by day order by day - """ - ) - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" - assert result == expected_result - - -def test_parquet_in_filter(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - SELECT count(*) FROM default.demo_parquet_parts where day = '2021-11-05' and hour in ('00') - """ - ) - expected_result = """2 -""" - logging.info("query result:{}".format(result)) - assert result == expected_result - - -def test_orc_groupby(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - DROP TABLE IF EXISTS default.demo_orc; - CREATE TABLE default.demo_orc (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day); - SELECT day, count(*) FROM default.demo_orc group by day order by day - """ - ) - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" - assert result == expected_result - - -@pytest.mark.parametrize( - "table,use_local_cache_for_remote_storage,enable_orc_file_minmax_index,enable_orc_stripe_minmax_index", - [ - pytest.param( - "demo_orc_no_cache_no_index", - "false", - "false", - "false", - id="demo_orc_no_cache_no_index", - ), - pytest.param( - "demo_orc_with_cache_no_index", - "true", - "false", - "false", - id="demo_orc_with_cache_no_index", - ), - pytest.param( - "demo_orc_no_cache_file_index", - "false", - "true", - "false", - id="demo_orc_no_cache_file_index", - ), - pytest.param( - "demo_orc_with_cache_file_index", - "true", - "true", - "false", - id="demo_orc_with_cache_file_index", - ), - pytest.param( - "demo_orc_no_cache_stripe_index", - "false", - "true", - "true", - id="demo_orc_no_cache_stripe_index", - ), - pytest.param( - "demo_orc_with_cache_stripe_index", - "true", - "true", - "true", - id="demo_orc_with_cache_stripe_index", - ), - ], -) -def test_orc_minmax_index( - started_cluster, - table, - use_local_cache_for_remote_storage, - enable_orc_file_minmax_index, - enable_orc_stripe_minmax_index, -): - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - DROP TABLE IF EXISTS default.{table}; - CREATE TABLE default.{table} (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day) - SETTINGS enable_orc_file_minmax_index = {enable_orc_file_minmax_index}, enable_orc_stripe_minmax_index = {enable_orc_stripe_minmax_index}; - """.format( - table=table, - enable_orc_file_minmax_index=enable_orc_file_minmax_index, - enable_orc_stripe_minmax_index=enable_orc_stripe_minmax_index, - ) - ) - assert result.strip() == "" - - for i in range(2): - result = node.query( - """ - SELECT day, id, score FROM default.{table} where day >= '2021-11-05' and day <= '2021-11-16' and score >= 15 and score <= 30 order by day, id - SETTINGS use_local_cache_for_remote_storage = {use_local_cache_for_remote_storage} - """.format( - table=table, - use_local_cache_for_remote_storage=use_local_cache_for_remote_storage, - ) - ) - - assert ( - result - == """2021-11-05 abd 15 -2021-11-16 aaa 22 -""" - ) - - -@pytest.mark.parametrize( - "table,use_local_cache_for_remote_storage,enable_parquet_rowgroup_minmax_index", - [ - pytest.param( - "demo_parquet_no_cache_no_index", - "false", - "false", - id="demo_parquet_no_cache_no_index", - ), - pytest.param( - "demo_parquet_with_cache_no_index", - "true", - "false", - id="demo_parquet_with_cache_no_index", - ), - pytest.param( - "demo_parquet_no_cache_rowgroup_index", - "false", - "true", - id="demo_parquet_no_cache_rowgroup_index", - ), - pytest.param( - "demo_parquet_with_cache_rowgroup_index", - "true", - "true", - id="demo_parquet_with_cache_rowgroup_index", - ), - ], -) -def test_parquet_minmax_index( - started_cluster, - table, - use_local_cache_for_remote_storage, - enable_parquet_rowgroup_minmax_index, -): - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - DROP TABLE IF EXISTS default.{table}; - CREATE TABLE default.{table} (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - SETTINGS enable_parquet_rowgroup_minmax_index = {enable_parquet_rowgroup_minmax_index} - """.format( - table=table, - enable_parquet_rowgroup_minmax_index=enable_parquet_rowgroup_minmax_index, - ) - ) - assert result.strip() == "" - - for i in range(2): - result = node.query( - """ - SELECT day, id, score FROM default.{table} where day >= '2021-11-05' and day <= '2021-11-16' and score >= 15 and score <= 30 order by day, id - SETTINGS use_local_cache_for_remote_storage = {use_local_cache_for_remote_storage} - """.format( - table=table, - use_local_cache_for_remote_storage=use_local_cache_for_remote_storage, - ) - ) - - assert ( - result - == """2021-11-05 abd 15 -2021-11-16 aaa 22 -""" - ) - - -def test_hive_columns_prunning(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - SELECT count(*) FROM default.demo_parquet_parts where day = '2021-11-05' - """ - ) - expected_result = """4 -""" - logging.info("query result:{}".format(result)) - assert result == expected_result - - -def test_text_count(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - DROP TABLE IF EXISTS default.demo_orc; - CREATE TABLE default.demo_orc (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo_orc') PARTITION BY(day); - SELECT day, count(*) FROM default.demo_orc group by day order by day SETTINGS format_csv_delimiter = '\x01' - """ - ) - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" - assert result == expected_result - - -def test_parquet_groupby_with_cache(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - SELECT day, count(*) FROM default.demo_parquet group by day order by day - """ - ) - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" - assert result == expected_result - - -def test_parquet_groupby_by_hive_function(started_cluster): - logging.info("Start testing groupby ...") - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - SELECT day, count(*) FROM hive('thrift://hivetest:9083', 'test', 'demo', '`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)', 'day') group by day order by day - """ - ) - expected_result = """2021-11-01 1 -2021-11-05 2 -2021-11-11 1 -2021-11-16 2 -""" - assert result == expected_result - - -def test_cache_read_bytes(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - """ - ) - test_passed = False - for i in range(10): - result = node.query( - """ - SELECT * FROM default.demo_parquet_1 settings input_format_parquet_allow_missing_columns = true - """ - ) - node.query("system flush logs") - result = node.query( - "select sum(ProfileEvent_ExternalDataSourceLocalCacheReadBytes) from system.metric_log where ProfileEvent_ExternalDataSourceLocalCacheReadBytes > 0" - ) - if result.strip() == "0": - logging.info("ProfileEvent_ExternalDataSourceLocalCacheReadBytes == 0") - time.sleep(10) - continue - test_passed = True - break - assert test_passed - - -def test_cache_dir_use(started_cluster): - node = started_cluster.instances["h0_0_0"] - result0 = node.exec_in_container( - ["bash", "-c", "ls /tmp/clickhouse_local_cache | wc -l"] - ) - result1 = node.exec_in_container( - ["bash", "-c", "ls /tmp/clickhouse_local_cache1 | wc -l"] - ) - assert result0 != "0" and result1 != "0" - - -def test_hive_struct_type(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query( - """ - CREATE TABLE IF NOT EXISTS default.test_hive_types (`f_tinyint` Int8, `f_smallint` Int16, `f_int` Int32, `f_integer` Int32, `f_bigint` Int64, `f_float` Float32, `f_double` Float64, `f_decimal` Float64, `f_timestamp` DateTime, `f_date` Date, `f_string` String, `f_varchar` String, `f_char` String, `f_bool` Boolean, `f_array_int` Array(Int32), `f_array_string` Array(String), `f_array_float` Array(Float32), `f_map_int` Map(String, Int32), `f_map_string` Map(String, String), `f_map_float` Map(String, Float32), `f_struct` Tuple(a String, b Int32, c Float32, d Tuple(x Int32, y String)), `day` String) ENGINE = Hive('thrift://hivetest:9083', 'test', 'test_hive_types') PARTITION BY (day) - """ - ) - result = node.query( - """ - SELECT * FROM default.test_hive_types WHERE day = '2022-02-20' SETTINGS input_format_parquet_import_nested=1 - """ - ) - expected_result = """1 2 3 4 5 6.11 7.22 8 2022-02-20 14:47:04 2022-02-20 hello world hello world hello world true [1,2,3] ['hello world','hello world'] [1.1,1.2] {'a':100,'b':200,'c':300} {'a':'aa','b':'bb','c':'cc'} {'a':111.1,'b':222.2,'c':333.3} ('aaa',200,333.3,(10,'xyz')) 2022-02-20""" - assert result.strip() == expected_result - - result = node.query( - """ - SELECT day, f_struct.a, f_struct.d.x FROM default.test_hive_types WHERE day = '2022-02-20' SETTINGS input_format_parquet_import_nested=1 - """ - ) - expected_result = """2022-02-20 aaa 10""" - - -def test_table_alter_add(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query("DROP TABLE IF EXISTS default.demo_parquet_1") - result = node.query( - """ -CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - """ - ) - result = node.query( - """ -ALTER TABLE default.demo_parquet_1 ADD COLUMN id Nullable(String) FIRST - """ - ) - result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""") - - expected_result = "id\tNullable(String)\t\t\t\t\t\nscore\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)" - assert result.strip() == expected_result - - -def test_table_alter_drop(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query("DROP TABLE IF EXISTS default.demo_parquet_1") - result = node.query( - """ -CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - """ - ) - result = node.query( - """ -ALTER TABLE default.demo_parquet_1 DROP COLUMN id - """ - ) - - result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""") - expected_result = """score\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)""" - assert result.strip() == expected_result - - -def test_table_alter_comment(started_cluster): - node = started_cluster.instances["h0_0_0"] - result = node.query("DROP TABLE IF EXISTS default.demo_parquet_1") - result = node.query( - """ -CREATE TABLE IF NOT EXISTS default.demo_parquet_1 (`id` Nullable(String), `score` Nullable(Int32), `day` Nullable(String)) ENGINE = Hive('thrift://hivetest:9083', 'test', 'demo') PARTITION BY(day) - """ - ) - - result = node.query( - """ALTER TABLE default.demo_parquet_1 COMMENT COLUMN id 'Text comment'""" - ) - result = node.query("""DESC default.demo_parquet_1 FORMAT TSV""") - expected_result = """id\tNullable(String)\t\t\tText comment\t\t\nscore\tNullable(Int32)\t\t\t\t\t\nday\tNullable(String)""" - assert result.strip() == expected_result diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/__init__.py b/tests/integration/test_host_regexp_multiple_ptr_records/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml b/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml deleted file mode 100644 index 7a2141e6c7e..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - - - - test1\.example\.com$ - - default - - - \ No newline at end of file diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml b/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml deleted file mode 100644 index 58ef55cd3f3..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml +++ /dev/null @@ -1,5 +0,0 @@ - - :: - 0.0.0.0 - 1 - diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile deleted file mode 100644 index 3edf37dafa5..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile +++ /dev/null @@ -1,8 +0,0 @@ -. { - hosts /example.com { - reload "20ms" - fallthrough - } - forward . 127.0.0.11 - log -} diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com deleted file mode 100644 index 9beb415c290..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com +++ /dev/null @@ -1 +0,0 @@ -filled in runtime, but needs to exist in order to be volume mapped in docker \ No newline at end of file diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/test.py b/tests/integration/test_host_regexp_multiple_ptr_records/test.py deleted file mode 100644 index 36f76140a2c..00000000000 --- a/tests/integration/test_host_regexp_multiple_ptr_records/test.py +++ /dev/null @@ -1,115 +0,0 @@ -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/51471 - -pytestmark = pytest.mark.skip - -import socket -from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check -from time import sleep -import os - -DOCKER_COMPOSE_PATH = get_docker_compose_path() -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - -cluster = ClickHouseCluster(__file__) - -ch_server = cluster.add_instance( - "clickhouse-server", - with_coredns=True, - main_configs=["configs/listen_host.xml"], - user_configs=["configs/host_regexp.xml"], - ipv6_address="2001:3984:3989::1:1111", -) - -client = cluster.add_instance( - "clickhouse-client", - ipv6_address="2001:3984:3989::1:1112", -) - - -@pytest.fixture(scope="module") -def started_cluster(): - global cluster - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def check_ptr_record(ip, hostname): - try: - host, aliaslist, ipaddrlist = socket.gethostbyaddr(ip) - if hostname.lower() == host.lower(): - return True - except socket.herror: - pass - return False - - -def setup_dns_server(ip): - domains_string = "test3.example.com test2.example.com test1.example.com" - example_file_path = f'{ch_server.env_variables["COREDNS_CONFIG_DIR"]}/example.com' - run_and_check(f"echo '{ip} {domains_string}' > {example_file_path}", shell=True) - - # DNS server takes time to reload the configuration. - for try_num in range(10): - if all(check_ptr_record(ip, host) for host in domains_string.split()): - break - sleep(1) - - -def setup_ch_server(dns_server_ip): - ch_server.exec_in_container( - (["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"]) - ) - ch_server.exec_in_container( - (["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"]) - ) - ch_server.query("SYSTEM DROP DNS CACHE") - - -def build_endpoint_v4(ip): - return f"'http://{ip}:8123/?query=SELECT+1&user=test_dns'" - - -def build_endpoint_v6(ip): - return build_endpoint_v4(f"[{ip}]") - - -def test_host_regexp_multiple_ptr_v4_fails_with_wrong_resolution(started_cluster): - server_ip = cluster.get_instance_ip("clickhouse-server") - random_ip = "9.9.9.9" - dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) - - setup_dns_server(random_ip) - setup_ch_server(dns_server_ip) - - endpoint = build_endpoint_v4(server_ip) - - assert "1\n" != client.exec_in_container(["bash", "-c", f"curl {endpoint}"]) - - -def test_host_regexp_multiple_ptr_v4(started_cluster): - server_ip = cluster.get_instance_ip("clickhouse-server") - client_ip = cluster.get_instance_ip("clickhouse-client") - dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) - - setup_dns_server(client_ip) - setup_ch_server(dns_server_ip) - - endpoint = build_endpoint_v4(server_ip) - - assert "1\n" == client.exec_in_container(["bash", "-c", f"curl {endpoint}"]) - - -def test_host_regexp_multiple_ptr_v6(started_cluster): - setup_dns_server(client.ipv6_address) - setup_ch_server(cluster.get_instance_global_ipv6(cluster.coredns_host)) - - endpoint = build_endpoint_v6(ch_server.ipv6_address) - - assert "1\n" == client.exec_in_container(["bash", "-c", f"curl -6 {endpoint}"]) diff --git a/tests/integration/test_insert_into_distributed_through_materialized_view/test.py b/tests/integration/test_insert_into_distributed_through_materialized_view/test.py index a5f92002450..b1eb0df2d43 100644 --- a/tests/integration/test_insert_into_distributed_through_materialized_view/test.py +++ b/tests/integration/test_insert_into_distributed_through_materialized_view/test.py @@ -108,74 +108,6 @@ def test_reconnect(started_cluster): assert remote.query("SELECT count(*) FROM local1").strip() == "3" -@pytest.mark.skip(reason="Flapping test") -def test_inserts_batching(started_cluster): - instance = instance_test_inserts_batching - - with PartitionManager() as pm: - pm.partition_instances(instance, remote) - - instance.query("INSERT INTO local2_source(d, x) VALUES ('2000-01-01', 1)") - # Sleep a bit so that this INSERT forms a batch of its own. - time.sleep(0.2) - - instance.query("INSERT INTO local2_source(x, d) VALUES (2, '2000-01-01')") - - for i in range(3, 7): - instance.query( - "INSERT INTO local2_source(d, x) VALUES ('2000-01-01', {})".format(i) - ) - - for i in range(7, 9): - instance.query( - "INSERT INTO local2_source(x, d) VALUES ({}, '2000-01-01')".format(i) - ) - - instance.query("INSERT INTO local2_source(d, x) VALUES ('2000-01-01', 9)") - - # After ALTER the structure of the saved blocks will be different - instance.query("DROP TABLE local2_view") - instance.query("ALTER TABLE distributed ADD COLUMN s String") - - # Memory Engine doesn't support ALTER so we just DROP/CREATE everything - instance.query("DROP TABLE local2_source") - instance.query( - "CREATE TABLE local2_source (d Date, x UInt32, s String) ENGINE = Memory" - ) - instance.query( - "CREATE MATERIALIZED VIEW local2_view to distributed AS SELECT d,x,s FROM local2_source" - ) - - for i in range(10, 13): - instance.query( - "INSERT INTO local2_source(d, x) VALUES ('2000-01-01', {})".format(i) - ) - - time.sleep(1.0) - - result = remote.query( - "SELECT _part, groupArray(x) FROM local2 GROUP BY _part ORDER BY _part" - ) - - # Explanation: as merges are turned off on remote instance, active parts in local2 table correspond 1-to-1 - # to inserted blocks. - # Batches of max 3 rows are formed as min_insert_block_size_rows = 3. - # Blocks: - # 1. Failed batch that is retried with the same contents. - # 2. Full batch of inserts regardless of the order of columns thanks to the view. - # 3. Full batch of inserts regardless order of columns thanks to the view. - # 4. Full batch of inserts after ALTER (that have different block structure). - # 5. What was left to insert before ALTER. - expected = """\ -20000101_20000101_1_1_0 [1] -20000101_20000101_2_2_0 [2,3,4] -20000101_20000101_3_3_0 [5,6,7] -20000101_20000101_4_4_0 [10,11,12] -20000101_20000101_5_5_0 [8,9] -""" - assert TSV(result) == TSV(expected) - - def test_inserts_local(started_cluster): instance = instance_test_inserts_local_cluster instance.query("INSERT INTO local_source VALUES ('2000-01-01', 1)") diff --git a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py index 1ec44d8a002..b099d0513e1 100644 --- a/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py +++ b/tests/integration/test_keeper_reconfig_replace_leader_in_one_command/test.py @@ -41,81 +41,3 @@ def started_cluster(): def get_fake_zk(node): return ku.get_fake_zk(cluster, node) - - -@pytest.mark.skip(reason="test is flaky because changes are not properly waited for") -def test_reconfig_replace_leader_in_one_command(started_cluster): - """ - Remove leader from a cluster of 3 and add a new node to this cluster in a single command - """ - - zk1 = get_fake_zk(node1) - config = ku.get_config_str(zk1) - - assert len(config.split("\n")) == 3 - assert "node1" in config - assert "node2" in config - assert "node3" in config - assert "node4" not in config - - for i in range(100): - zk1.create(f"/test_four_{i}", b"somedata") - - zk2 = get_fake_zk(node2) - zk2.sync("/test_four_0") - ku.wait_configs_equal(config, zk2) - - zk3 = get_fake_zk(node3) - zk3.sync("/test_four_0") - ku.wait_configs_equal(config, zk3) - - for i in range(100): - assert zk2.exists(f"/test_four_{i}") is not None - assert zk3.exists(f"/test_four_{i}") is not None - - assert ku.is_leader(cluster, node1) - node4.start_clickhouse() - config, _ = zk2.reconfig( - joining="server.4=node4:9234", leaving="1", new_members=None - ) - config = config.decode("utf-8") - - print("After removing 1 and adding 4", config) - assert len(config.split("\n")) == 3 - assert "node1" not in config - assert "node2" in config - assert "node3" in config - assert "node4" in config - - ku.wait_until_connected(cluster, node4) - time.sleep(1) - - zk4 = get_fake_zk(node4) - zk4.sync("/test_four_0") - ku.wait_configs_equal(config, zk4) - - for i in range(100): - assert zk4.exists(f"test_four_{i}") is not None - zk4.create(f"/test_four_{100 + i}", b"somedata") - - with pytest.raises(Exception): - zk1.stop() - zk1.close() - zk1 = get_fake_zk(node1) - zk1.sync("/test_four_0") - - zk2.stop() - zk2.close() - zk2 = get_fake_zk(node2) - zk2.sync("/test_four_0") - ku.wait_configs_equal(config, zk2) - - zk3.stop() - zk3.close() - zk3 = get_fake_zk(node3) - zk3.sync("/test_four_0") - ku.wait_configs_equal(config, zk3) - - for i in range(200): - assert zk2.exists(f"test_four_{i}") is not None - assert zk3.exists(f"test_four_{i}") is not None diff --git a/tests/integration/test_mutations_with_merge_tree/test.py b/tests/integration/test_mutations_with_merge_tree/test.py index e7d7abc8c7c..0b4a750b38e 100644 --- a/tests/integration/test_mutations_with_merge_tree/test.py +++ b/tests/integration/test_mutations_with_merge_tree/test.py @@ -27,111 +27,6 @@ def started_cluster(): cluster.shutdown() -@pytest.mark.skip(reason="Skipping mutations in partition does not work") -def test_mutations_in_partition_background(started_cluster): - try: - numbers = 100 - - name = "test_mutations_in_partition" - instance_test_mutations.query( - f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a""" - ) - instance_test_mutations.query( - f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})""" - ) - - for i in range(0, numbers, 3): - instance_test_mutations.query( - f"""ALTER TABLE {name} DELETE IN PARTITION {i} WHERE a = {i}""" - ) - - for i in range(1, numbers, 3): - instance_test_mutations.query( - f"""ALTER TABLE {name} UPDATE b = 'changed' IN PARTITION {i} WHERE a = {i} """ - ) - - def count_and_changed(): - return instance_test_mutations.query( - f"SELECT count(), countIf(b == 'changed') FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() - - all_done = False - for wait_times_for_mutation in range( - 100 - ): # wait for replication 80 seconds max - time.sleep(0.8) - - if count_and_changed() == ["66,33"]: - all_done = True - break - - print( - instance_test_mutations.query( - f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason, parts_to_do_names FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames" - ) - ) - - assert (count_and_changed(), all_done) == (["66,33"], True) - assert instance_test_mutations.query( - f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() == ["67,67"] - - finally: - instance_test_mutations.query(f"""DROP TABLE {name}""") - - -@pytest.mark.skip(reason="Skipping mutations in partition does not work") -@pytest.mark.parametrize("sync", [("last",), ("all",)]) -def test_mutations_in_partition_sync(started_cluster, sync): - try: - numbers = 10 - - name = "test_mutations_in_partition_sync" - instance_test_mutations.query( - f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a""" - ) - instance_test_mutations.query( - f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})""" - ) - - for i in range(0, numbers, 3): - instance_test_mutations.query( - f"""ALTER TABLE {name} DELETE IN PARTITION {i} WHERE a = {i}""" - + (" SETTINGS mutations_sync = 1" if sync == "all" else "") - ) - - for reverse_index, i in reversed( - list(enumerate(reversed(range(1, numbers, 3)))) - ): - instance_test_mutations.query( - f"""ALTER TABLE {name} UPDATE b = 'changed' IN PARTITION {i} WHERE a = {i}""" - + ( - " SETTINGS mutations_sync = 1" - if not reverse_index or sync == "all" - else "" - ) - ) - - def count_and_changed(): - return instance_test_mutations.query( - f"SELECT count(), countIf(b == 'changed') FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() - - print( - instance_test_mutations.query( - f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames" - ) - ) - - assert count_and_changed() == ["6,3"] - assert instance_test_mutations.query( - f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() == ["7,7"] - - finally: - instance_test_mutations.query(f"""DROP TABLE {name}""") - - def test_mutations_with_merge_background_task(started_cluster): instance_test_mutations.query( """SYSTEM STOP MERGES test_mutations_with_ast_elements""" @@ -190,92 +85,3 @@ def test_mutations_with_truncate_table(started_cluster): ).rstrip() == "0" ) - - -@pytest.mark.skip(reason="Skipping mutations in partition does not work") -def test_mutations_will_not_hang_for_non_existing_parts_sync(started_cluster): - try: - numbers = 100 - - name = "test_mutations_will_not_hang_for_non_existing_parts_sync" - instance_test_mutations.query( - f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a""" - ) - instance_test_mutations.query( - f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})""" - ) - - for i in range(0, numbers, 3): - instance_test_mutations.query( - f"""ALTER TABLE {name} DELETE IN PARTITION {i+1000} WHERE a = {i} SETTINGS mutations_sync = 1""" - ) - - def count(): - return instance_test_mutations.query( - f"SELECT count() FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() - - print( - instance_test_mutations.query( - f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason, parts_to_do_names FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames" - ) - ) - - assert count() == [f"{numbers}"] - assert instance_test_mutations.query( - f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() == [f"34,34"] - - finally: - instance_test_mutations.query(f"""DROP TABLE {name}""") - - -@pytest.mark.skip(reason="Skipping mutations in partition does not work") -def test_mutations_will_not_hang_for_non_existing_parts_async(started_cluster): - try: - numbers = 100 - - name = "test_mutations_will_not_hang_for_non_existing_parts_async" - instance_test_mutations.query( - f"""CREATE TABLE {name} (date Date, a UInt64, b String) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY a""" - ) - instance_test_mutations.query( - f"""INSERT INTO {name} SELECT '2019-07-29' AS date, number, toString(number) FROM numbers({numbers})""" - ) - - for i in range(0, numbers, 3): - instance_test_mutations.query( - f"""ALTER TABLE {name} DELETE IN PARTITION {i+1000} WHERE a = {i}""" - ) - - def count(): - return instance_test_mutations.query( - f"SELECT count() FROM {name} SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() - - def count_and_sum_is_done(): - return instance_test_mutations.query( - f"SELECT count(), sum(is_done) FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT CSV" - ).splitlines() - - all_done = False - for wait_times_for_mutation in range( - 100 - ): # wait for replication 80 seconds max - time.sleep(0.8) - - if count_and_sum_is_done() == ["34,34"]: - all_done = True - break - - print( - instance_test_mutations.query( - f"SELECT mutation_id, command, parts_to_do, is_done, latest_failed_part, latest_fail_reason, parts_to_do_names FROM system.mutations WHERE table = '{name}' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames" - ) - ) - - assert count() == [f"{numbers}"] - assert count_and_sum_is_done() == ["34,34"] - - finally: - instance_test_mutations.query(f"""DROP TABLE {name}""") diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index 48bcca7d505..debb509de90 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -414,54 +414,6 @@ def start_cluster_diff_versions(): cluster.shutdown() -@pytest.mark.skip(reason="compatability is temporary broken") -def test_polymorphic_parts_diff_versions(start_cluster_diff_versions): - # Check that replication with Wide parts works between different versions. - - node_old = node7 - node_new = node8 - - insert_random_data("polymorphic_table", node7, 100) - node8.query("SYSTEM SYNC REPLICA polymorphic_table", timeout=20) - - assert node8.query("SELECT count() FROM polymorphic_table") == "100\n" - assert ( - node8.query( - "SELECT DISTINCT part_type FROM system.parts WHERE table = 'polymorphic_table' and active" - ) - == "Wide\n" - ) - - -@pytest.mark.skip(reason="compatability is temporary broken") -def test_polymorphic_parts_diff_versions_2(start_cluster_diff_versions): - # Replication doesn't work on old version if part is created in compact format, because - # this version doesn't know anything about it. It's considered to be ok. - - node_old = node7 - node_new = node8 - - insert_random_data("polymorphic_table_2", node_new, 100) - - assert node_new.query("SELECT count() FROM polymorphic_table_2") == "100\n" - assert node_old.query("SELECT count() FROM polymorphic_table_2") == "0\n" - with pytest.raises(Exception): - node_old.query("SYSTEM SYNC REPLICA polymorphic_table_2", timeout=3) - - node_old.restart_with_latest_version(fix_metadata=True) - - node_old.query("SYSTEM SYNC REPLICA polymorphic_table_2", timeout=20) - - # Works after update - assert node_old.query("SELECT count() FROM polymorphic_table_2") == "100\n" - assert ( - node_old.query( - "SELECT DISTINCT part_type FROM system.parts WHERE table = 'polymorphic_table_2' and active" - ) - == "Compact\n" - ) - - def test_polymorphic_parts_non_adaptive(start_cluster): node1.query("SYSTEM STOP MERGES") node2.query("SYSTEM STOP MERGES") diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/__init__.py b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml deleted file mode 100644 index bb4aba94e0b..00000000000 --- a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/configs/config.d/storage_conf.xml +++ /dev/null @@ -1,93 +0,0 @@ - - - - - hdfs - hdfs://hdfs1:9000/clickhouse1/ - - true - - - hdfs - hdfs://hdfs1:9000/clickhouse1/ - - true - - - hdfs - hdfs://hdfs1:9000/clickhouse2/ - - true - - - - - -
- hdfs1 -
-
-
- - -
- default -
- - hdfs1 - -
- 0.0 -
- - -
- hdfs2 -
- - hdfs1 - -
-
- - -
- hdfs1_again -
- - hdfs1 - -
-
-
-
- - - 1024000 - 1 - true - - - - - - - node1 - 9000 - - - - - node2 - 9000 - - - - - - - test_cluster - 1 - - true -
diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py deleted file mode 100644 index eb3d62eb718..00000000000 --- a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py +++ /dev/null @@ -1,322 +0,0 @@ -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/42561 - -pytestmark = pytest.mark.skip - -import logging -from string import Template -import time - -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry - -from pyhdfs import HdfsClient - -SHARDS = 2 -FILES_OVERHEAD_PER_TABLE = 1 # format_version.txt -FILES_OVERHEAD_PER_PART_COMPACT = 7 - - -def wait_for_hdfs_objects(cluster, fp, expected, num_tries=30): - fs = HdfsClient(hosts=cluster.hdfs_ip) - while num_tries > 0: - num_hdfs_objects = len(fs.listdir(fp)) - if num_hdfs_objects == expected: - break - num_tries -= 1 - time.sleep(1) - assert len(fs.listdir(fp)) == expected - - -@pytest.fixture(scope="module") -def cluster(): - try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "node1", - main_configs=["configs/config.d/storage_conf.xml"], - macros={"replica": "node1"}, - with_zookeeper=True, - with_hdfs=True, - ) - cluster.add_instance( - "node2", - main_configs=["configs/config.d/storage_conf.xml"], - macros={"replica": "node2"}, - with_zookeeper=True, - with_hdfs=True, - ) - logging.info("Starting cluster...") - cluster.start() - if cluster.instances["node1"].is_debug_build(): - # https://github.com/ClickHouse/ClickHouse/issues/27814 - pytest.skip( - "libhdfs3 calls rand function which does not pass harmful check in debug build" - ) - logging.info("Cluster started") - - fs = HdfsClient(hosts=cluster.hdfs_ip) - fs.mkdirs("/clickhouse1") - fs.mkdirs("/clickhouse2") - logging.info("Created HDFS directory") - - yield cluster - finally: - cluster.shutdown() - - -def test_hdfs_zero_copy_replication_insert(cluster): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - try: - node1.query( - """ - CREATE TABLE hdfs_test ON CLUSTER test_cluster (dt DateTime, id Int64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/hdfs_test', '{replica}') - ORDER BY (dt, id) - SETTINGS storage_policy='hdfs_only' - """ - ) - wait_for_hdfs_objects( - cluster, "/clickhouse1", SHARDS * FILES_OVERHEAD_PER_TABLE - ) - - node1.query("INSERT INTO hdfs_test VALUES (now() - INTERVAL 3 DAY, 10)") - node2.query("SYSTEM SYNC REPLICA hdfs_test", timeout=30) - assert node1.query("SELECT count() FROM hdfs_test FORMAT Values") == "(1)" - assert node2.query("SELECT count() FROM hdfs_test FORMAT Values") == "(1)" - assert ( - node1.query("SELECT id FROM hdfs_test ORDER BY dt FORMAT Values") == "(10)" - ) - assert ( - node2.query("SELECT id FROM hdfs_test ORDER BY dt FORMAT Values") == "(10)" - ) - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hdfs_test' FORMAT Values" - ) - == "('all','hdfs1')" - ) - assert ( - node2.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hdfs_test' FORMAT Values" - ) - == "('all','hdfs1')" - ) - wait_for_hdfs_objects( - cluster, - "/clickhouse1", - SHARDS * FILES_OVERHEAD_PER_TABLE + FILES_OVERHEAD_PER_PART_COMPACT, - ) - finally: - node1.query("DROP TABLE IF EXISTS hdfs_test SYNC") - node2.query("DROP TABLE IF EXISTS hdfs_test SYNC") - - -@pytest.mark.parametrize( - ("storage_policy", "init_objects"), - [("hybrid", 0), ("tiered", 0), ("tiered_copy", FILES_OVERHEAD_PER_TABLE)], -) -def test_hdfs_zero_copy_replication_single_move(cluster, storage_policy, init_objects): - node1 = cluster.instances["node1"] - try: - node1.query( - Template( - """ - CREATE TABLE single_node_move_test (dt DateTime, id Int64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/single_node_move_test', '{replica}') - ORDER BY (dt, id) - SETTINGS storage_policy='$policy',temporary_directories_lifetime=1 - """ - ).substitute(policy=storage_policy) - ) - wait_for_hdfs_objects(cluster, "/clickhouse1", init_objects) - - node1.query( - "INSERT INTO single_node_move_test VALUES (now() - INTERVAL 3 DAY, 10), (now() - INTERVAL 1 DAY, 11)" - ) - assert ( - node1.query( - "SELECT id FROM single_node_move_test ORDER BY dt FORMAT Values" - ) - == "(10),(11)" - ) - - node1.query( - "ALTER TABLE single_node_move_test MOVE PARTITION ID 'all' TO VOLUME 'external'" - ) - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='single_node_move_test' FORMAT Values" - ) - == "('all','hdfs1')" - ) - assert ( - node1.query( - "SELECT id FROM single_node_move_test ORDER BY dt FORMAT Values" - ) - == "(10),(11)" - ) - wait_for_hdfs_objects( - cluster, "/clickhouse1", init_objects + FILES_OVERHEAD_PER_PART_COMPACT - ) - - node1.query( - "ALTER TABLE single_node_move_test MOVE PARTITION ID 'all' TO VOLUME 'main'" - ) - assert ( - node1.query( - "SELECT id FROM single_node_move_test ORDER BY dt FORMAT Values" - ) - == "(10),(11)" - ) - finally: - node1.query("DROP TABLE IF EXISTS single_node_move_test SYNC") - - -@pytest.mark.parametrize( - ("storage_policy", "init_objects"), - [("hybrid", 0), ("tiered", 0), ("tiered_copy", SHARDS * FILES_OVERHEAD_PER_TABLE)], -) -def test_hdfs_zero_copy_replication_move(cluster, storage_policy, init_objects): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - try: - node1.query( - Template( - """ - CREATE TABLE move_test ON CLUSTER test_cluster (dt DateTime, id Int64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/move_test', '{replica}') - ORDER BY (dt, id) - SETTINGS storage_policy='$policy' - """ - ).substitute(policy=storage_policy) - ) - wait_for_hdfs_objects(cluster, "/clickhouse1", init_objects) - - node1.query( - "INSERT INTO move_test VALUES (now() - INTERVAL 3 DAY, 10), (now() - INTERVAL 1 DAY, 11)" - ) - node2.query("SYSTEM SYNC REPLICA move_test", timeout=30) - - assert ( - node1.query("SELECT id FROM move_test ORDER BY dt FORMAT Values") - == "(10),(11)" - ) - assert ( - node2.query("SELECT id FROM move_test ORDER BY dt FORMAT Values") - == "(10),(11)" - ) - - node1.query( - "ALTER TABLE move_test MOVE PARTITION ID 'all' TO VOLUME 'external'" - ) - wait_for_hdfs_objects( - cluster, "/clickhouse1", init_objects + FILES_OVERHEAD_PER_PART_COMPACT - ) - - node2.query( - "ALTER TABLE move_test MOVE PARTITION ID 'all' TO VOLUME 'external'" - ) - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='move_test' FORMAT Values" - ) - == "('all','hdfs1')" - ) - assert ( - node2.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='move_test' FORMAT Values" - ) - == "('all','hdfs1')" - ) - assert ( - node1.query("SELECT id FROM move_test ORDER BY dt FORMAT Values") - == "(10),(11)" - ) - assert ( - node2.query("SELECT id FROM move_test ORDER BY dt FORMAT Values") - == "(10),(11)" - ) - wait_for_hdfs_objects( - cluster, "/clickhouse1", init_objects + FILES_OVERHEAD_PER_PART_COMPACT - ) - finally: - node1.query("DROP TABLE IF EXISTS move_test SYNC") - node2.query("DROP TABLE IF EXISTS move_test SYNC") - - -@pytest.mark.parametrize(("storage_policy"), ["hybrid", "tiered", "tiered_copy"]) -def test_hdfs_zero_copy_with_ttl_move(cluster, storage_policy): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - try: - node1.query( - Template( - """ - CREATE TABLE ttl_move_test ON CLUSTER test_cluster (dt DateTime, id Int64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/ttl_move_test', '{replica}') - ORDER BY (dt, id) - TTL dt + INTERVAL 2 DAY TO VOLUME 'external' - SETTINGS storage_policy='$policy' - """ - ).substitute(policy=storage_policy) - ) - - node1.query("INSERT INTO ttl_move_test VALUES (now() - INTERVAL 3 DAY, 10)") - node1.query("INSERT INTO ttl_move_test VALUES (now() - INTERVAL 1 DAY, 11)") - - node1.query("OPTIMIZE TABLE ttl_move_test FINAL") - node2.query("SYSTEM SYNC REPLICA ttl_move_test", timeout=30) - - assert_eq_with_retry(node1, "SELECT count() FROM ttl_move_test", "2") - assert_eq_with_retry(node2, "SELECT count() FROM ttl_move_test", "2") - assert ( - node1.query("SELECT id FROM ttl_move_test ORDER BY id FORMAT Values") - == "(10),(11)" - ) - assert ( - node2.query("SELECT id FROM ttl_move_test ORDER BY id FORMAT Values") - == "(10),(11)" - ) - finally: - node1.query("DROP TABLE IF EXISTS ttl_move_test SYNC") - node2.query("DROP TABLE IF EXISTS ttl_move_test SYNC") - - -def test_hdfs_zero_copy_with_ttl_delete(cluster): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - try: - node1.query( - """ - CREATE TABLE ttl_delete_test ON CLUSTER test_cluster (dt DateTime, id Int64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/ttl_delete_test', '{replica}') - ORDER BY (dt, id) - TTL dt + INTERVAL 2 DAY - SETTINGS storage_policy='tiered' - """ - ) - - node1.query("INSERT INTO ttl_delete_test VALUES (now() - INTERVAL 3 DAY, 10)") - node1.query("INSERT INTO ttl_delete_test VALUES (now() - INTERVAL 1 DAY, 11)") - - node1.query("OPTIMIZE TABLE ttl_delete_test FINAL") - node2.query("SYSTEM SYNC REPLICA ttl_delete_test", timeout=30) - - assert_eq_with_retry(node1, "SELECT count() FROM ttl_delete_test", "1") - assert_eq_with_retry(node2, "SELECT count() FROM ttl_delete_test", "1") - - assert ( - node1.query("SELECT id FROM ttl_delete_test ORDER BY id FORMAT Values") - == "(11)" - ) - assert ( - node2.query("SELECT id FROM ttl_delete_test ORDER BY id FORMAT Values") - == "(11)" - ) - finally: - node1.query("DROP TABLE IF EXISTS ttl_delete_test SYNC") - node2.query("DROP TABLE IF EXISTS ttl_delete_test SYNC") diff --git a/tests/integration/test_reverse_dns_query/__init__.py b/tests/integration/test_reverse_dns_query/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_reverse_dns_query/configs/config.xml b/tests/integration/test_reverse_dns_query/configs/config.xml deleted file mode 100644 index 5ce55afa2a7..00000000000 --- a/tests/integration/test_reverse_dns_query/configs/config.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 1 - diff --git a/tests/integration/test_reverse_dns_query/configs/listen_host.xml b/tests/integration/test_reverse_dns_query/configs/listen_host.xml deleted file mode 100644 index 58ef55cd3f3..00000000000 --- a/tests/integration/test_reverse_dns_query/configs/listen_host.xml +++ /dev/null @@ -1,5 +0,0 @@ - - :: - 0.0.0.0 - 1 - diff --git a/tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml b/tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml deleted file mode 100644 index 35d0a07c6a6..00000000000 --- a/tests/integration/test_reverse_dns_query/configs/reverse_dns_function.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 1 - diff --git a/tests/integration/test_reverse_dns_query/coredns_config/Corefile b/tests/integration/test_reverse_dns_query/coredns_config/Corefile deleted file mode 100644 index 84d297f7cdf..00000000000 --- a/tests/integration/test_reverse_dns_query/coredns_config/Corefile +++ /dev/null @@ -1,4 +0,0 @@ -. { - forward . 127.0.0.11 - log -} diff --git a/tests/integration/test_reverse_dns_query/test.py b/tests/integration/test_reverse_dns_query/test.py deleted file mode 100644 index 6a39bd7b586..00000000000 --- a/tests/integration/test_reverse_dns_query/test.py +++ /dev/null @@ -1,56 +0,0 @@ -import pytest - -# FIXME This test is broken -# https://github.com/ClickHouse/ClickHouse/issues/53194 - -pytestmark = pytest.mark.skip - -from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check -from time import sleep -import os - -DOCKER_COMPOSE_PATH = get_docker_compose_path() -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - -cluster = ClickHouseCluster(__file__) - -ch_server = cluster.add_instance( - "clickhouse-server", - with_coredns=True, - main_configs=[ - "configs/config.xml", - "configs/reverse_dns_function.xml", - "configs/listen_host.xml", - ], -) - - -@pytest.fixture(scope="module") -def started_cluster(): - global cluster - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def setup_ch_server(dns_server_ip): - ch_server.exec_in_container( - (["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"]) - ) - ch_server.exec_in_container( - (["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"]) - ) - ch_server.query("SYSTEM DROP DNS CACHE") - - -def test_reverse_dns_query(started_cluster): - dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) - - setup_ch_server(dns_server_ip) - - for _ in range(0, 200): - response = ch_server.query("select reverseDNSQuery('2001:4860:4860::8888')") - assert response == "['dns.google']\n" diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index 2a4e0eece08..0ca81a27802 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -153,96 +153,6 @@ def test_s3_zero_copy_replication(started_cluster, policy): node2.query("DROP TABLE IF EXISTS s3_test SYNC") -@pytest.mark.skip(reason="Test is flaky (and never was stable)") -def test_s3_zero_copy_on_hybrid_storage(started_cluster): - node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] - - node1.query( - """ - CREATE TABLE hybrid_test ON CLUSTER test_cluster (id UInt32, value String) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/hybrid_test', '{}') - ORDER BY id - SETTINGS storage_policy='hybrid',temporary_directories_lifetime=1 - """.format( - "{replica}" - ) - ) - - node1.query("INSERT INTO hybrid_test VALUES (0,'data'),(1,'data')") - node2.query("SYSTEM SYNC REPLICA hybrid_test", timeout=30) - - assert ( - node1.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") - == "(0,'data'),(1,'data')" - ) - assert ( - node2.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") - == "(0,'data'),(1,'data')" - ) - - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','default')" - ) - assert ( - node2.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','default')" - ) - - node1.query("ALTER TABLE hybrid_test MOVE PARTITION ID 'all' TO DISK 's31'") - - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','s31')" - ) - assert ( - node2.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','default')" - ) - - # Total objects in S3 - s3_objects = get_large_objects_count(cluster, size=0) - - node2.query("ALTER TABLE hybrid_test MOVE PARTITION ID 'all' TO DISK 's31'") - - assert ( - node1.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','s31')" - ) - assert ( - node2.query( - "SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values" - ) - == "('all','s31')" - ) - - # Check that after moving partition on node2 no new obects on s3 - wait_for_large_objects_count(cluster, s3_objects, size=0) - - assert ( - node1.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") - == "(0,'data'),(1,'data')" - ) - assert ( - node2.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") - == "(0,'data'),(1,'data')" - ) - - node1.query("DROP TABLE IF EXISTS hybrid_test SYNC") - node2.query("DROP TABLE IF EXISTS hybrid_test SYNC") - - def insert_data_time(node, table, number_of_mb, time, start=0): values = ",".join( f"({x},{time})" diff --git a/tests/integration/test_s3_zero_copy_ttl/__init__.py b/tests/integration/test_s3_zero_copy_ttl/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml b/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml deleted file mode 100644 index 54f7152690b..00000000000 --- a/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - - - - 10 - - - diff --git a/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml b/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml deleted file mode 100644 index 7bb7fa875e4..00000000000 --- a/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml +++ /dev/null @@ -1,39 +0,0 @@ - - - - - s3 - http://minio1:9001/root/data/ - minio - minio123 - - - - - - -
- default -
- - s3_disk - -
-
- - -
- s3_disk -
-
-
-
-
- - - true - 1.0 - - - true -
diff --git a/tests/integration/test_s3_zero_copy_ttl/test.py b/tests/integration/test_s3_zero_copy_ttl/test.py deleted file mode 100644 index 04bff4a44fb..00000000000 --- a/tests/integration/test_s3_zero_copy_ttl/test.py +++ /dev/null @@ -1,94 +0,0 @@ -#!/usr/bin/env python3 -import time - -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance( - "node1", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True -) -node2 = cluster.add_instance( - "node2", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True -) -node3 = cluster.add_instance( - "node3", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True -) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - yield cluster - finally: - cluster.shutdown() - - -def test_ttl_move_and_s3(started_cluster): - for i, node in enumerate([node1, node2, node3]): - node.query( - """ - CREATE TABLE s3_test_with_ttl (date DateTime, id UInt32, value String) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/s3_test', '{}') - ORDER BY id - PARTITION BY id - TTL date TO DISK 's3_disk' - SETTINGS storage_policy='s3_and_default', temporary_directories_lifetime=1 - """.format( - i - ) - ) - - node1.query("SYSTEM STOP MOVES s3_test_with_ttl") - - node2.query("SYSTEM STOP MOVES s3_test_with_ttl") - - for i in range(30): - if i % 2 == 0: - node = node1 - else: - node = node2 - - node.query( - f"INSERT INTO s3_test_with_ttl SELECT now() + 5, {i}, randomPrintableASCII(1048570)" - ) - - node1.query("SYSTEM SYNC REPLICA s3_test_with_ttl") - node2.query("SYSTEM SYNC REPLICA s3_test_with_ttl") - node3.query("SYSTEM SYNC REPLICA s3_test_with_ttl") - - assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" - assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" - - node1.query("SYSTEM START MOVES s3_test_with_ttl") - node2.query("SYSTEM START MOVES s3_test_with_ttl") - - assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" - assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" - - for attempt in reversed(range(5)): - time.sleep(5) - - print( - node1.query( - "SELECT * FROM system.parts WHERE table = 's3_test_with_ttl' FORMAT Vertical" - ) - ) - - minio = cluster.minio_client - objects = minio.list_objects(cluster.minio_bucket, "data/", recursive=True) - counter = 0 - for obj in objects: - print(f"Objectname: {obj.object_name}, metadata: {obj.metadata}") - counter += 1 - - print(f"Total objects: {counter}") - - if counter == 330: - break - - print(f"Attempts remaining: {attempt}") - - assert counter == 330 diff --git a/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py b/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py deleted file mode 100644 index 5fbe426074f..00000000000 --- a/tests/integration/test_s3_zero_copy_ttl/test_ttl_move_memory_usage.py +++ /dev/null @@ -1,81 +0,0 @@ -#!/usr/bin/env python3 -import time - -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/45887 - -pytestmark = pytest.mark.skip - -from helpers.cluster import ClickHouseCluster - - -single_node_cluster = ClickHouseCluster(__file__) -small_node = single_node_cluster.add_instance( - "small_node", main_configs=["configs/s3.xml"], with_minio=True -) - - -@pytest.fixture(scope="module") -def started_single_node_cluster(): - try: - single_node_cluster.start() - - yield single_node_cluster - finally: - single_node_cluster.shutdown() - - -def test_move_and_s3_memory_usage(started_single_node_cluster): - if small_node.is_built_with_sanitizer() or small_node.is_debug_build(): - pytest.skip("Disabled for debug and sanitizers. Too slow.") - - small_node.query( - "CREATE TABLE s3_test_with_ttl (x UInt32, a String codec(NONE), b String codec(NONE), c String codec(NONE), d String codec(NONE), e String codec(NONE)) engine = MergeTree order by x partition by x SETTINGS storage_policy='s3_and_default'" - ) - - for _ in range(10): - small_node.query( - "insert into s3_test_with_ttl select 0, repeat('a', 100), repeat('b', 100), repeat('c', 100), repeat('d', 100), repeat('e', 100) from zeros(400000) settings max_block_size = 8192, max_insert_block_size=10000000, min_insert_block_size_rows=10000000" - ) - - # After this, we should have 5 columns per 10 * 100 * 400000 ~ 400 MB; total ~2G data in partition - small_node.query( - "optimize table s3_test_with_ttl final", - settings={ - "send_logs_level": "error", - "allow_prefetched_read_pool_for_remote_filesystem": 0, - }, - ) - - small_node.query("system flush logs") - # Will take memory usage from metric_log. - # It is easier then specifying total memory limit (insert queries can hit this limit). - small_node.query("truncate table system.metric_log") - - small_node.query( - "alter table s3_test_with_ttl move partition 0 to volume 'external'", - settings={ - "send_logs_level": "error", - "allow_prefetched_read_pool_for_remote_filesystem": 0, - }, - ) - small_node.query("system flush logs") - max_usage = small_node.query( - """ - select max(m.val - am.val * 4096) from - (select toStartOfMinute(event_time) as time, max(CurrentMetric_MemoryTracking) as val from system.metric_log group by time) as m join - (select toStartOfMinute(event_time) as time, min(value) as val from system.asynchronous_metric_log where metric='jemalloc.arenas.all.pdirty' group by time) as am using time;""" - ) - # 3G limit is a big one. However, we can hit it anyway with parallel s3 writes enabled. - # Also actual value can be bigger because of memory drift. - # Increase it a little bit if test fails. - assert int(max_usage) < 3e9 - res = small_node.query( - "select * from system.errors where last_error_message like '%Memory limit%' limit 1", - settings={ - "allow_prefetched_read_pool_for_remote_filesystem": 0, - }, - ) - assert res == "" diff --git a/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py b/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py deleted file mode 100644 index fb9f3eb67b9..00000000000 --- a/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py +++ /dev/null @@ -1,46 +0,0 @@ -#!/usr/bin/env python3 -import time - -import pytest -from helpers.cluster import ClickHouseCluster - - -single_node_cluster = ClickHouseCluster(__file__) -small_node = single_node_cluster.add_instance( - "small_node", - main_configs=["configs/s3.xml"], - user_configs=["configs/max_delayed_streams.xml"], - with_minio=True, -) - - -@pytest.fixture(scope="module") -def started_single_node_cluster(): - try: - single_node_cluster.start() - - yield single_node_cluster - finally: - single_node_cluster.shutdown() - - -def test_vertical_merge_memory_usage(started_single_node_cluster): - if small_node.is_built_with_sanitizer() or small_node.is_debug_build(): - pytest.skip("Disabled for debug and sanitizers. Too slow.") - - small_node.query( - "create table tvm2 (c0 UInt64, c1 UInt64, c2 UInt64, c3 UInt64, c4 UInt64, c5 UInt64, c6 UInt64, c7 UInt64, c8 UInt64, c9 UInt64, c10 UInt64, c11 UInt64, c12 UInt64, c13 UInt64, c14 UInt64, c15 UInt64, c16 UInt64, c17 UInt64, c18 UInt64, c19 UInt64, c20 UInt64, c21 UInt64, c22 UInt64, c23 UInt64, c24 UInt64, c25 UInt64, c26 UInt64, c27 UInt64, c28 UInt64, c29 UInt64, c30 UInt64, c31 UInt64, c32 UInt64, c33 UInt64, c34 UInt64, c35 UInt64, c36 UInt64, c37 UInt64, c38 UInt64, c39 UInt64, c40 UInt64, c41 UInt64, c42 UInt64, c43 UInt64, c44 UInt64, c45 UInt64, c46 UInt64, c47 UInt64, c48 UInt64, c49 UInt64, c50 UInt64, c51 UInt64, c52 UInt64, c53 UInt64, c54 UInt64, c55 UInt64, c56 UInt64, c57 UInt64, c58 UInt64, c59 UInt64, c60 UInt64, c61 UInt64, c62 UInt64, c63 UInt64, c64 UInt64, c65 UInt64, c66 UInt64, c67 UInt64, c68 UInt64, c69 UInt64, c70 UInt64, c71 UInt64, c72 UInt64, c73 UInt64, c74 UInt64, c75 UInt64, c76 UInt64, c77 UInt64, c78 UInt64, c79 UInt64, c80 UInt64, c81 UInt64, c82 UInt64, c83 UInt64, c84 UInt64, c85 UInt64, c86 UInt64, c87 UInt64, c88 UInt64, c89 UInt64, c90 UInt64, c91 UInt64, c92 UInt64, c93 UInt64, c94 UInt64, c95 UInt64, c96 UInt64, c97 UInt64, c98 UInt64, c99 UInt64, c100 UInt64, c101 UInt64, c102 UInt64, c103 UInt64, c104 UInt64, c105 UInt64, c106 UInt64, c107 UInt64, c108 UInt64, c109 UInt64, c110 UInt64, c111 UInt64, c112 UInt64, c113 UInt64, c114 UInt64, c115 UInt64, c116 UInt64, c117 UInt64, c118 UInt64, c119 UInt64, c120 UInt64, c121 UInt64, c122 UInt64, c123 UInt64, c124 UInt64, c125 UInt64, c126 UInt64, c127 UInt64, c128 UInt64, c129 UInt64, c130 UInt64, c131 UInt64, c132 UInt64, c133 UInt64, c134 UInt64, c135 UInt64, c136 UInt64, c137 UInt64, c138 UInt64, c139 UInt64, c140 UInt64, c141 UInt64, c142 UInt64, c143 UInt64, c144 UInt64, c145 UInt64, c146 UInt64, c147 UInt64, c148 UInt64, c149 UInt64, c150 UInt64, c151 UInt64, c152 UInt64, c153 UInt64, c154 UInt64, c155 UInt64, c156 UInt64, c157 UInt64, c158 UInt64, c159 UInt64, c160 UInt64, c161 UInt64, c162 UInt64, c163 UInt64, c164 UInt64, c165 UInt64, c166 UInt64, c167 UInt64, c168 UInt64, c169 UInt64, c170 UInt64, c171 UInt64, c172 UInt64, c173 UInt64, c174 UInt64, c175 UInt64, c176 UInt64, c177 UInt64, c178 UInt64, c179 UInt64, c180 UInt64, c181 UInt64, c182 UInt64, c183 UInt64, c184 UInt64, c185 UInt64, c186 UInt64, c187 UInt64, c188 UInt64, c189 UInt64, c190 UInt64, c191 UInt64, c192 UInt64, c193 UInt64, c194 UInt64, c195 UInt64, c196 UInt64, c197 UInt64, c198 UInt64, c199 UInt64, c200 UInt64, c201 UInt64, c202 UInt64, c203 UInt64, c204 UInt64, c205 UInt64, c206 UInt64, c207 UInt64, c208 UInt64, c209 UInt64, c210 UInt64, c211 UInt64, c212 UInt64, c213 UInt64, c214 UInt64, c215 UInt64, c216 UInt64, c217 UInt64, c218 UInt64, c219 UInt64, c220 UInt64, c221 UInt64, c222 UInt64, c223 UInt64, c224 UInt64, c225 UInt64, c226 UInt64, c227 UInt64, c228 UInt64, c229 UInt64, c230 UInt64, c231 UInt64, c232 UInt64, c233 UInt64, c234 UInt64, c235 UInt64, c236 UInt64, c237 UInt64, c238 UInt64, c239 UInt64, c240 UInt64, c241 UInt64, c242 UInt64, c243 UInt64, c244 UInt64, c245 UInt64, c246 UInt64, c247 UInt64, c248 UInt64, c249 UInt64, c250 UInt64, c251 UInt64, c252 UInt64, c253 UInt64, c254 UInt64, c255 UInt64, c256 UInt64, c257 UInt64, c258 UInt64, c259 UInt64, c260 UInt64, c261 UInt64, c262 UInt64, c263 UInt64, c264 UInt64, c265 UInt64, c266 UInt64, c267 UInt64, c268 UInt64, c269 UInt64, c270 UInt64, c271 UInt64, c272 UInt64, c273 UInt64, c274 UInt64, c275 UInt64, c276 UInt64, c277 UInt64, c278 UInt64, c279 UInt64, c280 UInt64, c281 UInt64, c282 UInt64, c283 UInt64, c284 UInt64, c285 UInt64, c286 UInt64, c287 UInt64, c288 UInt64, c289 UInt64, c290 UInt64, c291 UInt64, c292 UInt64, c293 UInt64, c294 UInt64, c295 UInt64, c296 UInt64, c297 UInt64, c298 UInt64, c299 UInt64) engine = MergeTree order by tuple() settings min_rows_for_wide_part = 10, min_bytes_for_wide_part=0, storage_policy = 's3', vertical_merge_algorithm_min_rows_to_activate=1" - ) - - small_node.query( - "insert into tvm2 select number + 0, number + 1, number + 2, number + 3, number + 4, number + 5, number + 6, number + 7, number + 8, number + 9, number + 10, number + 11, number + 12, number + 13, number + 14, number + 15, number + 16, number + 17, number + 18, number + 19, number + 20, number + 21, number + 22, number + 23, number + 24, number + 25, number + 26, number + 27, number + 28, number + 29, number + 30, number + 31, number + 32, number + 33, number + 34, number + 35, number + 36, number + 37, number + 38, number + 39, number + 40, number + 41, number + 42, number + 43, number + 44, number + 45, number + 46, number + 47, number + 48, number + 49, number + 50, number + 51, number + 52, number + 53, number + 54, number + 55, number + 56, number + 57, number + 58, number + 59, number + 60, number + 61, number + 62, number + 63, number + 64, number + 65, number + 66, number + 67, number + 68, number + 69, number + 70, number + 71, number + 72, number + 73, number + 74, number + 75, number + 76, number + 77, number + 78, number + 79, number + 80, number + 81, number + 82, number + 83, number + 84, number + 85, number + 86, number + 87, number + 88, number + 89, number + 90, number + 91, number + 92, number + 93, number + 94, number + 95, number + 96, number + 97, number + 98, number + 99, number + 100, number + 101, number + 102, number + 103, number + 104, number + 105, number + 106, number + 107, number + 108, number + 109, number + 110, number + 111, number + 112, number + 113, number + 114, number + 115, number + 116, number + 117, number + 118, number + 119, number + 120, number + 121, number + 122, number + 123, number + 124, number + 125, number + 126, number + 127, number + 128, number + 129, number + 130, number + 131, number + 132, number + 133, number + 134, number + 135, number + 136, number + 137, number + 138, number + 139, number + 140, number + 141, number + 142, number + 143, number + 144, number + 145, number + 146, number + 147, number + 148, number + 149, number + 150, number + 151, number + 152, number + 153, number + 154, number + 155, number + 156, number + 157, number + 158, number + 159, number + 160, number + 161, number + 162, number + 163, number + 164, number + 165, number + 166, number + 167, number + 168, number + 169, number + 170, number + 171, number + 172, number + 173, number + 174, number + 175, number + 176, number + 177, number + 178, number + 179, number + 180, number + 181, number + 182, number + 183, number + 184, number + 185, number + 186, number + 187, number + 188, number + 189, number + 190, number + 191, number + 192, number + 193, number + 194, number + 195, number + 196, number + 197, number + 198, number + 199, number + 200, number + 201, number + 202, number + 203, number + 204, number + 205, number + 206, number + 207, number + 208, number + 209, number + 210, number + 211, number + 212, number + 213, number + 214, number + 215, number + 216, number + 217, number + 218, number + 219, number + 220, number + 221, number + 222, number + 223, number + 224, number + 225, number + 226, number + 227, number + 228, number + 229, number + 230, number + 231, number + 232, number + 233, number + 234, number + 235, number + 236, number + 237, number + 238, number + 239, number + 240, number + 241, number + 242, number + 243, number + 244, number + 245, number + 246, number + 247, number + 248, number + 249, number + 250, number + 251, number + 252, number + 253, number + 254, number + 255, number + 256, number + 257, number + 258, number + 259, number + 260, number + 261, number + 262, number + 263, number + 264, number + 265, number + 266, number + 267, number + 268, number + 269, number + 270, number + 271, number + 272, number + 273, number + 274, number + 275, number + 276, number + 277, number + 278, number + 279, number + 280, number + 281, number + 282, number + 283, number + 284, number + 285, number + 286, number + 287, number + 288, number + 289, number + 290, number + 291, number + 292, number + 293, number + 294, number + 295, number + 296, number + 297, number + 298, number + 299 from numbers(20)" - ) - small_node.query("optimize table tvm2 final") - small_node.query("system flush logs") - - # Should be about 25M - res = small_node.query( - "select formatReadableSize(peak_memory_usage), * from system.part_log where table = 'tvm2' and database = currentDatabase() and event_date >= today() - 1 and event_type = 'MergeParts' and peak_memory_usage > 100000000 format Vertical" - ) - - assert res == "" diff --git a/tests/integration/test_storage_nats/__init__.py b/tests/integration/test_storage_nats/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto b/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto deleted file mode 100755 index 090ed917cdd..00000000000 --- a/tests/integration/test_storage_nats/clickhouse_path/format_schemas/nats.proto +++ /dev/null @@ -1,6 +0,0 @@ -syntax = "proto3"; - - message ProtoKeyValue { - uint64 key = 1; - string value = 2; - } diff --git a/tests/integration/test_storage_nats/configs/macros.xml b/tests/integration/test_storage_nats/configs/macros.xml deleted file mode 100644 index 4aa547e049e..00000000000 --- a/tests/integration/test_storage_nats/configs/macros.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - nats1:4444 - macro - JSONEachRow - - diff --git a/tests/integration/test_storage_nats/configs/named_collection.xml b/tests/integration/test_storage_nats/configs/named_collection.xml deleted file mode 100644 index 15817f321f0..00000000000 --- a/tests/integration/test_storage_nats/configs/named_collection.xml +++ /dev/null @@ -1,13 +0,0 @@ - - - - nats1:4444 - named - JSONEachRow - 111 - 12 - click - house - - - diff --git a/tests/integration/test_storage_nats/configs/nats.xml b/tests/integration/test_storage_nats/configs/nats.xml deleted file mode 100644 index 0a8be9122ad..00000000000 --- a/tests/integration/test_storage_nats/configs/nats.xml +++ /dev/null @@ -1,6 +0,0 @@ - - - click - house - - diff --git a/tests/integration/test_storage_nats/configs/users.xml b/tests/integration/test_storage_nats/configs/users.xml deleted file mode 100644 index 2cef0a6de3c..00000000000 --- a/tests/integration/test_storage_nats/configs/users.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - - 1 - - - diff --git a/tests/integration/test_storage_nats/nats_certs.sh b/tests/integration/test_storage_nats/nats_certs.sh deleted file mode 100755 index 689221c39e4..00000000000 --- a/tests/integration/test_storage_nats/nats_certs.sh +++ /dev/null @@ -1,13 +0,0 @@ -#!/bin/bash -set -euxo pipefail - -mkdir -p "${NATS_CERT_DIR}/ca" -mkdir -p "${NATS_CERT_DIR}/nats" -openssl req -newkey rsa:4096 -x509 -days 365 -nodes -batch -keyout "${NATS_CERT_DIR}/ca/ca-key.pem" -out "${NATS_CERT_DIR}/ca/ca-cert.pem" -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=ca" -openssl req -newkey rsa:4096 -nodes -batch -keyout "${NATS_CERT_DIR}/nats/server-key.pem" -out "${NATS_CERT_DIR}/nats/server-req.pem" -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=server" -openssl x509 -req -days 365 -in "${NATS_CERT_DIR}/nats/server-req.pem" -CA "${NATS_CERT_DIR}/ca/ca-cert.pem" -CAkey "${NATS_CERT_DIR}/ca/ca-key.pem" -CAcreateserial -out "${NATS_CERT_DIR}/nats/server-cert.pem" -extfile <( -cat <<-EOF -subjectAltName = DNS:localhost, DNS:nats1 -EOF -) -rm -f "${NATS_CERT_DIR}/nats/server-req.pem" diff --git a/tests/integration/test_storage_nats/nats_pb2.py b/tests/integration/test_storage_nats/nats_pb2.py deleted file mode 100644 index e9e5cb72363..00000000000 --- a/tests/integration/test_storage_nats/nats_pb2.py +++ /dev/null @@ -1,37 +0,0 @@ -# -*- coding: utf-8 -*- -# Generated by the protocol buffer compiler. DO NOT EDIT! -# source: clickhouse_path/format_schemas/nats.proto -"""Generated protocol buffer code.""" -from google.protobuf import descriptor as _descriptor -from google.protobuf import descriptor_pool as _descriptor_pool -from google.protobuf import message as _message -from google.protobuf import reflection as _reflection -from google.protobuf import symbol_database as _symbol_database - -# @@protoc_insertion_point(imports) - -_sym_db = _symbol_database.Default() - - -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n)clickhouse_path/format_schemas/nats.proto"+\n\rProtoKeyValue\x12\x0b\n\x03key\x18\x01 \x01(\x04\x12\r\n\x05value\x18\x02 \x01(\tb\x06proto3' -) - - -_PROTOKEYVALUE = DESCRIPTOR.message_types_by_name["ProtoKeyValue"] -ProtoKeyValue = _reflection.GeneratedProtocolMessageType( - "ProtoKeyValue", - (_message.Message,), - { - "DESCRIPTOR": _PROTOKEYVALUE, - "__module__": "clickhouse_path.format_schemas.nats_pb2" - # @@protoc_insertion_point(class_scope:ProtoKeyValue) - }, -) -_sym_db.RegisterMessage(ProtoKeyValue) - -if _descriptor._USE_C_DESCRIPTORS == False: - DESCRIPTOR._options = None - _PROTOKEYVALUE._serialized_start = 45 - _PROTOKEYVALUE._serialized_end = 88 -# @@protoc_insertion_point(module_scope) diff --git a/tests/integration/test_storage_nats/test.py b/tests/integration/test_storage_nats/test.py deleted file mode 100644 index 4d7e4cf813d..00000000000 --- a/tests/integration/test_storage_nats/test.py +++ /dev/null @@ -1,1875 +0,0 @@ -import pytest - -# FIXME This test is too flaky -# https://github.com/ClickHouse/ClickHouse/issues/39185 - -pytestmark = pytest.mark.skip - -import json -import os.path as p -import random -import subprocess -import threading -import logging -import time -from random import randrange -import math - -import asyncio -from google.protobuf.internal.encoder import _VarintBytes -from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster, check_nats_is_available, nats_connect_ssl -from helpers.test_tools import TSV - -from . import nats_pb2 - -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "instance", - main_configs=[ - "configs/nats.xml", - "configs/macros.xml", - "configs/named_collection.xml", - ], - user_configs=["configs/users.xml"], - with_nats=True, - clickhouse_path_dir="clickhouse_path", -) - - -# Helpers - - -def wait_nats_to_start(nats_port, ssl_ctx=None, timeout=180): - start = time.time() - while time.time() - start < timeout: - try: - if asyncio.run(check_nats_is_available(nats_port, ssl_ctx=ssl_ctx)): - logging.debug("NATS is available") - return - time.sleep(0.5) - except Exception as ex: - logging.debug("Can't connect to NATS " + str(ex)) - time.sleep(0.5) - - -def nats_check_result(result, check=False, ref_file="test_nats_json.reference"): - fpath = p.join(p.dirname(__file__), ref_file) - with open(fpath) as reference: - if check: - assert TSV(result) == TSV(reference) - else: - return TSV(result) == TSV(reference) - - -def kill_nats(nats_id): - p = subprocess.Popen(("docker", "stop", nats_id), stdout=subprocess.PIPE) - p.communicate() - return p.returncode == 0 - - -def revive_nats(nats_id, nats_port): - p = subprocess.Popen(("docker", "start", nats_id), stdout=subprocess.PIPE) - p.communicate() - wait_nats_to_start(nats_port) - - -# Fixtures - - -@pytest.fixture(scope="module") -def nats_cluster(): - try: - cluster.start() - logging.debug("nats_id is {}".format(instance.cluster.nats_docker_id)) - instance.query("CREATE DATABASE test") - - yield cluster - - finally: - cluster.shutdown() - - -@pytest.fixture(autouse=True) -def nats_setup_teardown(): - print("NATS is available - running test") - yield # run test - instance.query("DROP DATABASE test SYNC") - instance.query("CREATE DATABASE test") - - -# Tests - - -async def nats_produce_messages(cluster_inst, subject, messages=(), bytes=None): - nc = await nats_connect_ssl( - cluster_inst.nats_port, - user="click", - password="house", - ssl_ctx=cluster_inst.nats_ssl_context, - ) - logging.debug("NATS connection status: " + str(nc.is_connected)) - - for message in messages: - await nc.publish(subject, message.encode()) - if bytes is not None: - await nc.publish(subject, bytes) - logging.debug("Finished publising to " + subject) - - await nc.close() - return messages - - -def check_table_is_ready(instance, table_name): - try: - instance.query("SELECT * FROM {}".format(table_name)) - return True - except Exception: - return False - - -def test_nats_select(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'select', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "select", messages)) - - # The order of messages in select * from test.nats is not guaranteed, so sleep to collect everything in one select - time.sleep(1) - - result = "" - while True: - result += instance.query( - "SELECT * FROM test.nats ORDER BY key", ignore_error=True - ) - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_select_empty(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'empty', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - """ - ) - - assert int(instance.query("SELECT count() FROM test.nats")) == 0 - - -def test_nats_json_without_delimiter(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'json', - nats_format = 'JSONEachRow'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = "" - for i in range(25): - messages += json.dumps({"key": i, "value": i}) + "\n" - - all_messages = [messages] - asyncio.run(nats_produce_messages(nats_cluster, "json", all_messages)) - - messages = "" - for i in range(25, 50): - messages += json.dumps({"key": i, "value": i}) + "\n" - all_messages = [messages] - asyncio.run(nats_produce_messages(nats_cluster, "json", all_messages)) - - time.sleep(1) - - result = "" - time_limit_sec = 60 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result += instance.query( - "SELECT * FROM test.nats ORDER BY key", ignore_error=True - ) - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_csv_with_delimiter(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'csv', - nats_format = 'CSV', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append("{i}, {i}".format(i=i)) - - asyncio.run(nats_produce_messages(nats_cluster, "csv", messages)) - - time.sleep(1) - - result = "" - for _ in range(60): - result += instance.query( - "SELECT * FROM test.nats ORDER BY key", ignore_error=True - ) - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_tsv_with_delimiter(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'tsv', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append("{i}\t{i}".format(i=i)) - - asyncio.run(nats_produce_messages(nats_cluster, "tsv", messages)) - - result = "" - for _ in range(60): - result = instance.query("SELECT * FROM test.view ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - -# - - -def test_nats_macros(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = '{nats_url}', - nats_subjects = '{nats_subjects}', - nats_format = '{nats_format}' - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - message = "" - for i in range(50): - message += json.dumps({"key": i, "value": i}) + "\n" - asyncio.run(nats_produce_messages(nats_cluster, "macro", [message])) - - time.sleep(1) - - result = "" - for _ in range(60): - result += instance.query( - "SELECT * FROM test.nats ORDER BY key", ignore_error=True - ) - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_materialized_view(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'mv', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - - CREATE TABLE test.view2 (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer2 TO test.view2 AS - SELECT * FROM test.nats group by (key, value); - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - - asyncio.run(nats_produce_messages(nats_cluster, "mv", messages)) - - time_limit_sec = 60 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT * FROM test.view ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT * FROM test.view2 ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_materialized_view_with_subquery(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'mvsq', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM (SELECT * FROM test.nats); - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mvsq", messages)) - - time_limit_sec = 60 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT * FROM test.view ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_many_materialized_views(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.view1; - DROP TABLE IF EXISTS test.view2; - DROP TABLE IF EXISTS test.consumer1; - DROP TABLE IF EXISTS test.consumer2; - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'mmv', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view1 (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE TABLE test.view2 (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer1 TO test.view1 AS - SELECT * FROM test.nats; - CREATE MATERIALIZED VIEW test.consumer2 TO test.view2 AS - SELECT * FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(50): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mmv", messages)) - - time_limit_sec = 60 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result1 = instance.query("SELECT * FROM test.view1 ORDER BY key") - result2 = instance.query("SELECT * FROM test.view2 ORDER BY key") - if nats_check_result(result1) and nats_check_result(result2): - break - - instance.query( - """ - DROP TABLE test.consumer1; - DROP TABLE test.consumer2; - DROP TABLE test.view1; - DROP TABLE test.view2; - """ - ) - - nats_check_result(result1, True) - nats_check_result(result2, True) - - -def test_nats_protobuf(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value String) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'pb', - nats_format = 'Protobuf', - nats_schema = 'nats.proto:ProtoKeyValue'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - data = b"" - for i in range(0, 20): - msg = nats_pb2.ProtoKeyValue() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - asyncio.run(nats_produce_messages(nats_cluster, "pb", bytes=data)) - data = b"" - for i in range(20, 21): - msg = nats_pb2.ProtoKeyValue() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - asyncio.run(nats_produce_messages(nats_cluster, "pb", bytes=data)) - data = b"" - for i in range(21, 50): - msg = nats_pb2.ProtoKeyValue() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - asyncio.run(nats_produce_messages(nats_cluster, "pb", bytes=data)) - - result = "" - time_limit_sec = 60 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT * FROM test.view ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - -def test_nats_big_message(nats_cluster): - # Create batchs of messages of size ~100Kb - nats_messages = 1000 - batch_messages = 1000 - messages = [ - json.dumps({"key": i, "value": "x" * 100}) * batch_messages - for i in range(nats_messages) - ] - - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value String) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'big', - nats_format = 'JSONEachRow'; - CREATE TABLE test.view (key UInt64, value String) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - asyncio.run(nats_produce_messages(nats_cluster, "big", messages)) - - while True: - result = instance.query("SELECT count() FROM test.view") - if int(result) == batch_messages * nats_messages: - break - - assert ( - int(result) == nats_messages * batch_messages - ), "ClickHouse lost some messages: {}".format(result) - - -def test_nats_mv_combo(nats_cluster): - NUM_MV = 5 - NUM_CONSUMERS = 4 - - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'combo', - nats_num_consumers = {}, - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - """.format( - NUM_CONSUMERS - ) - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - for mv_id in range(NUM_MV): - instance.query( - """ - DROP TABLE IF EXISTS test.combo_{0}; - DROP TABLE IF EXISTS test.combo_{0}_mv; - CREATE TABLE test.combo_{0} (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.combo_{0}_mv TO test.combo_{0} AS - SELECT * FROM test.nats; - """.format( - mv_id - ) - ) - - time.sleep(2) - - i = [0] - messages_num = 10000 - - def produce(): - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({"key": i[0], "value": i[0]})) - i[0] += 1 - asyncio.run(nats_produce_messages(nats_cluster, "combo", messages)) - - threads = [] - threads_num = 20 - - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - while True: - result = 0 - for mv_id in range(NUM_MV): - result += int( - instance.query("SELECT count() FROM test.combo_{0}".format(mv_id)) - ) - if int(result) == messages_num * threads_num * NUM_MV: - break - time.sleep(1) - - for thread in threads: - thread.join() - - for mv_id in range(NUM_MV): - instance.query( - """ - DROP TABLE test.combo_{0}_mv; - DROP TABLE test.combo_{0}; - """.format( - mv_id - ) - ) - - assert ( - int(result) == messages_num * threads_num * NUM_MV - ), "ClickHouse lost some messages: {}".format(result) - - -def test_nats_insert(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'insert', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - values = [] - for i in range(50): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - insert_messages = [] - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe("insert") - await sub.unsubscribe(50) - async for msg in sub.messages: - insert_messages.append(msg.data.decode()) - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - while True: - try: - instance.query("INSERT INTO test.nats VALUES {}".format(values)) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - thread.join() - - result = "\n".join(insert_messages) - nats_check_result(result, True) - - -def test_nats_many_subjects_insert_wrong(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'insert1,insert2.>,insert3.*.foo', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - values = [] - for i in range(50): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - # no subject specified - instance.query_and_get_error("INSERT INTO test.nats VALUES {}".format(values)) - - # can't insert into wildcard subjects - instance.query_and_get_error( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert2.>' VALUES {}".format( - values - ) - ) - instance.query_and_get_error( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert3.*.foo' VALUES {}".format( - values - ) - ) - - # specified subject is not among engine's subjects - instance.query_and_get_error( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert4' VALUES {}".format( - values - ) - ) - instance.query_and_get_error( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='insert3.foo.baz' VALUES {}".format( - values - ) - ) - instance.query_and_get_error( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='foo.insert2' VALUES {}".format( - values - ) - ) - - -def test_nats_many_subjects_insert_right(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'right_insert1,right_insert2', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - values = [] - for i in range(50): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - insert_messages = [] - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe("right_insert1") - await sub.unsubscribe(50) - async for msg in sub.messages: - insert_messages.append(msg.data.decode()) - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - while True: - try: - instance.query( - "INSERT INTO test.nats SETTINGS stream_like_engine_insert_queue='right_insert1' VALUES {}".format( - values - ) - ) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - thread.join() - - result = "\n".join(insert_messages) - nats_check_result(result, True) - - -def test_nats_many_inserts(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.nats_many; - DROP TABLE IF EXISTS test.nats_consume; - DROP TABLE IF EXISTS test.view_many; - DROP TABLE IF EXISTS test.consumer_many; - CREATE TABLE test.nats_many (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'many_inserts', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - CREATE TABLE test.nats_consume (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'many_inserts', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view_many (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer_many TO test.view_many AS - SELECT * FROM test.nats_consume; - """ - ) - while not check_table_is_ready(instance, "test.nats_consume"): - logging.debug("Table test.nats_consume is not yet ready") - time.sleep(0.5) - - messages_num = 10000 - values = [] - for i in range(messages_num): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - def insert(): - while True: - try: - instance.query("INSERT INTO test.nats_many VALUES {}".format(values)) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - - threads = [] - threads_num = 10 - for _ in range(threads_num): - threads.append(threading.Thread(target=insert)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - for thread in threads: - thread.join() - - time_limit_sec = 300 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT count() FROM test.view_many") - print(result, messages_num * threads_num) - if int(result) >= messages_num * threads_num: - break - time.sleep(1) - - instance.query( - """ - DROP TABLE test.nats_consume; - DROP TABLE test.nats_many; - DROP TABLE test.consumer_many; - DROP TABLE test.view_many; - """ - ) - - assert ( - int(result) == messages_num * threads_num - ), "ClickHouse lost some messages or got duplicated ones. Total count: {}".format( - result - ) - - -def test_nats_overloaded_insert(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.view_overload; - DROP TABLE IF EXISTS test.consumer_overload; - DROP TABLE IF EXISTS test.nats_consume; - CREATE TABLE test.nats_consume (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'over', - nats_num_consumers = 5, - nats_max_block_size = 10000, - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - CREATE TABLE test.nats_overload (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'over', - nats_format = 'TSV', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view_overload (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key - SETTINGS old_parts_lifetime=5, cleanup_delay_period=2, cleanup_delay_period_random_add=3, - cleanup_thread_preferred_points_per_iteration=0; - CREATE MATERIALIZED VIEW test.consumer_overload TO test.view_overload AS - SELECT * FROM test.nats_consume; - """ - ) - while not check_table_is_ready(instance, "test.nats_consume"): - logging.debug("Table test.nats_consume is not yet ready") - time.sleep(0.5) - - messages_num = 100000 - - def insert(): - values = [] - for i in range(messages_num): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - while True: - try: - instance.query( - "INSERT INTO test.nats_overload VALUES {}".format(values) - ) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - - threads = [] - threads_num = 5 - for _ in range(threads_num): - threads.append(threading.Thread(target=insert)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - time_limit_sec = 300 - deadline = time.monotonic() + time_limit_sec - - while time.monotonic() < deadline: - result = instance.query("SELECT count() FROM test.view_overload") - time.sleep(1) - if int(result) >= messages_num * threads_num: - break - - instance.query( - """ - DROP TABLE test.consumer_overload; - DROP TABLE test.view_overload; - DROP TABLE test.nats_consume; - DROP TABLE test.nats_overload; - """ - ) - - for thread in threads: - thread.join() - - assert ( - int(result) == messages_num * threads_num - ), "ClickHouse lost some messages or got duplicated ones. Total count: {}".format( - result - ) - - -def test_nats_virtual_column(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats_virtuals (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'virtuals', - nats_format = 'JSONEachRow'; - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT value, key, _subject FROM test.nats_virtuals; - """ - ) - while not check_table_is_ready(instance, "test.nats_virtuals"): - logging.debug("Table test.nats_virtuals is not yet ready") - time.sleep(0.5) - - message_num = 10 - i = 0 - messages = [] - for _ in range(message_num): - messages.append(json.dumps({"key": i, "value": i})) - i += 1 - - asyncio.run(nats_produce_messages(nats_cluster, "virtuals", messages)) - - while True: - result = instance.query("SELECT count() FROM test.view") - time.sleep(1) - if int(result) == message_num: - break - - result = instance.query( - """ - SELECT key, value, _subject - FROM test.view ORDER BY key - """ - ) - - expected = """\ -0 0 virtuals -1 1 virtuals -2 2 virtuals -3 3 virtuals -4 4 virtuals -5 5 virtuals -6 6 virtuals -7 7 virtuals -8 8 virtuals -9 9 virtuals -""" - - instance.query( - """ - DROP TABLE test.nats_virtuals; - DROP TABLE test.view; - """ - ) - - assert TSV(result) == TSV(expected) - - -def test_nats_virtual_column_with_materialized_view(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats_virtuals_mv (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'virtuals_mv', - nats_format = 'JSONEachRow'; - CREATE TABLE test.view (key UInt64, value UInt64, subject String) ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT *, _subject as subject - FROM test.nats_virtuals_mv; - """ - ) - while not check_table_is_ready(instance, "test.nats_virtuals_mv"): - logging.debug("Table test.nats_virtuals_mv is not yet ready") - time.sleep(0.5) - - message_num = 10 - i = 0 - messages = [] - for _ in range(message_num): - messages.append(json.dumps({"key": i, "value": i})) - i += 1 - - asyncio.run(nats_produce_messages(nats_cluster, "virtuals_mv", messages)) - - while True: - result = instance.query("SELECT count() FROM test.view") - time.sleep(1) - if int(result) == message_num: - break - - result = instance.query("SELECT key, value, subject FROM test.view ORDER BY key") - expected = """\ -0 0 virtuals_mv -1 1 virtuals_mv -2 2 virtuals_mv -3 3 virtuals_mv -4 4 virtuals_mv -5 5 virtuals_mv -6 6 virtuals_mv -7 7 virtuals_mv -8 8 virtuals_mv -9 9 virtuals_mv -""" - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.view; - DROP TABLE test.nats_virtuals_mv - """ - ) - - assert TSV(result) == TSV(expected) - - -def test_nats_many_consumers_to_each_queue(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.destination; - CREATE TABLE test.destination(key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - """ - ) - - num_tables = 4 - for table_id in range(num_tables): - print(("Setting up table {}".format(table_id))) - instance.query( - """ - DROP TABLE IF EXISTS test.many_consumers_{0}; - DROP TABLE IF EXISTS test.many_consumers_{0}_mv; - CREATE TABLE test.many_consumers_{0} (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'many_consumers', - nats_num_consumers = 2, - nats_queue_group = 'many_consumers', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - CREATE MATERIALIZED VIEW test.many_consumers_{0}_mv TO test.destination AS - SELECT key, value FROM test.many_consumers_{0}; - """.format( - table_id - ) - ) - while not check_table_is_ready( - instance, "test.many_consumers_{}".format(table_id) - ): - logging.debug( - "Table test.many_consumers_{} is not yet ready".format(table_id) - ) - time.sleep(0.5) - - i = [0] - messages_num = 1000 - - def produce(): - messages = [] - for _ in range(messages_num): - messages.append(json.dumps({"key": i[0], "value": i[0]})) - i[0] += 1 - asyncio.run(nats_produce_messages(nats_cluster, "many_consumers", messages)) - - threads = [] - threads_num = 20 - - for _ in range(threads_num): - threads.append(threading.Thread(target=produce)) - for thread in threads: - time.sleep(random.uniform(0, 1)) - thread.start() - - result1 = "" - while True: - result1 = instance.query("SELECT count() FROM test.destination") - time.sleep(1) - if int(result1) == messages_num * threads_num: - break - - for thread in threads: - thread.join() - - for consumer_id in range(num_tables): - instance.query( - """ - DROP TABLE test.many_consumers_{0}; - DROP TABLE test.many_consumers_{0}_mv; - """.format( - consumer_id - ) - ) - - instance.query( - """ - DROP TABLE test.destination; - """ - ) - - assert ( - int(result1) == messages_num * threads_num - ), "ClickHouse lost some messages: {}".format(result1) - - -def test_nats_restore_failed_connection_without_losses_on_write(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.consume; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE TABLE test.consume (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'producer_reconnect', - nats_format = 'JSONEachRow', - nats_num_consumers = 2, - nats_row_delimiter = '\\n'; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.consume; - DROP TABLE IF EXISTS test.producer_reconnect; - CREATE TABLE test.producer_reconnect (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'producer_reconnect', - nats_format = 'JSONEachRow', - nats_row_delimiter = '\\n'; - """ - ) - while not check_table_is_ready(instance, "test.consume"): - logging.debug("Table test.consume is not yet ready") - time.sleep(0.5) - - messages_num = 100000 - values = [] - for i in range(messages_num): - values.append("({i}, {i})".format(i=i)) - values = ",".join(values) - - while True: - try: - instance.query( - "INSERT INTO test.producer_reconnect VALUES {}".format(values) - ) - break - except QueryRuntimeException as e: - if "Local: Timed out." in str(e): - continue - else: - raise - - while int(instance.query("SELECT count() FROM test.view")) == 0: - time.sleep(0.1) - - kill_nats(nats_cluster.nats_docker_id) - time.sleep(4) - revive_nats(nats_cluster.nats_docker_id, nats_cluster.nats_port) - - while True: - result = instance.query("SELECT count(DISTINCT key) FROM test.view") - time.sleep(1) - if int(result) == messages_num: - break - - instance.query( - """ - DROP TABLE test.consume; - DROP TABLE test.producer_reconnect; - """ - ) - - assert int(result) == messages_num, "ClickHouse lost some messages: {}".format( - result - ) - - -def test_nats_no_connection_at_startup_1(nats_cluster): - # no connection when table is initialized - nats_cluster.pause_container("nats1") - instance.query_and_get_error( - """ - CREATE TABLE test.cs (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'cs', - nats_format = 'JSONEachRow', - nats_num_consumers = '5', - nats_row_delimiter = '\\n'; - """ - ) - nats_cluster.unpause_container("nats1") - - -def test_nats_no_connection_at_startup_2(nats_cluster): - instance.query( - """ - CREATE TABLE test.cs (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'cs', - nats_format = 'JSONEachRow', - nats_num_consumers = '5', - nats_row_delimiter = '\\n'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.cs; - """ - ) - - instance.query("DETACH TABLE test.cs") - nats_cluster.pause_container("nats1") - instance.query("ATTACH TABLE test.cs") - nats_cluster.unpause_container("nats1") - while not check_table_is_ready(instance, "test.cs"): - logging.debug("Table test.cs is not yet ready") - time.sleep(0.5) - - messages_num = 1000 - messages = [] - for i in range(messages_num): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "cs", messages)) - - for _ in range(20): - result = instance.query("SELECT count() FROM test.view") - time.sleep(1) - if int(result) == messages_num: - break - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.cs; - """ - ) - - assert int(result) == messages_num, "ClickHouse lost some messages: {}".format( - result - ) - - -def test_nats_format_factory_settings(nats_cluster): - instance.query( - """ - CREATE TABLE test.format_settings ( - id String, date DateTime - ) ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'format_settings', - nats_format = 'JSONEachRow', - date_time_input_format = 'best_effort'; - """ - ) - while not check_table_is_ready(instance, "test.format_settings"): - logging.debug("Table test.format_settings is not yet ready") - time.sleep(0.5) - - message = json.dumps( - {"id": "format_settings_test", "date": "2021-01-19T14:42:33.1829214Z"} - ) - expected = instance.query( - """SELECT parseDateTimeBestEffort(CAST('2021-01-19T14:42:33.1829214Z', 'String'))""" - ) - - asyncio.run(nats_produce_messages(nats_cluster, "format_settings", [message])) - - while True: - result = instance.query("SELECT date FROM test.format_settings") - if result == expected: - break - - instance.query( - """ - CREATE TABLE test.view ( - id String, date DateTime - ) ENGINE = MergeTree ORDER BY id; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.format_settings; - """ - ) - - asyncio.run(nats_produce_messages(nats_cluster, "format_settings", [message])) - while True: - result = instance.query("SELECT date FROM test.view") - if result == expected: - break - - instance.query( - """ - DROP TABLE test.consumer; - DROP TABLE test.format_settings; - """ - ) - - assert result == expected - - -def test_nats_bad_args(nats_cluster): - instance.query_and_get_error( - """ - CREATE TABLE test.drop (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_secure = true, - nats_format = 'JSONEachRow'; - """ - ) - - -def test_nats_drop_mv(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'mv', - nats_format = 'JSONEachRow'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - messages = [] - for i in range(20): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mv", messages)) - - instance.query("DROP VIEW test.consumer") - messages = [] - for i in range(20, 40): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mv", messages)) - - instance.query( - """ - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.nats; - """ - ) - messages = [] - for i in range(40, 50): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mv", messages)) - - while True: - result = instance.query("SELECT * FROM test.view ORDER BY key") - if nats_check_result(result): - break - - nats_check_result(result, True) - - instance.query("DROP VIEW test.consumer") - messages = [] - for i in range(50, 60): - messages.append(json.dumps({"key": i, "value": i})) - asyncio.run(nats_produce_messages(nats_cluster, "mv", messages)) - - count = 0 - while True: - count = int(instance.query("SELECT count() FROM test.nats")) - if count: - break - - assert count > 0 - - -def test_nats_predefined_configuration(nats_cluster): - instance.query( - """ - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS(nats1) """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - asyncio.run( - nats_produce_messages( - nats_cluster, "named", [json.dumps({"key": 1, "value": 2})] - ) - ) - while True: - result = instance.query( - "SELECT * FROM test.nats ORDER BY key", ignore_error=True - ) - if result == "1\t2\n": - break - - -def test_format_with_prefix_and_suffix(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.nats; - - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'custom', - nats_format = 'CustomSeparated'; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - insert_messages = [] - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe("custom") - await sub.unsubscribe(2) - async for msg in sub.messages: - insert_messages.append(msg.data.decode()) - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - instance.query( - "INSERT INTO test.nats select number*10 as key, number*100 as value from numbers(2) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" - ) - - thread.join() - - assert ( - "".join(insert_messages) - == "\n0\t0\n\n\n10\t100\n\n" - ) - - -def test_max_rows_per_message(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.nats; - - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'custom1', - nats_format = 'CustomSeparated', - nats_max_rows_per_message = 3, - format_custom_result_before_delimiter = '\n', - format_custom_result_after_delimiter = '\n'; - - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.nats; - """ - ) - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - num_rows = 5 - - insert_messages = [] - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe("custom1") - await sub.unsubscribe(2) - async for msg in sub.messages: - insert_messages.append(msg.data.decode()) - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - instance.query( - f"INSERT INTO test.nats select number*10 as key, number*100 as value from numbers({num_rows}) settings format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n'" - ) - - thread.join() - - assert ( - "".join(insert_messages) - == "\n0\t0\n10\t100\n20\t200\n\n\n30\t300\n40\t400\n\n" - ) - - attempt = 0 - rows = 0 - while attempt < 100: - rows = int(instance.query("SELECT count() FROM test.view")) - if rows == num_rows: - break - attempt += 1 - - assert rows == num_rows - - result = instance.query("SELECT * FROM test.view") - assert result == "0\t0\n10\t100\n20\t200\n30\t300\n40\t400\n" - - -def test_row_based_formats(nats_cluster): - num_rows = 10 - - for format_name in [ - "TSV", - "TSVWithNamesAndTypes", - "TSKV", - "CSV", - "CSVWithNamesAndTypes", - "CustomSeparatedWithNamesAndTypes", - "Values", - "JSON", - "JSONEachRow", - "JSONCompactEachRow", - "JSONCompactEachRowWithNamesAndTypes", - "JSONObjectEachRow", - "Avro", - "RowBinary", - "RowBinaryWithNamesAndTypes", - "MsgPack", - ]: - print(format_name) - - instance.query( - f""" - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.nats; - - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = '{format_name}', - nats_format = '{format_name}'; - - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.nats; - """ - ) - - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - insert_messages = 0 - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe(format_name) - await sub.unsubscribe(2) - async for msg in sub.messages: - nonlocal insert_messages - insert_messages += 1 - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - instance.query( - f"INSERT INTO test.nats select number*10 as key, number*100 as value from numbers({num_rows})" - ) - - thread.join() - - assert insert_messages == 2 - - attempt = 0 - rows = 0 - while attempt < 100: - rows = int(instance.query("SELECT count() FROM test.view")) - if rows == num_rows: - break - attempt += 1 - - assert rows == num_rows - - expected = "" - for i in range(num_rows): - expected += str(i * 10) + "\t" + str(i * 100) + "\n" - - result = instance.query("SELECT * FROM test.view") - assert result == expected - - -def test_block_based_formats_1(nats_cluster): - instance.query( - """ - DROP TABLE IF EXISTS test.nats; - - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = 'PrettySpace', - nats_format = 'PrettySpace'; - """ - ) - - insert_messages = [] - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe("PrettySpace") - await sub.unsubscribe(3) - async for msg in sub.messages: - insert_messages.append(msg.data.decode()) - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - attempt = 0 - while attempt < 100: - try: - instance.query( - "INSERT INTO test.nats SELECT number * 10 as key, number * 100 as value FROM numbers(5) settings max_block_size=2, optimize_trivial_insert_select=0;" - ) - break - except Exception: - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - attempt += 1 - thread.join() - - data = [] - for message in insert_messages: - splitted = message.split("\n") - assert splitted[0] == " \x1b[1mkey\x1b[0m \x1b[1mvalue\x1b[0m" - assert splitted[1] == "" - assert splitted[-1] == "" - data += [line.split() for line in splitted[2:-1]] - - assert data == [ - ["0", "0"], - ["10", "100"], - ["20", "200"], - ["30", "300"], - ["40", "400"], - ] - - -def test_block_based_formats_2(nats_cluster): - num_rows = 100 - - for format_name in [ - "JSONColumns", - "Native", - "Arrow", - "Parquet", - "ORC", - "JSONCompactColumns", - ]: - print(format_name) - - instance.query( - f""" - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.nats; - - CREATE TABLE test.nats (key UInt64, value UInt64) - ENGINE = NATS - SETTINGS nats_url = 'nats1:4444', - nats_subjects = '{format_name}', - nats_format = '{format_name}'; - - CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT key, value FROM test.nats; - """ - ) - - while not check_table_is_ready(instance, "test.nats"): - logging.debug("Table test.nats is not yet ready") - time.sleep(0.5) - - insert_messages = 0 - - async def sub_to_nats(): - nc = await nats_connect_ssl( - nats_cluster.nats_port, - user="click", - password="house", - ssl_ctx=nats_cluster.nats_ssl_context, - ) - sub = await nc.subscribe(format_name) - await sub.unsubscribe(9) - async for msg in sub.messages: - nonlocal insert_messages - insert_messages += 1 - - await sub.drain() - await nc.drain() - - def run_sub(): - asyncio.run(sub_to_nats()) - - thread = threading.Thread(target=run_sub) - thread.start() - time.sleep(1) - - instance.query( - f"INSERT INTO test.nats SELECT number * 10 as key, number * 100 as value FROM numbers({num_rows}) settings max_block_size=12, optimize_trivial_insert_select=0;" - ) - - thread.join() - - assert insert_messages == 9 - - attempt = 0 - rows = 0 - while attempt < 100: - rows = int(instance.query("SELECT count() FROM test.view")) - if rows == num_rows: - break - attempt += 1 - - assert rows == num_rows - - result = instance.query("SELECT * FROM test.view ORDER by key") - expected = "" - for i in range(num_rows): - expected += str(i * 10) + "\t" + str(i * 100) + "\n" - assert result == expected - - -if __name__ == "__main__": - cluster.start() - input("Cluster created, press any key to destroy...") - cluster.shutdown() diff --git a/tests/integration/test_storage_nats/test_nats_json.reference b/tests/integration/test_storage_nats/test_nats_json.reference deleted file mode 100644 index 959bb2aad74..00000000000 --- a/tests/integration/test_storage_nats/test_nats_json.reference +++ /dev/null @@ -1,50 +0,0 @@ -0 0 -1 1 -2 2 -3 3 -4 4 -5 5 -6 6 -7 7 -8 8 -9 9 -10 10 -11 11 -12 12 -13 13 -14 14 -15 15 -16 16 -17 17 -18 18 -19 19 -20 20 -21 21 -22 22 -23 23 -24 24 -25 25 -26 26 -27 27 -28 28 -29 29 -30 30 -31 31 -32 32 -33 33 -34 34 -35 35 -36 36 -37 37 -38 38 -39 39 -40 40 -41 41 -42 42 -43 43 -44 44 -45 45 -46 46 -47 47 -48 48 -49 49 diff --git a/tests/integration/test_storage_postgresql_replica/__init__.py b/tests/integration/test_storage_postgresql_replica/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_storage_postgresql_replica/configs/log_conf.xml b/tests/integration/test_storage_postgresql_replica/configs/log_conf.xml deleted file mode 100644 index 27c7107ce5e..00000000000 --- a/tests/integration/test_storage_postgresql_replica/configs/log_conf.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - trace - /var/log/clickhouse-server/log.log - /var/log/clickhouse-server/log.err.log - 1000M - 10 - /var/log/clickhouse-server/stderr.log - /var/log/clickhouse-server/stdout.log - - diff --git a/tests/integration/test_storage_postgresql_replica/test.py b/tests/integration/test_storage_postgresql_replica/test.py deleted file mode 100644 index 66495700102..00000000000 --- a/tests/integration/test_storage_postgresql_replica/test.py +++ /dev/null @@ -1,780 +0,0 @@ -import pytest - -# FIXME Tests with MaterializedPostgresSQL are temporarily disabled -# https://github.com/ClickHouse/ClickHouse/issues/36898 -# https://github.com/ClickHouse/ClickHouse/issues/38677 -# https://github.com/ClickHouse/ClickHouse/pull/39272#issuecomment-1190087190 - -pytestmark = pytest.mark.skip - -import time -import psycopg2 -import os.path as p - -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry -from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT -from helpers.test_tools import TSV - -import threading - -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "instance", - main_configs=["configs/log_conf.xml"], - with_postgres=True, - stay_alive=True, -) - -postgres_table_template = """ - CREATE TABLE IF NOT EXISTS {} ( - key Integer NOT NULL, value Integer, PRIMARY KEY(key)) - """ - -queries = [ - "INSERT INTO {} select i, i from generate_series(0, 10000) as t(i);", - "DELETE FROM {} WHERE (value*value) % 3 = 0;", - "UPDATE {} SET value = value + 125 WHERE key % 2 = 0;", - "UPDATE {} SET key=key+20000 WHERE key%2=0", - "INSERT INTO {} select i, i from generate_series(40000, 50000) as t(i);", - "DELETE FROM {} WHERE key % 10 = 0;", - "UPDATE {} SET value = value + 101 WHERE key % 2 = 1;", - "UPDATE {} SET key=key+80000 WHERE key%2=1", - "DELETE FROM {} WHERE value % 2 = 0;", - "UPDATE {} SET value = value + 2000 WHERE key % 5 = 0;", - "INSERT INTO {} select i, i from generate_series(200000, 250000) as t(i);", - "DELETE FROM {} WHERE value % 3 = 0;", - "UPDATE {} SET value = value * 2 WHERE key % 3 = 0;", - "UPDATE {} SET key=key+500000 WHERE key%2=1", - "INSERT INTO {} select i, i from generate_series(1000000, 1050000) as t(i);", - "DELETE FROM {} WHERE value % 9 = 2;", - "UPDATE {} SET key=key+10000000", - "UPDATE {} SET value = value + 2 WHERE key % 3 = 1;", - "DELETE FROM {} WHERE value%5 = 0;", -] - - -@pytest.mark.timeout(30) -def check_tables_are_synchronized( - table_name, order_by="key", postgres_database="postgres_database" -): - while True: - expected = instance.query( - "select * from {}.{} order by {};".format( - postgres_database, table_name, order_by - ) - ) - result = instance.query( - "select * from test.{} order by {};".format(table_name, order_by) - ) - if result == expected: - break - - assert result == expected - - -def get_postgres_conn( - ip, port, database=False, auto_commit=True, database_name="postgres_database" -): - if database == True: - conn_string = "host={} port={} dbname='{}' user='postgres' password='mysecretpassword'".format( - ip, port, database_name - ) - else: - conn_string = ( - "host={} port={} user='postgres' password='mysecretpassword'".format( - ip, port - ) - ) - - conn = psycopg2.connect(conn_string) - if auto_commit: - conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) - conn.autocommit = True - return conn - - -def create_postgres_db(cursor, name): - cursor.execute("CREATE DATABASE {}".format(name)) - - -def create_clickhouse_postgres_db(ip, port, name="postgres_database"): - instance.query( - """ - CREATE DATABASE {} - ENGINE = PostgreSQL('{}:{}', '{}', 'postgres', 'mysecretpassword')""".format( - name, ip, port, name - ) - ) - - -def create_materialized_table(ip, port, table_name="postgresql_replica"): - instance.query( - f""" - CREATE TABLE test.{table_name} (key Int64, value Int64) - ENGINE = MaterializedPostgreSQL( - '{ip}:{port}', 'postgres_database', '{table_name}', 'postgres', 'mysecretpassword') - PRIMARY KEY key; """ - ) - - -def create_postgres_table(cursor, table_name, replica_identity_full=False): - cursor.execute("DROP TABLE IF EXISTS {}".format(table_name)) - cursor.execute(postgres_table_template.format(table_name)) - if replica_identity_full: - cursor.execute("ALTER TABLE {} REPLICA IDENTITY FULL;".format(table_name)) - - -def postgresql_replica_check_result( - result, check=False, ref_file="test_postgresql_replica.reference" -): - fpath = p.join(p.dirname(__file__), ref_file) - with open(fpath) as reference: - if check: - assert TSV(result) == TSV(reference) - else: - return TSV(result) == TSV(reference) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - conn = get_postgres_conn(ip=cluster.postgres_ip, port=cluster.postgres_port) - cursor = conn.cursor() - create_postgres_db(cursor, "postgres_database") - create_clickhouse_postgres_db( - ip=cluster.postgres_ip, port=cluster.postgres_port - ) - - instance.query("CREATE DATABASE test") - yield cluster - - finally: - cluster.shutdown() - - -@pytest.mark.timeout(320) -def test_initial_load_from_snapshot(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.2) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -@pytest.mark.timeout(320) -def test_no_connection_at_startup(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - time.sleep(3) - - instance.query("DETACH TABLE test.postgresql_replica") - started_cluster.pause_container("postgres1") - - instance.query("ATTACH TABLE test.postgresql_replica") - time.sleep(3) - started_cluster.unpause_container("postgres1") - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) == 0: - time.sleep(0.5) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -@pytest.mark.timeout(320) -def test_detach_attach_is_ok(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) == 0: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - postgresql_replica_check_result(result, True) - - instance.query("DETACH TABLE test.postgresql_replica") - instance.query("ATTACH TABLE test.postgresql_replica") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.5) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -@pytest.mark.timeout(320) -def test_replicating_insert_queries(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 10: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 10 + number, 10 + number from numbers(10)" - ) - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 20 + number, 20 + number from numbers(10)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 30: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 30 + number, 30 + number from numbers(10)" - ) - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 40 + number, 40 + number from numbers(10)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 50: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -@pytest.mark.timeout(320) -def test_replicating_delete_queries(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.2) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 100: - time.sleep(0.5) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - cursor.execute("DELETE FROM postgresql_replica WHERE key > 49;") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.5) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - - -@pytest.mark.timeout(320) -def test_replicating_update_queries(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 50: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - cursor.execute("UPDATE postgresql_replica SET value = value - 10;") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.5) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - - -@pytest.mark.timeout(320) -def test_resume_from_written_version(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number + 10 from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 50: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT 50 + number, 50 + number from numbers(50)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 100: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - instance.query("DETACH TABLE test.postgresql_replica") - - cursor.execute("DELETE FROM postgresql_replica WHERE key > 49;") - cursor.execute("UPDATE postgresql_replica SET value = value - 10;") - - instance.query("ATTACH TABLE test.postgresql_replica") - - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - while postgresql_replica_check_result(result) == False: - time.sleep(0.5) - result = instance.query("SELECT * FROM test.postgresql_replica ORDER BY key;") - - cursor.execute("DROP TABLE postgresql_replica;") - postgresql_replica_check_result(result, True) - - -@pytest.mark.timeout(320) -def test_many_replication_messages(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 100000: - time.sleep(0.2) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - print("SYNC OK") - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000, 100000)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 200000: - time.sleep(1) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - print("INSERT OK") - - result = instance.query("SELECT key FROM test.postgresql_replica ORDER BY key;") - expected = instance.query("SELECT number from numbers(200000)") - assert result == expected - - cursor.execute("UPDATE postgresql_replica SET value = key + 1 WHERE key < 100000;") - - result = instance.query( - "SELECT key FROM test.postgresql_replica WHERE value = key + 1 ORDER BY key;" - ) - expected = instance.query("SELECT number from numbers(100000)") - - while result != expected: - time.sleep(1) - result = instance.query( - "SELECT key FROM test.postgresql_replica WHERE value = key + 1 ORDER BY key;" - ) - print("UPDATE OK") - - cursor.execute("DELETE FROM postgresql_replica WHERE key % 2 = 1;") - cursor.execute("DELETE FROM postgresql_replica WHERE key != value;") - - result = instance.query( - "SELECT count() FROM (SELECT * FROM test.postgresql_replica);" - ) - while int(result) != 50000: - time.sleep(1) - result = instance.query( - "SELECT count() FROM (SELECT * FROM test.postgresql_replica);" - ) - print("DELETE OK") - - cursor.execute("DROP TABLE postgresql_replica;") - - -@pytest.mark.timeout(320) -def test_connection_loss(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - i = 50 - while i < 100000: - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT {} + number, number from numbers(10000)".format( - i - ) - ) - i += 10000 - - started_cluster.pause_container("postgres1") - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - print(int(result)) - time.sleep(6) - - started_cluster.unpause_container("postgres1") - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) < 100050: - time.sleep(1) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - cursor.execute("DROP TABLE postgresql_replica;") - assert int(result) == 100050 - - -@pytest.mark.timeout(320) -def test_clickhouse_restart(started_cluster): - pytest.skip("Temporary disabled (FIXME)") - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(50)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - i = 50 - while i < 100000: - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT {} + number, number from numbers(10000)".format( - i - ) - ) - i += 10000 - - instance.restart_clickhouse() - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) < 100050: - time.sleep(1) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - cursor.execute("DROP TABLE postgresql_replica;") - print(result) - assert int(result) == 100050 - - -def test_rename_table(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(25)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 25: - time.sleep(0.5) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - instance.query( - "RENAME TABLE test.postgresql_replica TO test.postgresql_replica_renamed" - ) - assert ( - int(instance.query("SELECT count() FROM test.postgresql_replica_renamed;")) - == 25 - ) - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(25, 25)" - ) - - result = instance.query("SELECT count() FROM test.postgresql_replica_renamed;") - while int(result) != 50: - time.sleep(0.5) - result = instance.query("SELECT count() FROM test.postgresql_replica_renamed;") - - result = instance.query( - "SELECT * FROM test.postgresql_replica_renamed ORDER BY key;" - ) - postgresql_replica_check_result(result, True) - cursor.execute("DROP TABLE postgresql_replica;") - instance.query("DROP TABLE IF EXISTS test.postgresql_replica_renamed") - - -def test_virtual_columns(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(10)" - ) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - while int(result) != 10: - time.sleep(0.5) - result = instance.query("SELECT count() FROM test.postgresql_replica;") - - # just check that it works, no check with `expected` because _version is taken as LSN, which will be different each time. - result = instance.query( - "SELECT key, value, _sign, _version FROM test.postgresql_replica;" - ) - print(result) - cursor.execute("DROP TABLE postgresql_replica;") - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -def test_abrupt_connection_loss_while_heavy_replication(started_cluster): - instance.query("DROP DATABASE IF EXISTS test_database") - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - table_name = "postgresql_replica" - create_postgres_table(cursor, table_name) - - instance.query(f"DROP TABLE IF EXISTS test.{table_name}") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - - for i in range(len(queries)): - query = queries[i].format(table_name) - cursor.execute(query) - print("query {}".format(query.format(table_name))) - - started_cluster.pause_container("postgres1") - - result = instance.query("SELECT count() FROM test.postgresql_replica") - print(result) # Just debug - - started_cluster.unpause_container("postgres1") - - check_tables_are_synchronized("postgresql_replica") - - result = instance.query("SELECT count() FROM test.postgresql_replica") - print(result) # Just debug - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -def test_abrupt_server_restart_while_heavy_replication(started_cluster): - # FIXME (kssenii) temporary disabled - if instance.is_built_with_sanitizer(): - pytest.skip("Temporary disabled (FIXME)") - - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - table_name = "postgresql_replica_697" - create_postgres_table(cursor, table_name) - - instance.query(f"INSERT INTO postgres_database.{table_name} SELECT -1, 1") - instance.query(f"DROP TABLE IF EXISTS test.{table_name} SYNC") - create_materialized_table( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - table_name=table_name, - ) - - n = 1 - while int(instance.query(f"select count() from test.{table_name}")) != 1: - sleep(1) - n += 1 - if n > 10: - break - - for query in queries: - cursor.execute(query.format(table_name)) - print("query {}".format(query.format(table_name))) - - instance.restart_clickhouse() - - result = instance.query(f"SELECT count() FROM test.{table_name}") - print(result) # Just debug - - check_tables_are_synchronized(table_name) - - result = instance.query(f"SELECT count() FROM test.{table_name}") - print(result) # Just debug - instance.query(f"DROP TABLE test.{table_name} SYNC") - - -def test_drop_table_immediately(started_cluster): - conn = get_postgres_conn( - ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True, - ) - cursor = conn.cursor() - create_postgres_table(cursor, "postgresql_replica") - instance.query( - "INSERT INTO postgres_database.postgresql_replica SELECT number, number from numbers(100000)" - ) - - instance.query("DROP TABLE IF EXISTS test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - instance.query("DROP TABLE test.postgresql_replica") - create_materialized_table( - ip=started_cluster.postgres_ip, port=started_cluster.postgres_port - ) - check_tables_are_synchronized("postgresql_replica") - instance.query(f"DROP TABLE test.postgresql_replica SYNC") - - -if __name__ == "__main__": - cluster.start() - input("Cluster created, press any key to destroy...") - cluster.shutdown() diff --git a/tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference b/tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference deleted file mode 100644 index 959bb2aad74..00000000000 --- a/tests/integration/test_storage_postgresql_replica/test_postgresql_replica.reference +++ /dev/null @@ -1,50 +0,0 @@ -0 0 -1 1 -2 2 -3 3 -4 4 -5 5 -6 6 -7 7 -8 8 -9 9 -10 10 -11 11 -12 12 -13 13 -14 14 -15 15 -16 16 -17 17 -18 18 -19 19 -20 20 -21 21 -22 22 -23 23 -24 24 -25 25 -26 26 -27 27 -28 28 -29 29 -30 30 -31 31 -32 32 -33 33 -34 34 -35 35 -36 36 -37 37 -38 38 -39 39 -40 40 -41 41 -42 42 -43 43 -44 44 -45 45 -46 46 -47 47 -48 48 -49 49 diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 751279f5e5a..80d2050b394 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -511,69 +511,6 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): rabbitmq_check_result(result2, True) -@pytest.mark.skip(reason="clichouse_path with rabbitmq.proto fails to be exported") -def test_rabbitmq_protobuf(rabbitmq_cluster): - instance.query( - """ - CREATE TABLE test.rabbitmq (key UInt64, value String) - ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', - rabbitmq_exchange_name = 'pb', - rabbitmq_format = 'Protobuf', - rabbitmq_flush_interval_ms=1000, - rabbitmq_max_block_size=100, - rabbitmq_schema = 'rabbitmq.proto:KeyValueProto'; - CREATE TABLE test.view (key UInt64, value UInt64) - ENGINE = MergeTree() - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.rabbitmq; - """ - ) - - credentials = pika.PlainCredentials("root", "clickhouse") - parameters = pika.ConnectionParameters( - rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, "/", credentials - ) - connection = pika.BlockingConnection(parameters) - channel = connection.channel() - - data = "" - for i in range(0, 20): - msg = rabbitmq_pb2.KeyValueProto() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - channel.basic_publish(exchange="pb", routing_key="", body=data) - data = "" - for i in range(20, 21): - msg = rabbitmq_pb2.KeyValueProto() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - channel.basic_publish(exchange="pb", routing_key="", body=data) - data = "" - for i in range(21, 50): - msg = rabbitmq_pb2.KeyValueProto() - msg.key = i - msg.value = str(i) - serialized_msg = msg.SerializeToString() - data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - channel.basic_publish(exchange="pb", routing_key="", body=data) - - connection.close() - - result = "" - while True: - result = instance.query("SELECT * FROM test.view ORDER BY key") - if rabbitmq_check_result(result): - break - - rabbitmq_check_result(result, True) - - def test_rabbitmq_big_message(rabbitmq_cluster): # Create batchs of messages of size ~100Kb rabbitmq_messages = 1000 diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index a2f28e21666..c1c076277bb 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1529,106 +1529,6 @@ def test_concurrent_alter_with_ttl_move(started_cluster, name, engine): node1.query("DROP TABLE IF EXISTS {name} SYNC".format(name=name)) -@pytest.mark.skip(reason="Flacky test") -@pytest.mark.parametrize( - "name,positive", - [ - pytest.param("test_double_move_while_select_negative", 0, id="negative"), - pytest.param("test_double_move_while_select_positive", 1, id="positive"), - ], -) -def test_double_move_while_select(started_cluster, name, positive): - name = unique_table_name(name) - - try: - node1.query( - """ - CREATE TABLE {name} ( - n Int64, - s String - ) ENGINE = MergeTree - ORDER BY tuple() - PARTITION BY n - SETTINGS storage_policy='small_jbod_with_external',temporary_directories_lifetime=1 - """.format( - name=name - ) - ) - - node1.query( - "INSERT INTO {name} VALUES (1, randomPrintableASCII(10*1024*1024))".format( - name=name - ) - ) - - parts = node1.query( - "SELECT name FROM system.parts WHERE table = '{name}' AND active = 1".format( - name=name - ) - ).splitlines() - assert len(parts) == 1 - - node1.query( - "ALTER TABLE {name} MOVE PART '{part}' TO DISK 'external'".format( - name=name, part=parts[0] - ) - ) - - def long_select(): - if positive: - node1.query( - "SELECT sleep(3), sleep(2), sleep(1), n FROM {name}".format( - name=name - ) - ) - - thread = threading.Thread(target=long_select) - thread.start() - - time.sleep(1) - - node1.query( - "ALTER TABLE {name} MOVE PART '{part}' TO DISK 'jbod1'".format( - name=name, part=parts[0] - ) - ) - - # Fill jbod1 to force ClickHouse to make move of partition 1 to external. - node1.query( - "INSERT INTO {name} VALUES (2, randomPrintableASCII(9*1024*1024))".format( - name=name - ) - ) - node1.query( - "INSERT INTO {name} VALUES (3, randomPrintableASCII(9*1024*1024))".format( - name=name - ) - ) - node1.query( - "INSERT INTO {name} VALUES (4, randomPrintableASCII(9*1024*1024))".format( - name=name - ) - ) - - wait_parts_mover(node1, name, retry_count=40) - - # If SELECT locked old part on external, move shall fail. - assert node1.query( - "SELECT disk_name FROM system.parts WHERE table = '{name}' AND active = 1 AND name = '{part}'".format( - name=name, part=parts[0] - ) - ).splitlines() == ["jbod1" if positive else "external"] - - thread.join() - - assert node1.query( - "SELECT n FROM {name} ORDER BY n".format(name=name) - ).splitlines() == ["1", "2", "3", "4"] - - finally: - node1.query("DROP TABLE IF EXISTS {name} SYNC".format(name=name)) - - @pytest.mark.parametrize( "name,engine,positive", [ From f2c3000a93e70d3eb6994a79bfccc84c5ba275e4 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 9 Aug 2023 21:40:06 +0000 Subject: [PATCH 285/777] Improve exception in ALTER query --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++++ tests/queries/0_stateless/02834_alter_exception.reference | 0 tests/queries/0_stateless/02834_alter_exception.sql | 4 ++++ 3 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/02834_alter_exception.reference create mode 100644 tests/queries/0_stateless/02834_alter_exception.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6179c70ca57..8938a3b6fbd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3166,6 +3166,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context } } + if (command.type == AlterCommand::MODIFY_QUERY) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "ALTER MODIFY QUERY is not supported by MergeTree engines family"); + if (command.type == AlterCommand::MODIFY_ORDER_BY && !is_custom_partitioned) { throw Exception(ErrorCodes::BAD_ARGUMENTS, diff --git a/tests/queries/0_stateless/02834_alter_exception.reference b/tests/queries/0_stateless/02834_alter_exception.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02834_alter_exception.sql b/tests/queries/0_stateless/02834_alter_exception.sql new file mode 100644 index 00000000000..d42f40fcbf7 --- /dev/null +++ b/tests/queries/0_stateless/02834_alter_exception.sql @@ -0,0 +1,4 @@ +DROP TABLE IF EXISTS alter_02834; +CREATE TABLE alter_02834 (a UInt64) ENGINE=MergeTree() ORDER BY a; +ALTER TABLE alter_02834 MODIFY QUERY SELECT a FROM alter_02834; -- { serverError NOT_IMPLEMENTED } +DROP TABLE alter_02834; From ce30b93b34348700351e0d462edb5baa6ddb5685 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 9 Aug 2023 23:46:45 +0200 Subject: [PATCH 286/777] Move repeated code for ClickHouseHelper to _insert_post --- tests/ci/clickhouse_helper.py | 63 ++++++++--------------------------- 1 file changed, 14 insertions(+), 49 deletions(-) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py index dbf26288469..b165cda37e8 100644 --- a/tests/ci/clickhouse_helper.py +++ b/tests/ci/clickhouse_helper.py @@ -47,45 +47,9 @@ class ClickHouseHelper: params[k] = v with open(file, "rb") as data_fd: - for i in range(5): - try: - response = requests.post( - url, params=params, data=data_fd, headers=auth - ) - except Exception as e: - error = f"Received exception while sending data to {url} on {i} attempt: {e}" - logging.warning(error) - continue - - logging.info("Response content '%s'", response.content) - - if response.ok: - break - - error = ( - "Cannot insert data into clickhouse at try " - + str(i) - + ": HTTP code " - + str(response.status_code) - + ": '" - + str(response.text) - + "'" - ) - - if response.status_code >= 500: - # A retriable error - time.sleep(1) - continue - - logging.info( - "Request headers '%s', body '%s'", - response.request.headers, - response.request.body, - ) - - raise InsertException(error) - else: - raise InsertException(error) + ClickHouseHelper._insert_post( + url, params=params, data=data_fd, headers=auth + ) @staticmethod def insert_json_str(url, auth, db, table, json_str): @@ -95,12 +59,18 @@ class ClickHouseHelper: "date_time_input_format": "best_effort", "send_logs_level": "warning", } + ClickHouseHelper._insert_post(url, params=params, data=json_str, headers=auth) + + @staticmethod + def _insert_post(*args, **kwargs): + url = "" + if args: + url = args[0] + url = kwargs.get("url", url) for i in range(5): try: - response = requests.post( - url, params=params, data=json_str, headers=auth - ) + response = requests.post(*args, **kwargs) except Exception as e: error = f"Received exception while sending data to {url} on {i} attempt: {e}" logging.warning(error) @@ -112,13 +82,8 @@ class ClickHouseHelper: break error = ( - "Cannot insert data into clickhouse at try " - + str(i) - + ": HTTP code " - + str(response.status_code) - + ": '" - + str(response.text) - + "'" + f"Cannot insert data into clickhouse at try {i}: HTTP code " + f"{response.status_code}: '{response.text}'" ) if response.status_code >= 500: From 8c63088a31d2b1d829a356f810392f9cf1484e93 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 9 Aug 2023 21:54:58 +0000 Subject: [PATCH 287/777] Fixes --- .../02415_all_new_functions_must_be_documented.reference | 1 + utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 61a2e4e9f02..231f268ba57 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -881,6 +881,7 @@ tumble tumbleEnd tumbleStart tuple +tupleConcat tupleDivide tupleDivideByNumber tupleElement diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a314815e2c4..1a7e8e16bd1 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2421,6 +2421,7 @@ tsv tui tumbleEnd tumbleStart +tupleConcat tupleDivide tupleDivideByNumber tupleElement From a79f2f140d25657a248f0d96bbe0241065a4260d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 00:19:24 +0200 Subject: [PATCH 288/777] Fix bad test `00417_kill_query` --- .../0_stateless/00417_kill_query.reference | 4 +-- tests/queries/0_stateless/00417_kill_query.sh | 25 ++++++++++++------- 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/00417_kill_query.reference b/tests/queries/0_stateless/00417_kill_query.reference index 1a3b47964c0..0dd1d78ac82 100644 --- a/tests/queries/0_stateless/00417_kill_query.reference +++ b/tests/queries/0_stateless/00417_kill_query.reference @@ -1,2 +1,2 @@ -SELECT sleep(1) FROM system.numbers LIMIT 30 -SELECT sleep(1) FROM system.numbers LIMIT 31 +SELECT sleep(1) FROM system.numbers LIMIT 300 +SELECT sleep(1) FROM system.numbers LIMIT 301 diff --git a/tests/queries/0_stateless/00417_kill_query.sh b/tests/queries/0_stateless/00417_kill_query.sh index cd5b788a147..e64e93fd4a0 100755 --- a/tests/queries/0_stateless/00417_kill_query.sh +++ b/tests/queries/0_stateless/00417_kill_query.sh @@ -7,19 +7,26 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -QUERY_FIELND_NUM=4 +QUERY_FIELD_NUM=4 -$CLICKHOUSE_CLIENT --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 30" &>/dev/null & -sleep 1 -$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE current_database = '${CLICKHOUSE_DATABASE}' and query LIKE 'SELECT sleep(%' AND (elapsed >= 0.) SYNC" | cut -f $QUERY_FIELND_NUM +$CLICKHOUSE_CLIENT --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 300" &>/dev/null & -# 31 is for the query to be different from the previous one -$CLICKHOUSE_CLIENT --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 31" &>/dev/null & -sleep 1 -$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE current_database = '${CLICKHOUSE_DATABASE}' and query = 'SELECT sleep(1) FROM system.numbers LIMIT 31' ASYNC" | cut -f $QUERY_FIELND_NUM +while true +do + $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE current_database = '${CLICKHOUSE_DATABASE}' AND query LIKE 'SELECT sleep(%' AND (elapsed >= 0.) SYNC" | cut -f $QUERY_FIELD_NUM | grep '.' && break + sleep 0.1 +done + +# 31 is for the query to be different from the previous one +$CLICKHOUSE_CLIENT --max_block_size=1 -q "SELECT sleep(1) FROM system.numbers LIMIT 301" &>/dev/null & + +while true +do + $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE current_database = '${CLICKHOUSE_DATABASE}' AND query = 'SELECT sleep(1) FROM system.numbers LIMIT 301' ASYNC" | cut -f $QUERY_FIELD_NUM | grep '.' && break + sleep 0.1 +done $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 ASYNC" $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 FORMAT TabSeparated" $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 SYNC FORMAT TabSeparated" $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 1 TEST" &>/dev/null - From 2802508c07917dc0b1d9b2d9ba91752c21a59cdf Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 9 Aug 2023 22:27:49 +0000 Subject: [PATCH 289/777] different linker --- cmake/linux/toolchain-s390x.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index f5e1cf660c9..349d7fc83a6 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -23,7 +23,6 @@ set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") -set (CMAKE_STATIC_LINKER_FLAGS "${CMAKE_STATIC_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From 087cc5bfed6736fc31f29e20a16f8161e40018c6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 9 Aug 2023 23:18:27 +0000 Subject: [PATCH 290/777] different linker --- cmake/linux/toolchain-s390x.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 349d7fc83a6..188d9ed75b7 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,9 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") -set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=s390x-linux-gnu-ld") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=gold") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=gold") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=gold") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From 65d3778b031dab996556e7e95597d7802daf3432 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 8 Aug 2023 20:15:04 +0000 Subject: [PATCH 291/777] Fix warning in test_replicated_database --- tests/integration/test_replicated_database/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 1235f7d34df..d0a04f40b69 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1226,7 +1226,7 @@ def test_force_synchronous_settings(started_cluster): def select_func(): dummy_node.query( - "SELECT sleepEachRow(1) FROM test_force_synchronous_settings.t" + "SELECT sleepEachRow(1) FROM test_force_synchronous_settings.t SETTINGS function_sleep_max_microseconds_per_block = 0" ) select_thread = threading.Thread(target=select_func) From eb61074e070587edb048fb02f26852dd3a2fd398 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 02:37:23 +0200 Subject: [PATCH 292/777] Fix test `02428_delete_with_settings` --- tests/queries/0_stateless/02428_delete_with_settings.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02428_delete_with_settings.sql b/tests/queries/0_stateless/02428_delete_with_settings.sql index 071a3f74184..618c08608fc 100644 --- a/tests/queries/0_stateless/02428_delete_with_settings.sql +++ b/tests/queries/0_stateless/02428_delete_with_settings.sql @@ -1,5 +1,5 @@ drop table if exists test; -create table test (id Int32, key String) engine=MergeTree() order by tuple(); +create table test (id Int32, key String) engine=MergeTree() order by tuple() settings index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into test select number, toString(number) from numbers(1000000); delete from test where id % 2 = 0 SETTINGS mutations_sync=0; select count() from test; From a9b760a15bf284dbe20882ddc696bdf28efed061 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Aug 2023 00:40:06 +0000 Subject: [PATCH 293/777] different linker --- cmake/linux/toolchain-s390x.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 188d9ed75b7..23d14d6ed26 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,9 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=gold") -set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=gold") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=gold") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=bfd") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=bfd") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=bfd") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From c071f6755d70a5ce9f1924fc44a905cc3122da5d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:10:03 +0300 Subject: [PATCH 294/777] Revert "Added new tests for session_log and fixed the inconsistency between login and logout." --- src/Interpreters/Session.cpp | 44 +-- src/Interpreters/Session.h | 2 - src/Interpreters/SessionLog.cpp | 14 +- src/Interpreters/SessionLog.h | 9 +- src/Server/HTTPHandler.cpp | 3 +- .../test.py | 5 +- tests/integration/test_session_log/.gitignore | 1 - .../integration/test_session_log/__init__.py | 0 .../test_session_log/configs/log.xml | 9 - .../test_session_log/configs/ports.xml | 9 - .../test_session_log/configs/session_log.xml | 9 - .../test_session_log/configs/users.xml | 23 -- .../protos/clickhouse_grpc.proto | 1 - tests/integration/test_session_log/test.py | 289 ------------------ .../02833_concurrrent_sessions.reference | 34 --- .../0_stateless/02833_concurrrent_sessions.sh | 138 --------- .../02834_remote_session_log.reference | 13 - .../0_stateless/02834_remote_session_log.sh | 56 ---- .../02835_drop_user_during_session.reference | 8 - .../02835_drop_user_during_session.sh | 113 ------- 20 files changed, 27 insertions(+), 753 deletions(-) delete mode 100644 tests/integration/test_session_log/.gitignore delete mode 100644 tests/integration/test_session_log/__init__.py delete mode 100644 tests/integration/test_session_log/configs/log.xml delete mode 100644 tests/integration/test_session_log/configs/ports.xml delete mode 100644 tests/integration/test_session_log/configs/session_log.xml delete mode 100644 tests/integration/test_session_log/configs/users.xml delete mode 120000 tests/integration/test_session_log/protos/clickhouse_grpc.proto delete mode 100644 tests/integration/test_session_log/test.py delete mode 100644 tests/queries/0_stateless/02833_concurrrent_sessions.reference delete mode 100755 tests/queries/0_stateless/02833_concurrrent_sessions.sh delete mode 100644 tests/queries/0_stateless/02834_remote_session_log.reference delete mode 100755 tests/queries/0_stateless/02834_remote_session_log.sh delete mode 100644 tests/queries/0_stateless/02835_drop_user_during_session.reference delete mode 100755 tests/queries/0_stateless/02835_drop_user_during_session.sh diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 0a6435cff75..de2a779b740 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -520,8 +520,6 @@ ContextMutablePtr Session::makeSessionContext() {}, session_context->getSettingsRef().max_sessions_for_user); - recordLoginSucess(session_context); - return session_context; } @@ -584,8 +582,6 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: { session_name_ }, max_sessions_for_user); - recordLoginSucess(session_context); - return session_context; } @@ -659,38 +655,24 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t if (user_id) user = query_context->getUser(); - /// Interserver does not create session context - recordLoginSucess(query_context); + if (!notified_session_log_about_login) + { + if (auto session_log = getSessionLog()) + { + session_log->addLoginSuccess( + auth_id, + named_session ? std::optional(named_session->key.second) : std::nullopt, + *query_context, + user); + + notified_session_log_about_login = true; + } + } return query_context; } -void Session::recordLoginSucess(ContextPtr login_context) const -{ - if (notified_session_log_about_login) - return; - - if (!login_context) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Session or query context must be created"); - - if (auto session_log = getSessionLog()) - { - const auto & settings = login_context->getSettingsRef(); - const auto access = login_context->getAccess(); - - session_log->addLoginSuccess(auth_id, - named_session ? named_session->key.second : "", - settings, - access, - getClientInfo(), - user); - - notified_session_log_about_login = true; - } -} - - void Session::releaseSessionID() { if (!named_session) diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 81ef987b428..51c0e3c71fa 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -97,8 +97,6 @@ public: private: std::shared_ptr getSessionLog() const; ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; - void recordLoginSucess(ContextPtr login_context) const; - mutable bool notified_session_log_about_login = false; const UUID auth_id; diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index b847eaf9824..0a8a7fc18c5 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -199,13 +199,12 @@ void SessionLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(auth_failure_reason.data(), auth_failure_reason.length()); } -void SessionLog::addLoginSuccess(const UUID & auth_id, - const String & session_id, - const Settings & settings, - const ContextAccessPtr & access, - const ClientInfo & client_info, - const UserPtr & login_user) +void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user) { + const auto access = login_context.getAccess(); + const auto & settings = login_context.getSettingsRef(); + const auto & client_info = login_context.getClientInfo(); + DB::SessionLogElement log_entry(auth_id, SESSION_LOGIN_SUCCESS); log_entry.client_info = client_info; @@ -216,7 +215,8 @@ void SessionLog::addLoginSuccess(const UUID & auth_id, } log_entry.external_auth_server = login_user ? login_user->auth_data.getLDAPServerName() : ""; - log_entry.session_id = session_id; + if (session_id) + log_entry.session_id = *session_id; if (const auto roles_info = access->getRolesInfo()) log_entry.roles = roles_info->getCurrentRolesNames(); diff --git a/src/Interpreters/SessionLog.h b/src/Interpreters/SessionLog.h index 8757bc12270..1282ac09c4d 100644 --- a/src/Interpreters/SessionLog.h +++ b/src/Interpreters/SessionLog.h @@ -20,7 +20,6 @@ enum SessionLogElementType : int8_t class ContextAccess; struct User; using UserPtr = std::shared_ptr; -using ContextAccessPtr = std::shared_ptr; /** A struct which will be inserted as row into session_log table. * @@ -73,13 +72,7 @@ class SessionLog : public SystemLog using SystemLog::SystemLog; public: - void addLoginSuccess(const UUID & auth_id, - const String & session_id, - const Settings & settings, - const ContextAccessPtr & access, - const ClientInfo & client_info, - const UserPtr & login_user); - + void addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user); void addLoginFailure(const UUID & auth_id, const ClientInfo & info, const std::optional & user, const Exception & reason); void addLogOut(const UUID & auth_id, const UserPtr & login_user, const ClientInfo & client_info); }; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 10d5e7a0242..ff5690a3b07 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -561,7 +561,8 @@ void HTTPHandler::processQuery( session->makeSessionContext(); } - auto context = session->makeQueryContext(); + auto client_info = session->getClientInfo(); + auto context = session->makeQueryContext(std::move(client_info)); /// This parameter is used to tune the behavior of output formats (such as Native) for compatibility. if (params.has("client_protocol_version")) diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 65587933fed..2930262f63e 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -27,7 +27,10 @@ proto_dir = os.path.join(SCRIPT_DIR, "./protos") gen_dir = os.path.join(SCRIPT_DIR, "./_gen") os.makedirs(gen_dir, exist_ok=True) run_and_check( - f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", + "python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \ + {proto_dir}/clickhouse_grpc.proto".format( + proto_dir=proto_dir, gen_dir=gen_dir + ), shell=True, ) diff --git a/tests/integration/test_session_log/.gitignore b/tests/integration/test_session_log/.gitignore deleted file mode 100644 index edf565ec632..00000000000 --- a/tests/integration/test_session_log/.gitignore +++ /dev/null @@ -1 +0,0 @@ -_gen diff --git a/tests/integration/test_session_log/__init__.py b/tests/integration/test_session_log/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_session_log/configs/log.xml b/tests/integration/test_session_log/configs/log.xml deleted file mode 100644 index 7a079b81e69..00000000000 --- a/tests/integration/test_session_log/configs/log.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - trace - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log - 1000M - 10 - - \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/ports.xml b/tests/integration/test_session_log/configs/ports.xml deleted file mode 100644 index fbaefc16b3a..00000000000 --- a/tests/integration/test_session_log/configs/ports.xml +++ /dev/null @@ -1,9 +0,0 @@ - - 5433 - 9001 - 9100 - - - false - - \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/session_log.xml b/tests/integration/test_session_log/configs/session_log.xml deleted file mode 100644 index a0e4e3e2216..00000000000 --- a/tests/integration/test_session_log/configs/session_log.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - system - session_log
- - toYYYYMM(event_date) - 7500 -
-
diff --git a/tests/integration/test_session_log/configs/users.xml b/tests/integration/test_session_log/configs/users.xml deleted file mode 100644 index 0416dfadc8a..00000000000 --- a/tests/integration/test_session_log/configs/users.xml +++ /dev/null @@ -1,23 +0,0 @@ - - - - 0 - - - - - - - pass - - - pass - - - pass - - - pass - - - \ No newline at end of file diff --git a/tests/integration/test_session_log/protos/clickhouse_grpc.proto b/tests/integration/test_session_log/protos/clickhouse_grpc.proto deleted file mode 120000 index 25d15f11e3b..00000000000 --- a/tests/integration/test_session_log/protos/clickhouse_grpc.proto +++ /dev/null @@ -1 +0,0 @@ -../../../../src/Server/grpc_protos/clickhouse_grpc.proto \ No newline at end of file diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py deleted file mode 100644 index dbb39993ce3..00000000000 --- a/tests/integration/test_session_log/test.py +++ /dev/null @@ -1,289 +0,0 @@ -import os - -import grpc -import pymysql.connections -import psycopg2 as py_psql -import pytest -import random -import sys -import threading - -from helpers.cluster import ClickHouseCluster, run_and_check - -POSTGRES_SERVER_PORT = 5433 -MYSQL_SERVER_PORT = 9001 -GRPC_PORT = 9100 -SESSION_LOG_MATCHING_FIELDS = "auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -DEFAULT_ENCODING = "utf-8" - -# Use grpcio-tools to generate *pb2.py files from *.proto. -proto_dir = os.path.join(SCRIPT_DIR, "./protos") -gen_dir = os.path.join(SCRIPT_DIR, "./_gen") -os.makedirs(gen_dir, exist_ok=True) -run_and_check( - f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", - shell=True, -) - -sys.path.append(gen_dir) - -import clickhouse_grpc_pb2 -import clickhouse_grpc_pb2_grpc - -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "node", - main_configs=[ - "configs/ports.xml", - "configs/log.xml", - "configs/session_log.xml", - ], - user_configs=["configs/users.xml"], - # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 - env_variables={ - "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="") - }, -) - - -def grpc_get_url(): - return f"{instance.ip_address}:{GRPC_PORT}" - - -def grpc_create_insecure_channel(): - channel = grpc.insecure_channel(grpc_get_url()) - grpc.channel_ready_future(channel).result(timeout=2) - return channel - - -session_id_counter = 0 - - -def next_session_id(): - global session_id_counter - session_id = session_id_counter - session_id_counter += 1 - return str(session_id) - - -def grpc_query(query, user_, pass_, raise_exception): - try: - query_info = clickhouse_grpc_pb2.QueryInfo( - query=query, - session_id=next_session_id(), - user_name=user_, - password=pass_, - ) - channel = grpc_create_insecure_channel() - stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) - result = stub.ExecuteQuery(query_info) - if result and result.HasField("exception"): - raise Exception(result.exception.display_text) - - return result.output.decode(DEFAULT_ENCODING) - except Exception: - assert raise_exception - - -def postgres_query(query, user_, pass_, raise_exception): - try: - client = py_psql.connect( - host=instance.ip_address, - port=POSTGRES_SERVER_PORT, - user=user_, - password=pass_, - database="default", - ) - cursor = client.cursor() - cursor.execute(query) - cursor.fetchall() - except Exception: - assert raise_exception - - -def mysql_query(query, user_, pass_, raise_exception): - try: - client = pymysql.connections.Connection( - host=instance.ip_address, - user=user_, - password=pass_, - database="default", - port=MYSQL_SERVER_PORT, - ) - cursor = client.cursor(pymysql.cursors.DictCursor) - if raise_exception: - with pytest.raises(Exception): - cursor.execute(query) - else: - cursor.execute(query) - cursor.fetchall() - except Exception: - assert raise_exception - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_grpc_session(started_cluster): - grpc_query("SELECT 1", "grpc_user", "pass", False) - grpc_query("SELECT 2", "grpc_user", "wrong_pass", True) - grpc_query("SELECT 3", "wrong_grpc_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "grpc_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'Logout'" - ) - assert logout_records == "grpc_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "grpc_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_mysql_session(started_cluster): - mysql_query("SELECT 1", "mysql_user", "pass", False) - mysql_query("SELECT 2", "mysql_user", "wrong_pass", True) - mysql_query("SELECT 3", "wrong_mysql_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "mysql_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'Logout'" - ) - assert logout_records == "mysql_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "mysql_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_postgres_session(started_cluster): - postgres_query("SELECT 1", "postgres_user", "pass", False) - postgres_query("SELECT 2", "postgres_user", "wrong_pass", True) - postgres_query("SELECT 3", "wrong_postgres_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "postgres_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'Logout'" - ) - assert logout_records == "postgres_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "postgres_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_parallel_sessions(started_cluster): - thread_list = [] - for _ in range(10): - # Sleep time does not significantly matter here, - # test should pass even without sleeping. - for function in [postgres_query, grpc_query, mysql_query]: - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "parallel_user", - "pass", - False, - ), - ) - thread.start() - thread_list.append(thread) - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "parallel_user", - "wrong_pass", - True, - ), - ) - thread.start() - thread_list.append(thread) - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "wrong_parallel_user", - "pass", - True, - ), - ) - thread.start() - thread_list.append(thread) - - for thread in thread_list: - thread.join() - - instance.query("SYSTEM FLUSH LOGS") - port_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user'" - ) - assert port_0_sessions == "90\n" - - port_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_port = 0" - ) - assert port_0_sessions == "0\n" - - address_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_address = toIPv6('::')" - ) - assert address_0_sessions == "0\n" - - grpc_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'gRPC'" - ) - assert grpc_sessions == "30\n" - - mysql_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'MySQL'" - ) - assert mysql_sessions == "30\n" - - postgres_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'PostgreSQL'" - ) - assert postgres_sessions == "30\n" - - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "30\n" - - logout_failure_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'" - ) - assert logout_failure_sessions == "30\n" diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.reference b/tests/queries/0_stateless/02833_concurrrent_sessions.reference deleted file mode 100644 index bfe507e8eac..00000000000 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.reference +++ /dev/null @@ -1,34 +0,0 @@ -sessions: -150 -port_0_sessions: -0 -address_0_sessions: -0 -tcp_sessions -60 -http_sessions -30 -http_with_session_id_sessions -30 -my_sql_sessions -30 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh deleted file mode 100755 index d1d571c6985..00000000000 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.sh +++ /dev/null @@ -1,138 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest, long - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ - -# Each user uses a separate thread. -readonly TCP_USERS=( "02833_TCP_USER_${PID}"_{1,2} ) # 2 concurrent TCP users -readonly HTTP_USERS=( "02833_HTTP_USER_${PID}" ) -readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "02833_HTTP_WITH_SESSION_ID_USER_${PID}" ) -readonly MYSQL_USERS=( "02833_MYSQL_USER_${PID}") -readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) - -readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" - -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -for user in "${ALL_USERS[@]}"; do - ${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${user} IDENTIFIED WITH plaintext_password BY 'pass'" - ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${user}" - ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${user}"; -done - -# All _session functions execute in separate threads. -# These functions try to create a session with successful login and logout. -# Sleep a small, random amount of time to make concurrency more intense. -# and try to login with an invalid password. -function tcp_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM})" --user="${user}" --password="pass" - # login failure - ${CLICKHOUSE_CLIENT} -q "SELECT 2" --user="${user}" --password 'invalid' - done -} - -function http_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep(0.01${RANDOM})" - - # login failure - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=wrong" -d "SELECT 4" - done -} - -function http_with_session_id_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.01${RANDOM}" - - # login failure - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=wrong" -d "SELECT 6" - done -} - -function mysql_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" - - # login failure - ${CLICKHOUSE_CLIENT} -q "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'wrong', SETTINGS connection_max_tries=1)" - done -} - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" - -export -f tcp_session; -export -f http_session; -export -f http_with_session_id_session; -export -f mysql_session; - -for user in "${TCP_USERS[@]}"; do - timeout 60s bash -c "tcp_session ${user}" >/dev/null 2>&1 & -done - -for user in "${HTTP_USERS[@]}"; do - timeout 60s bash -c "http_session ${user}" >/dev/null 2>&1 & -done - -for user in "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}"; do - timeout 60s bash -c "http_with_session_id_session ${user}" >/dev/null 2>&1 & -done - -for user in "${MYSQL_USERS[@]}"; do - timeout 60s bash -c "mysql_session ${user}" >/dev/null 2>&1 & -done - -wait - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" - -echo "port_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_port = 0" - -echo "address_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_address = toIPv6('::')" - -echo "tcp_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${TCP_USERS_SQL_COLLECTION_STRING}) AND interface = 'TCP'" -echo "http_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" -echo "http_with_session_id_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" -echo "my_sql_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${MYSQL_USERS_SQL_COLLECTION_STRING}) AND interface = 'MySQL'" - -for user in "${ALL_USERS[@]}"; do - ${CLICKHOUSE_CLIENT} -q "DROP USER ${user}" - echo "Corresponding LoginSuccess/Logout" - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'Logout')" - echo "LoginFailure" - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" - done diff --git a/tests/queries/0_stateless/02834_remote_session_log.reference b/tests/queries/0_stateless/02834_remote_session_log.reference deleted file mode 100644 index e2680982ab0..00000000000 --- a/tests/queries/0_stateless/02834_remote_session_log.reference +++ /dev/null @@ -1,13 +0,0 @@ -0 -0 -0 -0 -client_port 0 connections: -0 -client_address '::' connections: -0 -login failures: -0 -TCP Login and logout count is equal -HTTP Login and logout count is equal -MySQL Login and logout count is equal diff --git a/tests/queries/0_stateless/02834_remote_session_log.sh b/tests/queries/0_stateless/02834_remote_session_log.sh deleted file mode 100755 index 64f20a70cb1..00000000000 --- a/tests/queries/0_stateless/02834_remote_session_log.sh +++ /dev/null @@ -1,56 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ -readonly TEST_USER=$"02834_USER_${PID}" -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER} IDENTIFIED WITH plaintext_password BY 'pass'" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT CREATE TEMPORARY TABLE, MYSQL, REMOTE ON *.* TO ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" - -${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ - -d "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" - -${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ - -d "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" - -${CLICKHOUSE_CLIENT} -q "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "client_port 0 connections:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_port = 0" - -echo "client_address '::' connections:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_address = toIPv6('::')" - -echo "login failures:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and type = 'LoginFailure'" - -# remote(...) function sometimes reuses old cached sessions for query execution. -# This makes LoginSuccess/Logout entries count unstable, but success and logouts must always match. - -for interface in 'TCP' 'HTTP' 'MySQL' -do - LOGIN_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}'"` - CORRESPONDING_LOGOUT_RECORDS_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}')"` - - if [ "$LOGIN_COUNT" == "$CORRESPONDING_LOGOUT_RECORDS_COUNT" ]; then - echo "${interface} Login and logout count is equal" - else - TOTAL_LOGOUT_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}'"` - echo "${interface} Login count ${LOGIN_COUNT} != corresponding logout count ${CORRESPONDING_LOGOUT_RECORDS_COUNT}. TOTAL_LOGOUT_COUNT ${TOTAL_LOGOUT_COUNT}" - fi -done - -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.reference b/tests/queries/0_stateless/02835_drop_user_during_session.reference deleted file mode 100644 index 7252faab8c6..00000000000 --- a/tests/queries/0_stateless/02835_drop_user_during_session.reference +++ /dev/null @@ -1,8 +0,0 @@ -port_0_sessions: -0 -address_0_sessions: -0 -Corresponding LoginSuccess/Logout -9 -LoginFailure -0 diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh deleted file mode 100755 index e848e1fe90d..00000000000 --- a/tests/queries/0_stateless/02835_drop_user_during_session.sh +++ /dev/null @@ -1,113 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ - -readonly TEST_USER="02835_USER_${PID}" -readonly TEST_ROLE="02835_ROLE_${PID}" -readonly TEST_PROFILE="02835_PROFILE_${PID}" -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -function tcp_session() -{ - local user=$1 - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.numbers" --user="${user}" -} - -function http_session() -{ - local user=$1 - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" -} - -function http_with_session_id_session() -{ - local user=$1 - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" -} - -# Busy-waits until user $1, specified amount of queries ($2) will run simultaneously. -function wait_for_queries_start() -{ - local user=$1 - local queries_count=$2 - # 10 seconds waiting - counter=0 retries=100 - while [[ $counter -lt $retries ]]; do - result=$($CLICKHOUSE_CLIENT --query "SELECT COUNT(*) FROM system.processes WHERE user = '${user}'") - if [[ $result == "${queries_count}" ]]; then - break; - fi - sleep 0.1 - ((++counter)) - done -} - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" - -# DROP USE CASE -${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -export -f tcp_session; -export -f http_session; -export -f http_with_session_id_session; - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -# DROP ROLE CASE -${CLICKHOUSE_CLIENT} -q "CREATE ROLE IF NOT EXISTS ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} DEFAULT ROLE ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP ROLE ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -# DROP PROFILE CASE -${CLICKHOUSE_CLIENT} -q "CREATE SETTINGS PROFILE IF NOT EXISTS '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} SETTINGS PROFILE '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP SETTINGS PROFILE '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "port_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_port = 0" -echo "address_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_address = toIPv6('::')" -echo "Corresponding LoginSuccess/Logout" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout')" -echo "LoginFailure" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" From 6edf4a777f33a84db9c18b08741207251fa5fc3d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 03:26:11 +0200 Subject: [PATCH 295/777] Remove unrecognizable garbage from the performance test --- docker/test/performance-comparison/compare.sh | 64 ++++++------------- 1 file changed, 21 insertions(+), 43 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 798d2a40b12..c0422816787 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -666,8 +666,8 @@ create view partial_query_times as select * from -- queries with new functions added in the tested PR). create table partial_queries_report engine File(TSV, 'report/partial-queries-report.tsv') settings output_format_decimal_trailing_zeros = 1 - as select toDecimal64(time_median, 3) time, - toDecimal64(time_stddev / time_median, 3) relative_time_stddev, + as select round(time_median, 3) time, + round(time_stddev / time_median, 3) relative_time_stddev, test, query_index, query_display_name from partial_query_times join query_display_names using (test, query_index) @@ -746,21 +746,21 @@ create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') -- of NaN to decimal. (left > right ? left / right : right / left) as times_change_float, isFinite(times_change_float) as times_change_finite, - toDecimal64(times_change_finite ? times_change_float : 1., 3) as times_change_decimal, + round(times_change_finite ? times_change_float : 1., 3) as times_change_decimal, times_change_finite ? (left > right ? '-' : '+') || toString(times_change_decimal) || 'x' : '--' as times_change_str select - toDecimal64(left, 3), toDecimal64(right, 3), times_change_str, - toDecimal64(diff, 3), toDecimal64(stat_threshold, 3), + round(left, 3), round(right, 3), times_change_str, + round(diff, 3), round(stat_threshold, 3), changed_fail, test, query_index, query_display_name from queries where changed_show order by abs(diff) desc; create table unstable_queries_report engine File(TSV, 'report/unstable-queries.tsv') settings output_format_decimal_trailing_zeros = 1 as select - toDecimal64(left, 3), toDecimal64(right, 3), toDecimal64(diff, 3), - toDecimal64(stat_threshold, 3), unstable_fail, test, query_index, query_display_name + round(left, 3), round(right, 3), round(diff, 3), + round(stat_threshold, 3), unstable_fail, test, query_index, query_display_name from queries where unstable_show order by stat_threshold desc; @@ -792,8 +792,8 @@ create table test_perf_changes_report engine File(TSV, 'report/test-perf-changes settings output_format_decimal_trailing_zeros = 1 as with (times_speedup >= 1 - ? '-' || toString(toDecimal64(times_speedup, 3)) || 'x' - : '+' || toString(toDecimal64(1 / times_speedup, 3)) || 'x') + ? '-' || toString(round(times_speedup, 3)) || 'x' + : '+' || toString(round(1 / times_speedup, 3)) || 'x') as times_speedup_str select test, times_speedup_str, queries, bad, changed, unstable -- Not sure what's the precedence of UNION ALL vs WHERE & ORDER BY, hence all @@ -818,10 +818,10 @@ create view total_client_time_per_query as select * create table slow_on_client_report engine File(TSV, 'report/slow-on-client.tsv') settings output_format_decimal_trailing_zeros = 1 - as select client, server, toDecimal64(client/server, 3) p, + as select client, server, round(client/server, 3) p, test, query_display_name from total_client_time_per_query left join query_display_names using (test, query_index) - where p > toDecimal64(1.02, 3) order by p desc; + where p > round(1.02, 3) order by p desc; create table wall_clock_time_per_test engine Memory as select * from file('wall-clock-times.tsv', TSV, 'test text, real float, user float, system float'); @@ -902,12 +902,12 @@ create table test_times_report engine File(TSV, 'report/test-times.tsv') settings output_format_decimal_trailing_zeros = 1 as select test, - toDecimal64(real, 3), - toDecimal64(total_client_time, 3), + round(real, 3), + round(total_client_time, 3), queries, - toDecimal64(query_max, 3), - toDecimal64(avg_real_per_query, 3), - toDecimal64(query_min, 3), + round(query_max, 3), + round(avg_real_per_query, 3), + round(query_min, 3), runs from ( select * from test_times_view @@ -926,14 +926,14 @@ create table all_tests_report engine File(TSV, 'report/all-queries.tsv') -- of NaN to decimal. (left > right ? left / right : right / left) as times_change_float, isFinite(times_change_float) as times_change_finite, - toDecimal64(times_change_finite ? times_change_float : 1., 3) as times_change_decimal, + round(times_change_finite ? times_change_float : 1., 3) as times_change_decimal, times_change_finite ? (left > right ? '-' : '+') || toString(times_change_decimal) || 'x' : '--' as times_change_str select changed_fail, unstable_fail, - toDecimal64(left, 3), toDecimal64(right, 3), times_change_str, - toDecimal64(isFinite(diff) ? diff : 0, 3), - toDecimal64(isFinite(stat_threshold) ? stat_threshold : 0, 3), + round(left, 3), round(right, 3), times_change_str, + round(isFinite(diff) ? diff : 0, 3), + round(isFinite(stat_threshold) ? stat_threshold : 0, 3), test, query_index, query_display_name from queries order by test, query_index; @@ -1044,27 +1044,6 @@ create table unstable_run_traces engine File(TSVWithNamesAndTypes, order by count() desc ; -create table metric_devation engine File(TSVWithNamesAndTypes, - 'report/metric-deviation.$version.tsv') - settings output_format_decimal_trailing_zeros = 1 - -- first goes the key used to split the file with grep - as select test, query_index, query_display_name, - toDecimal64(d, 3) d, q, metric - from ( - select - test, query_index, - (q[3] - q[1])/q[2] d, - quantilesExact(0, 0.5, 1)(value) q, metric - from (select * from unstable_run_metrics - union all select * from unstable_run_traces - union all select * from unstable_run_metrics_2) mm - group by test, query_index, metric - having isFinite(d) and d > 0.5 and q[3] > 5 - ) metrics - left join query_display_names using (test, query_index) - order by test, query_index, d desc - ; - create table stacks engine File(TSV, 'report/stacks.$version.tsv') as select -- first goes the key used to split the file with grep @@ -1175,7 +1154,7 @@ create table metrics engine File(TSV, 'metrics/metrics.tsv') as create table changes engine File(TSV, 'metrics/changes.tsv') settings output_format_decimal_trailing_zeros = 1 as select metric, left, right, - toDecimal64(diff, 3), toDecimal64(times_diff, 3) + round(diff, 3), round(times_diff, 3) from ( select metric, median(left) as left, median(right) as right, (right - left) / left diff, @@ -1226,7 +1205,6 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv') '$SHA_TO_TEST' :: LowCardinality(String) AS commit_sha, '${CLICKHOUSE_PERFORMANCE_COMPARISON_CHECK_NAME:-Performance}' :: LowCardinality(String) AS check_name, '$(sed -n 's/.*/\1/p' report.html)' :: LowCardinality(String) AS check_status, - -- TODO toDateTime() can't parse output of 'date', so no time for now. (($(date +%s) - $CHPC_CHECK_START_TIMESTAMP) * 1000) :: UInt64 AS check_duration_ms, fromUnixTimestamp($CHPC_CHECK_START_TIMESTAMP) check_start_time, test_name :: LowCardinality(String) AS test_name , From f2621e70aed39f2bf1a061b4820714f84bf7324a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 03:27:07 +0200 Subject: [PATCH 296/777] Remove unrecognizable garbage from the performance test --- docker/test/performance-comparison/compare.sh | 8 -------- 1 file changed, 8 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index c0422816787..f949e66ab17 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -665,7 +665,6 @@ create view partial_query_times as select * from -- Report for backward-incompatible ('partial') queries that we could only run on the new server (e.g. -- queries with new functions added in the tested PR). create table partial_queries_report engine File(TSV, 'report/partial-queries-report.tsv') - settings output_format_decimal_trailing_zeros = 1 as select round(time_median, 3) time, round(time_stddev / time_median, 3) relative_time_stddev, test, query_index, query_display_name @@ -739,7 +738,6 @@ create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv') ; create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') - settings output_format_decimal_trailing_zeros = 1 as with -- server_time is sometimes reported as zero (if it's less than 1 ms), -- so we have to work around this to not get an error about conversion @@ -757,7 +755,6 @@ create table changed_perf_report engine File(TSV, 'report/changed-perf.tsv') from queries where changed_show order by abs(diff) desc; create table unstable_queries_report engine File(TSV, 'report/unstable-queries.tsv') - settings output_format_decimal_trailing_zeros = 1 as select round(left, 3), round(right, 3), round(diff, 3), round(stat_threshold, 3), unstable_fail, test, query_index, query_display_name @@ -789,7 +786,6 @@ create view total_speedup as ; create table test_perf_changes_report engine File(TSV, 'report/test-perf-changes.tsv') - settings output_format_decimal_trailing_zeros = 1 as with (times_speedup >= 1 ? '-' || toString(round(times_speedup, 3)) || 'x' @@ -817,7 +813,6 @@ create view total_client_time_per_query as select * 'test text, query_index int, client float, server float'); create table slow_on_client_report engine File(TSV, 'report/slow-on-client.tsv') - settings output_format_decimal_trailing_zeros = 1 as select client, server, round(client/server, 3) p, test, query_display_name from total_client_time_per_query left join query_display_names using (test, query_index) @@ -899,7 +894,6 @@ create view test_times_view_total as ; create table test_times_report engine File(TSV, 'report/test-times.tsv') - settings output_format_decimal_trailing_zeros = 1 as select test, round(real, 3), @@ -919,7 +913,6 @@ create table test_times_report engine File(TSV, 'report/test-times.tsv') -- report for all queries page, only main metric create table all_tests_report engine File(TSV, 'report/all-queries.tsv') - settings output_format_decimal_trailing_zeros = 1 as with -- server_time is sometimes reported as zero (if it's less than 1 ms), -- so we have to work around this to not get an error about conversion @@ -1152,7 +1145,6 @@ create table metrics engine File(TSV, 'metrics/metrics.tsv') as -- Show metrics that have changed create table changes engine File(TSV, 'metrics/changes.tsv') - settings output_format_decimal_trailing_zeros = 1 as select metric, left, right, round(diff, 3), round(times_diff, 3) from ( From 3c6d140c179c29ce890f190529e29d1e9ec3db8b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 10 Aug 2023 01:31:59 +0000 Subject: [PATCH 297/777] Fix sigle quote escaping in PostgreSQL engine --- src/Storages/StoragePostgreSQL.cpp | 5 +++++ tests/integration/test_storage_postgresql/test.py | 14 ++++++++++++++ 2 files changed, 19 insertions(+) diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 3551ee36819..11558b39ad3 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -45,6 +45,7 @@ #include +#include namespace DB { @@ -123,6 +124,10 @@ Pipe StoragePostgreSQL::read( column_names_, storage_snapshot->metadata->getColumns().getOrdinary(), IdentifierQuotingStyle::DoubleQuotes, remote_table_schema, remote_table_name, context_); + + /// Single quotes in PostgreSQL are escaped through repetition + boost::replace_all(query, "\\'", "''"); + LOG_TRACE(log, "Query: {}", query); Block sample_block; diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 686eb1ea751..3a36d050f17 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -726,6 +726,20 @@ def test_auto_close_connection(started_cluster): assert count == 2 +def test_single_quotes(started_cluster): + cursor = started_cluster.postgres_conn.cursor() + cursor.execute(f"DROP TABLE IF EXISTS single_quote_fails") + cursor.execute(f"CREATE TABLE single_quote_fails(text varchar(255))") + node1.query( + "CREATE TABLE default.single_quote_fails (text String) ENGINE = PostgreSQL('postgres1:5432', 'postgres', 'single_quote_fails', 'postgres', 'mysecretpassword')" + ) + node1.query("SELECT * FROM single_quote_fails WHERE text = ''''") + node1.query("SELECT * FROM single_quote_fails WHERE text = '\\''") + node1.query("SELECT * FROM single_quote_fails WHERE text like '%a''a%'") + node1.query("SELECT * FROM single_quote_fails WHERE text like '%a\\'a%'") + cursor.execute(f"DROP TABLE single_quote_fails") + + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 4ba5273939804ed2f639cc4748eaaa7f3b9d4af2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 03:56:27 +0200 Subject: [PATCH 298/777] Disable randomization in `02273_full_sort_join` --- tests/queries/0_stateless/02273_full_sort_join.sql.j2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 6500306356c..2a13e11e116 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-random-settings DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; From 0b138fdf16eeeebef564a20928e83e6a7b1d7122 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:15:19 +0200 Subject: [PATCH 299/777] Remove outdated Dockerfile --- docker/client/Dockerfile | 34 ---------------------------------- docker/client/README.md | 7 ------- 2 files changed, 41 deletions(-) delete mode 100644 docker/client/Dockerfile delete mode 100644 docker/client/README.md diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile deleted file mode 100644 index 1c185daec75..00000000000 --- a/docker/client/Dockerfile +++ /dev/null @@ -1,34 +0,0 @@ -FROM ubuntu:18.04 - -# ARG for quick switch to a given ubuntu mirror -ARG apt_archive="http://archive.ubuntu.com" -RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list - -ARG repository="deb https://repo.clickhouse.com/deb/stable/ main/" -ARG version=22.1.1.* - -RUN apt-get update \ - && apt-get install --yes --no-install-recommends \ - apt-transport-https \ - ca-certificates \ - dirmngr \ - gnupg \ - && mkdir -p /etc/apt/sources.list.d \ - && apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 \ - && echo $repository > /etc/apt/sources.list.d/clickhouse.list \ - && apt-get update \ - && env DEBIAN_FRONTEND=noninteractive \ - apt-get install --allow-unauthenticated --yes --no-install-recommends \ - clickhouse-client=$version \ - clickhouse-common-static=$version \ - locales \ - tzdata \ - && rm -rf /var/lib/apt/lists/* /var/cache/debconf \ - && apt-get clean - -RUN locale-gen en_US.UTF-8 -ENV LANG en_US.UTF-8 -ENV LANGUAGE en_US:en -ENV LC_ALL en_US.UTF-8 - -ENTRYPOINT ["/usr/bin/clickhouse-client"] diff --git a/docker/client/README.md b/docker/client/README.md deleted file mode 100644 index bbcc7d60794..00000000000 --- a/docker/client/README.md +++ /dev/null @@ -1,7 +0,0 @@ -# ClickHouse Client Docker Image - -For more information see [ClickHouse Server Docker Image](https://hub.docker.com/r/clickhouse/clickhouse-server/). - -## License - -View [license information](https://github.com/ClickHouse/ClickHouse/blob/master/LICENSE) for the software contained in this image. From fc3970e3ff46d6795ac811d51edb29c5b6128a2d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:30:03 +0200 Subject: [PATCH 300/777] Update readme --- docker/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/README.md b/docker/README.md index ec52ddd143e..bae02201add 100644 --- a/docker/README.md +++ b/docker/README.md @@ -1,5 +1,5 @@ ## ClickHouse Dockerfiles -This directory contain Dockerfiles for `clickhouse-client` and `clickhouse-server`. They are updated in each release. +This directory contain Dockerfiles for `clickhouse-server`. They are updated in each release. Also there is bunch of images for testing and CI. They are listed in `images.json` file and updated on each commit to master. If you need to add another image, place information about it into `images.json`. From 2a89587300a408610591cf251546a6c59affc034 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:32:39 +0200 Subject: [PATCH 301/777] Address review comments --- docker/test/stateless/run.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 290222c9d26..0ef25741b18 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -109,8 +109,8 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] fi -# Wait for the server to start -while true +# Wait for the server to start, but not for too long. +for _ in {1..100} do clickhouse-client --query "SELECT 1" && break sleep 1 From 8fba4d2c4a0bf6b9f9061bf19980f98158ef6846 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:41:19 +0200 Subject: [PATCH 302/777] Protection if the repository is old --- docker/test/stateless/run.sh | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 0ef25741b18..6c930fa583b 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -117,7 +117,7 @@ do done # Initialize export of system logs to ClickHouse Cloud -if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] +if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" -a -f /repository/utils/export-logs-in-ci/setup.sh ] then export EXTRA_COLUMNS_EXPRESSION="$PULL_REQUEST_NUMBER AS pull_request_number, '$COMMIT_SHA' AS commit_sha, '$CHECK_START_TIME' AS check_start_time, '$CHECK_NAME' AS check_name, '$INSTANCE_TYPE' AS instance_type" # TODO: Check if the password will appear in the logs. @@ -131,9 +131,6 @@ then export CLICKHOUSE_CI_LOGS_PASSWORD='' fi -# TODO what is this? Remove it. -sleep 5 - attach_gdb_to_clickhouse || true # FIXME: to not break old builds, clean on 2023-09-01 function fn_exists() { From 0ba3fea7fefa1237434e1cca36b8d8ac11fb73e5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:44:20 +0200 Subject: [PATCH 303/777] Apply recommendations --- docker/test/stateless/Dockerfile | 2 ++ docker/test/stateless/run.sh | 4 ++-- docker/test/stateless/setup_export_logs.sh | 1 + 3 files changed, 5 insertions(+), 2 deletions(-) create mode 120000 docker/test/stateless/setup_export_logs.sh diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 86ca7924f74..48d26c84ed8 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -87,4 +87,6 @@ RUN npm install -g azurite \ COPY run.sh / COPY setup_minio.sh / COPY setup_hdfs_minicluster.sh / +COPY setup_export_logs.sh / + CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 6c930fa583b..015ba7d7423 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -117,13 +117,13 @@ do done # Initialize export of system logs to ClickHouse Cloud -if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" -a -f /repository/utils/export-logs-in-ci/setup.sh ] +if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] then export EXTRA_COLUMNS_EXPRESSION="$PULL_REQUEST_NUMBER AS pull_request_number, '$COMMIT_SHA' AS commit_sha, '$CHECK_START_TIME' AS check_start_time, '$CHECK_NAME' AS check_name, '$INSTANCE_TYPE' AS instance_type" # TODO: Check if the password will appear in the logs. export CONNECTION_PARAMETERS="--secure --user ci --host ${CLICKHOUSE_CI_LOGS_HOST} --password ${CLICKHOUSE_CI_LOGS_PASSWORD}" - /repository/utils/export-logs-in-ci/setup.sh + ./setup_export_logs.sh # Unset variables after use export CONNECTION_PARAMETERS='' diff --git a/docker/test/stateless/setup_export_logs.sh b/docker/test/stateless/setup_export_logs.sh new file mode 120000 index 00000000000..b6e2f981cd4 --- /dev/null +++ b/docker/test/stateless/setup_export_logs.sh @@ -0,0 +1 @@ +../../../utils/export-logs-in-ci/setup.sh \ No newline at end of file From e4fbca2e2e7e21229e2a68254570c7effeaddbf9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 04:47:44 +0200 Subject: [PATCH 304/777] Apply review recommendations --- tests/ci/functional_test_check.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index fdea5be9a6c..c84884bcfaf 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -130,7 +130,6 @@ def get_run_command( return ( f"docker run --volume={builds_path}:/package_folder " f"--volume={repo_path}/tests:/usr/share/clickhouse-test " - f"--volume={repo_path}:/repository " f"{volume_with_broken_test} " f"--volume={result_path}:/test_output --volume={server_log_path}:/var/log/clickhouse-server " f"--cap-add=SYS_PTRACE {env_str} {additional_options_str} {image}" From 931d75ec571882c86374b5a2cdfa7c4b1fe07012 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 05:05:17 +0200 Subject: [PATCH 305/777] Maybe better --- src/Core/Field.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index 89faaed8a72..de5f7ba9c92 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include namespace DB From 7a6d438b88c03004c105c614749fc7cbab69da42 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 05:11:37 +0200 Subject: [PATCH 306/777] Remove magic_enum --- src/Core/Field.cpp | 64 ++++++++++++++++++++++++---------------------- src/Core/Field.h | 2 +- 2 files changed, 34 insertions(+), 32 deletions(-) diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index de5f7ba9c92..1fcf663a744 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -9,9 +9,10 @@ #include #include #include -#include +using namespace std::literals; + namespace DB { @@ -21,12 +22,6 @@ namespace ErrorCodes extern const int DECIMAL_OVERFLOW; } -/// Keep in mind, that "magic_enum" is very expensive for compiler. -std::string_view Field::getTypeName() const -{ - return magic_enum::enum_name(which); -} - inline Field getBinaryValue(UInt8 type, ReadBuffer & buf) { switch (type) @@ -590,34 +585,41 @@ String toString(const Field & x) x); } -String fieldTypeToString(Field::Types::Which type) +std::string_view fieldTypeToString(Field::Types::Which type) { switch (type) { - case Field::Types::Which::Null: return "Null"; - case Field::Types::Which::Array: return "Array"; - case Field::Types::Which::Tuple: return "Tuple"; - case Field::Types::Which::Map: return "Map"; - case Field::Types::Which::Object: return "Object"; - case Field::Types::Which::AggregateFunctionState: return "AggregateFunctionState"; - case Field::Types::Which::Bool: return "Bool"; - case Field::Types::Which::String: return "String"; - case Field::Types::Which::Decimal32: return "Decimal32"; - case Field::Types::Which::Decimal64: return "Decimal64"; - case Field::Types::Which::Decimal128: return "Decimal128"; - case Field::Types::Which::Decimal256: return "Decimal256"; - case Field::Types::Which::Float64: return "Float64"; - case Field::Types::Which::Int64: return "Int64"; - case Field::Types::Which::Int128: return "Int128"; - case Field::Types::Which::Int256: return "Int256"; - case Field::Types::Which::UInt64: return "UInt64"; - case Field::Types::Which::UInt128: return "UInt128"; - case Field::Types::Which::UInt256: return "UInt256"; - case Field::Types::Which::UUID: return "UUID"; - case Field::Types::Which::IPv4: return "IPv4"; - case Field::Types::Which::IPv6: return "IPv6"; - case Field::Types::Which::CustomType: return "CustomType"; + case Field::Types::Which::Null: return "Null"sv; + case Field::Types::Which::Array: return "Array"sv; + case Field::Types::Which::Tuple: return "Tuple"sv; + case Field::Types::Which::Map: return "Map"sv; + case Field::Types::Which::Object: return "Object"sv; + case Field::Types::Which::AggregateFunctionState: return "AggregateFunctionState"sv; + case Field::Types::Which::Bool: return "Bool"sv; + case Field::Types::Which::String: return "String"sv; + case Field::Types::Which::Decimal32: return "Decimal32"sv; + case Field::Types::Which::Decimal64: return "Decimal64"sv; + case Field::Types::Which::Decimal128: return "Decimal128"sv; + case Field::Types::Which::Decimal256: return "Decimal256"sv; + case Field::Types::Which::Float64: return "Float64"sv; + case Field::Types::Which::Int64: return "Int64"sv; + case Field::Types::Which::Int128: return "Int128"sv; + case Field::Types::Which::Int256: return "Int256"sv; + case Field::Types::Which::UInt64: return "UInt64"sv; + case Field::Types::Which::UInt128: return "UInt128"sv; + case Field::Types::Which::UInt256: return "UInt256"sv; + case Field::Types::Which::UUID: return "UUID"sv; + case Field::Types::Which::IPv4: return "IPv4"sv; + case Field::Types::Which::IPv6: return "IPv6"sv; + case Field::Types::Which::CustomType: return "CustomType"sv; } } +/// Keep in mind, that "magic_enum" is very expensive for compiler, that's why we don't use it. +std::string_view Field::getTypeName() const +{ + return fieldTypeToString(which); +} + + } diff --git a/src/Core/Field.h b/src/Core/Field.h index 6666e66e8d5..12542ca0bf1 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -1004,7 +1004,7 @@ void writeFieldText(const Field & x, WriteBuffer & buf); String toString(const Field & x); -String fieldTypeToString(Field::Types::Which type); +std::string_view fieldTypeToString(Field::Types::Which type); } From b8be7eef044052a24d0c00873d150d8de3a5ea4f Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 03:45:59 +0000 Subject: [PATCH 307/777] Added new tests for session_log and bugfix --- src/Interpreters/Session.cpp | 44 ++- src/Interpreters/Session.h | 2 + src/Interpreters/SessionLog.cpp | 14 +- src/Interpreters/SessionLog.h | 9 +- src/Server/HTTPHandler.cpp | 3 +- tests/integration/test_session_log/.gitignore | 1 + .../integration/test_session_log/__init__.py | 0 .../test_session_log/configs/log.xml | 9 + .../test_session_log/configs/ports.xml | 9 + .../test_session_log/configs/session_log.xml | 9 + .../test_session_log/configs/users.xml | 23 ++ .../protos/clickhouse_grpc.proto | 1 + tests/integration/test_session_log/test.py | 289 ++++++++++++++++++ .../02833_concurrrent_sessions.reference | 34 +++ .../0_stateless/02833_concurrrent_sessions.sh | 138 +++++++++ .../02834_remote_session_log.reference | 13 + .../0_stateless/02834_remote_session_log.sh | 56 ++++ .../02835_drop_user_during_session.reference | 8 + .../02835_drop_user_during_session.sh | 114 +++++++ 19 files changed, 753 insertions(+), 23 deletions(-) create mode 100644 tests/integration/test_session_log/.gitignore create mode 100644 tests/integration/test_session_log/__init__.py create mode 100644 tests/integration/test_session_log/configs/log.xml create mode 100644 tests/integration/test_session_log/configs/ports.xml create mode 100644 tests/integration/test_session_log/configs/session_log.xml create mode 100644 tests/integration/test_session_log/configs/users.xml create mode 120000 tests/integration/test_session_log/protos/clickhouse_grpc.proto create mode 100644 tests/integration/test_session_log/test.py create mode 100644 tests/queries/0_stateless/02833_concurrrent_sessions.reference create mode 100755 tests/queries/0_stateless/02833_concurrrent_sessions.sh create mode 100644 tests/queries/0_stateless/02834_remote_session_log.reference create mode 100755 tests/queries/0_stateless/02834_remote_session_log.sh create mode 100644 tests/queries/0_stateless/02835_drop_user_during_session.reference create mode 100755 tests/queries/0_stateless/02835_drop_user_during_session.sh diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index de2a779b740..0a6435cff75 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -520,6 +520,8 @@ ContextMutablePtr Session::makeSessionContext() {}, session_context->getSettingsRef().max_sessions_for_user); + recordLoginSucess(session_context); + return session_context; } @@ -582,6 +584,8 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: { session_name_ }, max_sessions_for_user); + recordLoginSucess(session_context); + return session_context; } @@ -655,24 +659,38 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t if (user_id) user = query_context->getUser(); - if (!notified_session_log_about_login) - { - if (auto session_log = getSessionLog()) - { - session_log->addLoginSuccess( - auth_id, - named_session ? std::optional(named_session->key.second) : std::nullopt, - *query_context, - user); - - notified_session_log_about_login = true; - } - } + /// Interserver does not create session context + recordLoginSucess(query_context); return query_context; } +void Session::recordLoginSucess(ContextPtr login_context) const +{ + if (notified_session_log_about_login) + return; + + if (!login_context) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Session or query context must be created"); + + if (auto session_log = getSessionLog()) + { + const auto & settings = login_context->getSettingsRef(); + const auto access = login_context->getAccess(); + + session_log->addLoginSuccess(auth_id, + named_session ? named_session->key.second : "", + settings, + access, + getClientInfo(), + user); + + notified_session_log_about_login = true; + } +} + + void Session::releaseSessionID() { if (!named_session) diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 51c0e3c71fa..81ef987b428 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -97,6 +97,8 @@ public: private: std::shared_ptr getSessionLog() const; ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; + void recordLoginSucess(ContextPtr login_context) const; + mutable bool notified_session_log_about_login = false; const UUID auth_id; diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index 0a8a7fc18c5..b847eaf9824 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -199,12 +199,13 @@ void SessionLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(auth_failure_reason.data(), auth_failure_reason.length()); } -void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user) +void SessionLog::addLoginSuccess(const UUID & auth_id, + const String & session_id, + const Settings & settings, + const ContextAccessPtr & access, + const ClientInfo & client_info, + const UserPtr & login_user) { - const auto access = login_context.getAccess(); - const auto & settings = login_context.getSettingsRef(); - const auto & client_info = login_context.getClientInfo(); - DB::SessionLogElement log_entry(auth_id, SESSION_LOGIN_SUCCESS); log_entry.client_info = client_info; @@ -215,8 +216,7 @@ void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional ses } log_entry.external_auth_server = login_user ? login_user->auth_data.getLDAPServerName() : ""; - if (session_id) - log_entry.session_id = *session_id; + log_entry.session_id = session_id; if (const auto roles_info = access->getRolesInfo()) log_entry.roles = roles_info->getCurrentRolesNames(); diff --git a/src/Interpreters/SessionLog.h b/src/Interpreters/SessionLog.h index 1282ac09c4d..8757bc12270 100644 --- a/src/Interpreters/SessionLog.h +++ b/src/Interpreters/SessionLog.h @@ -20,6 +20,7 @@ enum SessionLogElementType : int8_t class ContextAccess; struct User; using UserPtr = std::shared_ptr; +using ContextAccessPtr = std::shared_ptr; /** A struct which will be inserted as row into session_log table. * @@ -72,7 +73,13 @@ class SessionLog : public SystemLog using SystemLog::SystemLog; public: - void addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context, const UserPtr & login_user); + void addLoginSuccess(const UUID & auth_id, + const String & session_id, + const Settings & settings, + const ContextAccessPtr & access, + const ClientInfo & client_info, + const UserPtr & login_user); + void addLoginFailure(const UUID & auth_id, const ClientInfo & info, const std::optional & user, const Exception & reason); void addLogOut(const UUID & auth_id, const UserPtr & login_user, const ClientInfo & client_info); }; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index ff5690a3b07..10d5e7a0242 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -561,8 +561,7 @@ void HTTPHandler::processQuery( session->makeSessionContext(); } - auto client_info = session->getClientInfo(); - auto context = session->makeQueryContext(std::move(client_info)); + auto context = session->makeQueryContext(); /// This parameter is used to tune the behavior of output formats (such as Native) for compatibility. if (params.has("client_protocol_version")) diff --git a/tests/integration/test_session_log/.gitignore b/tests/integration/test_session_log/.gitignore new file mode 100644 index 00000000000..edf565ec632 --- /dev/null +++ b/tests/integration/test_session_log/.gitignore @@ -0,0 +1 @@ +_gen diff --git a/tests/integration/test_session_log/__init__.py b/tests/integration/test_session_log/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_session_log/configs/log.xml b/tests/integration/test_session_log/configs/log.xml new file mode 100644 index 00000000000..7a079b81e69 --- /dev/null +++ b/tests/integration/test_session_log/configs/log.xml @@ -0,0 +1,9 @@ + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/ports.xml b/tests/integration/test_session_log/configs/ports.xml new file mode 100644 index 00000000000..fbaefc16b3a --- /dev/null +++ b/tests/integration/test_session_log/configs/ports.xml @@ -0,0 +1,9 @@ + + 5433 + 9001 + 9100 + + + false + + \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/session_log.xml b/tests/integration/test_session_log/configs/session_log.xml new file mode 100644 index 00000000000..a0e4e3e2216 --- /dev/null +++ b/tests/integration/test_session_log/configs/session_log.xml @@ -0,0 +1,9 @@ + + + system + session_log
+ + toYYYYMM(event_date) + 7500 +
+
diff --git a/tests/integration/test_session_log/configs/users.xml b/tests/integration/test_session_log/configs/users.xml new file mode 100644 index 00000000000..0416dfadc8a --- /dev/null +++ b/tests/integration/test_session_log/configs/users.xml @@ -0,0 +1,23 @@ + + + + 0 + + + + + + + pass + + + pass + + + pass + + + pass + + + \ No newline at end of file diff --git a/tests/integration/test_session_log/protos/clickhouse_grpc.proto b/tests/integration/test_session_log/protos/clickhouse_grpc.proto new file mode 120000 index 00000000000..25d15f11e3b --- /dev/null +++ b/tests/integration/test_session_log/protos/clickhouse_grpc.proto @@ -0,0 +1 @@ +../../../../src/Server/grpc_protos/clickhouse_grpc.proto \ No newline at end of file diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py new file mode 100644 index 00000000000..dbb39993ce3 --- /dev/null +++ b/tests/integration/test_session_log/test.py @@ -0,0 +1,289 @@ +import os + +import grpc +import pymysql.connections +import psycopg2 as py_psql +import pytest +import random +import sys +import threading + +from helpers.cluster import ClickHouseCluster, run_and_check + +POSTGRES_SERVER_PORT = 5433 +MYSQL_SERVER_PORT = 9001 +GRPC_PORT = 9100 +SESSION_LOG_MATCHING_FIELDS = "auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DEFAULT_ENCODING = "utf-8" + +# Use grpcio-tools to generate *pb2.py files from *.proto. +proto_dir = os.path.join(SCRIPT_DIR, "./protos") +gen_dir = os.path.join(SCRIPT_DIR, "./_gen") +os.makedirs(gen_dir, exist_ok=True) +run_and_check( + f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", + shell=True, +) + +sys.path.append(gen_dir) + +import clickhouse_grpc_pb2 +import clickhouse_grpc_pb2_grpc + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "node", + main_configs=[ + "configs/ports.xml", + "configs/log.xml", + "configs/session_log.xml", + ], + user_configs=["configs/users.xml"], + # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 + env_variables={ + "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="") + }, +) + + +def grpc_get_url(): + return f"{instance.ip_address}:{GRPC_PORT}" + + +def grpc_create_insecure_channel(): + channel = grpc.insecure_channel(grpc_get_url()) + grpc.channel_ready_future(channel).result(timeout=2) + return channel + + +session_id_counter = 0 + + +def next_session_id(): + global session_id_counter + session_id = session_id_counter + session_id_counter += 1 + return str(session_id) + + +def grpc_query(query, user_, pass_, raise_exception): + try: + query_info = clickhouse_grpc_pb2.QueryInfo( + query=query, + session_id=next_session_id(), + user_name=user_, + password=pass_, + ) + channel = grpc_create_insecure_channel() + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) + result = stub.ExecuteQuery(query_info) + if result and result.HasField("exception"): + raise Exception(result.exception.display_text) + + return result.output.decode(DEFAULT_ENCODING) + except Exception: + assert raise_exception + + +def postgres_query(query, user_, pass_, raise_exception): + try: + client = py_psql.connect( + host=instance.ip_address, + port=POSTGRES_SERVER_PORT, + user=user_, + password=pass_, + database="default", + ) + cursor = client.cursor() + cursor.execute(query) + cursor.fetchall() + except Exception: + assert raise_exception + + +def mysql_query(query, user_, pass_, raise_exception): + try: + client = pymysql.connections.Connection( + host=instance.ip_address, + user=user_, + password=pass_, + database="default", + port=MYSQL_SERVER_PORT, + ) + cursor = client.cursor(pymysql.cursors.DictCursor) + if raise_exception: + with pytest.raises(Exception): + cursor.execute(query) + else: + cursor.execute(query) + cursor.fetchall() + except Exception: + assert raise_exception + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_grpc_session(started_cluster): + grpc_query("SELECT 1", "grpc_user", "pass", False) + grpc_query("SELECT 2", "grpc_user", "wrong_pass", True) + grpc_query("SELECT 3", "wrong_grpc_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "grpc_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'Logout'" + ) + assert logout_records == "grpc_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "grpc_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_mysql_session(started_cluster): + mysql_query("SELECT 1", "mysql_user", "pass", False) + mysql_query("SELECT 2", "mysql_user", "wrong_pass", True) + mysql_query("SELECT 3", "wrong_mysql_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "mysql_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'Logout'" + ) + assert logout_records == "mysql_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "mysql_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_postgres_session(started_cluster): + postgres_query("SELECT 1", "postgres_user", "pass", False) + postgres_query("SELECT 2", "postgres_user", "wrong_pass", True) + postgres_query("SELECT 3", "wrong_postgres_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "postgres_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'Logout'" + ) + assert logout_records == "postgres_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "postgres_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_parallel_sessions(started_cluster): + thread_list = [] + for _ in range(10): + # Sleep time does not significantly matter here, + # test should pass even without sleeping. + for function in [postgres_query, grpc_query, mysql_query]: + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "parallel_user", + "pass", + False, + ), + ) + thread.start() + thread_list.append(thread) + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "parallel_user", + "wrong_pass", + True, + ), + ) + thread.start() + thread_list.append(thread) + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "wrong_parallel_user", + "pass", + True, + ), + ) + thread.start() + thread_list.append(thread) + + for thread in thread_list: + thread.join() + + instance.query("SYSTEM FLUSH LOGS") + port_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user'" + ) + assert port_0_sessions == "90\n" + + port_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_port = 0" + ) + assert port_0_sessions == "0\n" + + address_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_address = toIPv6('::')" + ) + assert address_0_sessions == "0\n" + + grpc_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'gRPC'" + ) + assert grpc_sessions == "30\n" + + mysql_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'MySQL'" + ) + assert mysql_sessions == "30\n" + + postgres_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'PostgreSQL'" + ) + assert postgres_sessions == "30\n" + + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "30\n" + + logout_failure_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'" + ) + assert logout_failure_sessions == "30\n" diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.reference b/tests/queries/0_stateless/02833_concurrrent_sessions.reference new file mode 100644 index 00000000000..bfe507e8eac --- /dev/null +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.reference @@ -0,0 +1,34 @@ +sessions: +150 +port_0_sessions: +0 +address_0_sessions: +0 +tcp_sessions +60 +http_sessions +30 +http_with_session_id_sessions +30 +my_sql_sessions +30 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh new file mode 100755 index 00000000000..26b48462a76 --- /dev/null +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.sh @@ -0,0 +1,138 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-debug + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ + +# Each user uses a separate thread. +readonly TCP_USERS=( "02833_TCP_USER_${PID}"_{1,2} ) # 2 concurrent TCP users +readonly HTTP_USERS=( "02833_HTTP_USER_${PID}" ) +readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "02833_HTTP_WITH_SESSION_ID_USER_${PID}" ) +readonly MYSQL_USERS=( "02833_MYSQL_USER_${PID}") +readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) + +readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" + +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +for user in "${ALL_USERS[@]}"; do + ${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${user} IDENTIFIED WITH plaintext_password BY 'pass'" + ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${user}" + ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${user}"; +done + +# All _session functions execute in separate threads. +# These functions try to create a session with successful login and logout. +# Sleep a small, random amount of time to make concurrency more intense. +# and try to login with an invalid password. +function tcp_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM})" --user="${user}" --password="pass" + # login failure + ${CLICKHOUSE_CLIENT} -q "SELECT 2" --user="${user}" --password 'invalid' + done +} + +function http_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep(0.01${RANDOM})" + + # login failure + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=wrong" -d "SELECT 4" + done +} + +function http_with_session_id_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.01${RANDOM}" + + # login failure + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=wrong" -d "SELECT 6" + done +} + +function mysql_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" + + # login failure + ${CLICKHOUSE_CLIENT} -q "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'wrong', SETTINGS connection_max_tries=1)" + done +} + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" + +export -f tcp_session; +export -f http_session; +export -f http_with_session_id_session; +export -f mysql_session; + +for user in "${TCP_USERS[@]}"; do + timeout 60s bash -c "tcp_session ${user}" >/dev/null 2>&1 & +done + +for user in "${HTTP_USERS[@]}"; do + timeout 60s bash -c "http_session ${user}" >/dev/null 2>&1 & +done + +for user in "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}"; do + timeout 60s bash -c "http_with_session_id_session ${user}" >/dev/null 2>&1 & +done + +for user in "${MYSQL_USERS[@]}"; do + timeout 60s bash -c "mysql_session ${user}" >/dev/null 2>&1 & +done + +wait + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" + +echo "port_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_port = 0" + +echo "address_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_address = toIPv6('::')" + +echo "tcp_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${TCP_USERS_SQL_COLLECTION_STRING}) AND interface = 'TCP'" +echo "http_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" +echo "http_with_session_id_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" +echo "my_sql_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${MYSQL_USERS_SQL_COLLECTION_STRING}) AND interface = 'MySQL'" + +for user in "${ALL_USERS[@]}"; do + ${CLICKHOUSE_CLIENT} -q "DROP USER ${user}" + echo "Corresponding LoginSuccess/Logout" + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'Logout')" + echo "LoginFailure" + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" + done diff --git a/tests/queries/0_stateless/02834_remote_session_log.reference b/tests/queries/0_stateless/02834_remote_session_log.reference new file mode 100644 index 00000000000..e2680982ab0 --- /dev/null +++ b/tests/queries/0_stateless/02834_remote_session_log.reference @@ -0,0 +1,13 @@ +0 +0 +0 +0 +client_port 0 connections: +0 +client_address '::' connections: +0 +login failures: +0 +TCP Login and logout count is equal +HTTP Login and logout count is equal +MySQL Login and logout count is equal diff --git a/tests/queries/0_stateless/02834_remote_session_log.sh b/tests/queries/0_stateless/02834_remote_session_log.sh new file mode 100755 index 00000000000..3bedfb6c9ee --- /dev/null +++ b/tests/queries/0_stateless/02834_remote_session_log.sh @@ -0,0 +1,56 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ +readonly TEST_USER=$"02834_USER_${PID}" +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER} IDENTIFIED WITH plaintext_password BY 'pass'" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT CREATE TEMPORARY TABLE, MYSQL, REMOTE ON *.* TO ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" + +${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ + -d "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" + +${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ + -d "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" + +${CLICKHOUSE_CLIENT} -q "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "client_port 0 connections:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_port = 0" + +echo "client_address '::' connections:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_address = toIPv6('::')" + +echo "login failures:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and type = 'LoginFailure'" + +# remote(...) function sometimes reuses old cached sessions for query execution. +# This makes LoginSuccess/Logout entries count unstable, but success and logouts must always match. + +for interface in 'TCP' 'HTTP' 'MySQL' +do + LOGIN_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}'"` + CORRESPONDING_LOGOUT_RECORDS_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}')"` + + if [ "$LOGIN_COUNT" == "$CORRESPONDING_LOGOUT_RECORDS_COUNT" ]; then + echo "${interface} Login and logout count is equal" + else + TOTAL_LOGOUT_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}'"` + echo "${interface} Login count ${LOGIN_COUNT} != corresponding logout count ${CORRESPONDING_LOGOUT_RECORDS_COUNT}. TOTAL_LOGOUT_COUNT ${TOTAL_LOGOUT_COUNT}" + fi +done + +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.reference b/tests/queries/0_stateless/02835_drop_user_during_session.reference new file mode 100644 index 00000000000..7252faab8c6 --- /dev/null +++ b/tests/queries/0_stateless/02835_drop_user_during_session.reference @@ -0,0 +1,8 @@ +port_0_sessions: +0 +address_0_sessions: +0 +Corresponding LoginSuccess/Logout +9 +LoginFailure +0 diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh new file mode 100755 index 00000000000..347ebd22f96 --- /dev/null +++ b/tests/queries/0_stateless/02835_drop_user_during_session.sh @@ -0,0 +1,114 @@ +#!/usr/bin/env bash +# Tags: no-debug + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ + +readonly TEST_USER="02835_USER_${PID}" +readonly TEST_ROLE="02835_ROLE_${PID}" +readonly TEST_PROFILE="02835_PROFILE_${PID}" +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +function tcp_session() +{ + local user=$1 + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.numbers" --user="${user}" +} + +function http_session() +{ + local user=$1 + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" +} + +function http_with_session_id_session() +{ + local user=$1 + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" +} + +# Busy-waits until user $1, specified amount of queries ($2) will run simultaneously. +function wait_for_queries_start() +{ + local user=$1 + local queries_count=$2 + # 10 seconds waiting + counter=0 retries=100 + while [[ $counter -lt $retries ]]; do + result=$($CLICKHOUSE_CLIENT --query "SELECT COUNT(*) FROM system.processes WHERE user = '${user}'") + if [[ $result == "${queries_count}" ]]; then + break; + fi + sleep 0.1 + ((++counter)) + done +} + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" + +# DROP USE CASE +${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" + +export -f tcp_session; +export -f http_session; +export -f http_with_session_id_session; + +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +# DROP ROLE CASE +${CLICKHOUSE_CLIENT} -q "CREATE ROLE IF NOT EXISTS ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} DEFAULT ROLE ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" + +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP ROLE ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +# DROP PROFILE CASE +${CLICKHOUSE_CLIENT} -q "CREATE SETTINGS PROFILE IF NOT EXISTS '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} SETTINGS PROFILE '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" + +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP SETTINGS PROFILE '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "port_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_port = 0" +echo "address_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_address = toIPv6('::')" +echo "Corresponding LoginSuccess/Logout" +${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout')" +echo "LoginFailure" +${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" From 26aded5062f73e14f428af0dc2f4280fae813964 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 04:11:07 +0000 Subject: [PATCH 308/777] Used main connections for suggestions --- src/Client/ClientBase.cpp | 8 ++++ src/Client/Suggest.cpp | 41 ++++++++++++++----- src/Client/Suggest.h | 9 ++++ tests/integration/parallel_skip.json | 3 +- .../test.py | 18 ++++++++ 5 files changed, 68 insertions(+), 11 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index a72de2645d4..9e4d79cd323 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -105,6 +105,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int CANNOT_OPEN_FILE; extern const int FILE_ALREADY_EXISTS; + extern const int USER_SESSION_LIMIT_EXCEEDED; } } @@ -2408,6 +2409,13 @@ void ClientBase::runInteractive() } } + if (suggest && suggest->getLastError() == ErrorCodes::USER_SESSION_LIMIT_EXCEEDED) + { + // If a separate connection loading suggestions failed to open a new session, + // use the main session to receive them. + suggest->load(*connection, connection_parameters.timeouts, config().getInt("suggestion_limit")); + } + try { if (!processQueryText(input)) diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 00e0ebd8b91..c854d471fae 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -22,9 +22,11 @@ namespace DB { namespace ErrorCodes { + extern const int OK; extern const int LOGICAL_ERROR; extern const int UNKNOWN_PACKET_FROM_SERVER; extern const int DEADLOCK_AVOIDED; + extern const int USER_SESSION_LIMIT_EXCEEDED; } Suggest::Suggest() @@ -121,21 +123,24 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p } catch (const Exception & e) { + last_error = e.code(); if (e.code() == ErrorCodes::DEADLOCK_AVOIDED) continue; - - /// Client can successfully connect to the server and - /// get ErrorCodes::USER_SESSION_LIMIT_EXCEEDED for suggestion connection. - - /// We should not use std::cerr here, because this method works concurrently with the main thread. - /// WriteBufferFromFileDescriptor will write directly to the file descriptor, avoiding data race on std::cerr. - - WriteBufferFromFileDescriptor out(STDERR_FILENO, 4096); - out << "Cannot load data for command line suggestions: " << getCurrentExceptionMessage(false, true) << "\n"; - out.next(); + else if (e.code() != ErrorCodes::USER_SESSION_LIMIT_EXCEEDED) + { + /// We should not use std::cerr here, because this method works concurrently with the main thread. + /// WriteBufferFromFileDescriptor will write directly to the file descriptor, avoiding data race on std::cerr. + /// + /// USER_SESSION_LIMIT_EXCEEDED is ignored here. The client will try to receive + /// suggestions using the main connection later. + WriteBufferFromFileDescriptor out(STDERR_FILENO, 4096); + out << "Cannot load data for command line suggestions: " << getCurrentExceptionMessage(false, true) << "\n"; + out.next(); + } } catch (...) { + last_error = getCurrentExceptionCode(); WriteBufferFromFileDescriptor out(STDERR_FILENO, 4096); out << "Cannot load data for command line suggestions: " << getCurrentExceptionMessage(false, true) << "\n"; out.next(); @@ -148,6 +153,21 @@ void Suggest::load(ContextPtr context, const ConnectionParameters & connection_p }); } +void Suggest::load(IServerConnection & connection, + const ConnectionTimeouts & timeouts, + Int32 suggestion_limit) +{ + try + { + fetch(connection, timeouts, getLoadSuggestionQuery(suggestion_limit, true)); + } + catch (...) + { + std::cerr << "Suggestions loading exception: " << getCurrentExceptionMessage(false, true) << std::endl; + last_error = getCurrentExceptionCode(); + } +} + void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & timeouts, const std::string & query) { connection.sendQuery( @@ -176,6 +196,7 @@ void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & t return; case Protocol::Server::EndOfStream: + last_error = ErrorCodes::OK; return; default: diff --git a/src/Client/Suggest.h b/src/Client/Suggest.h index cfe9315879c..5cecdc4501b 100644 --- a/src/Client/Suggest.h +++ b/src/Client/Suggest.h @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -28,9 +29,15 @@ public: template void load(ContextPtr context, const ConnectionParameters & connection_parameters, Int32 suggestion_limit); + void load(IServerConnection & connection, + const ConnectionTimeouts & timeouts, + Int32 suggestion_limit); + /// Older server versions cannot execute the query loading suggestions. static constexpr int MIN_SERVER_REVISION = DBMS_MIN_PROTOCOL_VERSION_WITH_VIEW_IF_PERMITTED; + int getLastError() const { return last_error.load(); } + private: void fetch(IServerConnection & connection, const ConnectionTimeouts & timeouts, const std::string & query); @@ -38,6 +45,8 @@ private: /// Words are fetched asynchronously. std::thread loading_thread; + + std::atomic last_error { -1 }; }; } diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index dec51396c51..d056225fee4 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -91,5 +91,6 @@ "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_http_named_session", "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_grpc", "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_tcp_and_others", - "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_setting_in_query" + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_setting_in_query", + "test_profile_max_sessions_for_user/test.py::test_profile_max_sessions_for_user_client_suggestions_load" ] diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 2930262f63e..925fa05881d 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -10,6 +10,7 @@ import threading from helpers.cluster import ClickHouseCluster, run_and_check from helpers.test_tools import assert_logs_contain_with_retry +from helpers.uclient import client, prompt MAX_SESSIONS_FOR_USER = 2 POSTGRES_SERVER_PORT = 5433 @@ -209,3 +210,20 @@ def test_profile_max_sessions_for_user_tcp_and_others(started_cluster): def test_profile_max_sessions_for_user_setting_in_query(started_cluster): instance.query_and_get_error("SET max_sessions_for_user = 10") + + +def test_profile_max_sessions_for_user_client_suggestions_connection(started_cluster): + command_text = f"{started_cluster.get_client_cmd()} --host {instance.ip_address} --port 9000 -u {TEST_USER} --password {TEST_PASSWORD}" + with client(name="client1>", log=None, command=command_text) as client1: + client1.expect(prompt) + with client(name="client2>", log=None, command=command_text) as client2: + client2.expect(prompt) + with client(name="client3>", log=None, command=command_text) as client3: + client3.expect("USER_SESSION_LIMIT_EXCEEDED") + + client1.send("SELECT 'CLIENT_1_SELECT' FORMAT CSV") + client1.expect("CLIENT_1_SELECT") + client1.expect(prompt) + client2.send("SELECT 'CLIENT_2_SELECT' FORMAT CSV") + client2.expect("CLIENT_2_SELECT") + client2.expect(prompt) From 7b0036dee3ef050c060ee2aee0ab9e6faf927e41 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 04:23:40 +0000 Subject: [PATCH 309/777] moved notified_session_log_about_login=true line --- src/Interpreters/Session.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 0a6435cff75..f8bd70afdb6 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -685,9 +685,9 @@ void Session::recordLoginSucess(ContextPtr login_context) const access, getClientInfo(), user); - - notified_session_log_about_login = true; } + + notified_session_log_about_login = true; } From d499f8030a34d5076c7a9fc97d94a5f04b2898d6 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 05:20:00 +0000 Subject: [PATCH 310/777] reverted change in test --- tests/integration/test_profile_max_sessions_for_user/test.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 2930262f63e..65587933fed 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -27,10 +27,7 @@ proto_dir = os.path.join(SCRIPT_DIR, "./protos") gen_dir = os.path.join(SCRIPT_DIR, "./_gen") os.makedirs(gen_dir, exist_ok=True) run_and_check( - "python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \ - {proto_dir}/clickhouse_grpc.proto".format( - proto_dir=proto_dir, gen_dir=gen_dir - ), + f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", shell=True, ) From 7321f5e543387aa65e20fe12421ee692eb05aa64 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 10 Aug 2023 06:32:28 +0000 Subject: [PATCH 311/777] Better --- src/IO/WriteHelpers.h | 15 ++++++++--- src/Parsers/ASTLiteral.cpp | 26 +++++++++++++++++-- src/Parsers/IAST.h | 7 ++++- src/Parsers/LiteralEscapingStyle.h | 14 ++++++++++ src/Storages/StorageMySQL.cpp | 1 + src/Storages/StoragePostgreSQL.cpp | 6 +---- src/Storages/StorageSQLite.cpp | 1 + src/Storages/StorageXDBC.cpp | 1 + .../transformQueryForExternalDatabase.cpp | 7 ++++- .../transformQueryForExternalDatabase.h | 1 + .../test_storage_postgresql/test.py | 20 +++++++------- 11 files changed, 78 insertions(+), 21 deletions(-) create mode 100644 src/Parsers/LiteralEscapingStyle.h diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index aa4c9b17e48..d092c7b8ea5 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -304,9 +304,10 @@ inline void writeJSONString(const char * begin, const char * end, WriteBuffer & /** Will escape quote_character and a list of special characters('\b', '\f', '\n', '\r', '\t', '\0', '\\'). * - when escape_quote_with_quote is true, use backslash to escape list of special characters, * and use quote_character to escape quote_character. such as: 'hello''world' - * - otherwise use backslash to escape list of special characters and quote_character + * otherwise use backslash to escape list of special characters and quote_character + * - when escape_backslash_with_backslash is true, backslash is escaped with another backslash */ -template +template void writeAnyEscapedString(const char * begin, const char * end, WriteBuffer & buf) { const char * pos = begin; @@ -360,7 +361,8 @@ void writeAnyEscapedString(const char * begin, const char * end, WriteBuffer & b writeChar('0', buf); break; case '\\': - writeChar('\\', buf); + if constexpr (escape_backslash_with_backslash) + writeChar('\\', buf); writeChar('\\', buf); break; default: @@ -466,6 +468,13 @@ inline void writeQuotedString(std::string_view ref, WriteBuffer & buf) writeAnyQuotedString<'\''>(ref.data(), ref.data() + ref.size(), buf); } +inline void writeQuotedStringPostgreSQL(std::string_view ref, WriteBuffer & buf) +{ + writeChar('\'', buf); + writeAnyEscapedString<'\'', true, false>(ref.data(), ref.data() + ref.size(), buf); + writeChar('\'', buf); +} + inline void writeDoubleQuotedString(const String & s, WriteBuffer & buf) { writeAnyQuotedString<'"'>(s, buf); diff --git a/src/Parsers/ASTLiteral.cpp b/src/Parsers/ASTLiteral.cpp index 5c76f6f33bf..4a9a3d8df5b 100644 --- a/src/Parsers/ASTLiteral.cpp +++ b/src/Parsers/ASTLiteral.cpp @@ -93,7 +93,7 @@ void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const void ASTLiteral::appendColumnNameImplLegacy(WriteBuffer & ostr) const { - /// 100 - just arbitrary value. + /// 100 - just arbitrary value. constexpr auto min_elements_for_hashing = 100; /// Special case for very large arrays. Instead of listing all elements, will use hash of them. @@ -118,9 +118,31 @@ void ASTLiteral::appendColumnNameImplLegacy(WriteBuffer & ostr) const } } +/// Use different rules for escaping backslashes and quotes +class FieldVisitorToStringPostgreSQL : public StaticVisitor +{ +public: + template + String operator() (const T & x) const { return visitor(x); } + +private: + FieldVisitorToString visitor; +}; + +template<> +String FieldVisitorToStringPostgreSQL::operator() (const String & x) const +{ + WriteBufferFromOwnString wb; + writeQuotedStringPostgreSQL(x, wb); + return wb.str(); +} + void ASTLiteral::formatImplWithoutAlias(const FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const { - settings.ostr << applyVisitor(FieldVisitorToString(), value); + if (settings.literal_escaping_style == LiteralEscapingStyle::Regular) + settings.ostr << applyVisitor(FieldVisitorToString(), value); + else + settings.ostr << applyVisitor(FieldVisitorToStringPostgreSQL(), value); } } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index d217876459f..58bc9702142 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -197,6 +198,7 @@ public: IdentifierQuotingStyle identifier_quoting_style; bool show_secrets; /// Show secret parts of the AST (e.g. passwords, encryption keys). char nl_or_ws; /// Newline or whitespace. + LiteralEscapingStyle literal_escaping_style; explicit FormatSettings( WriteBuffer & ostr_, @@ -204,7 +206,8 @@ public: bool hilite_ = false, bool always_quote_identifiers_ = false, IdentifierQuotingStyle identifier_quoting_style_ = IdentifierQuotingStyle::Backticks, - bool show_secrets_ = true) + bool show_secrets_ = true, + LiteralEscapingStyle literal_escaping_style_ = LiteralEscapingStyle::Regular) : ostr(ostr_) , one_line(one_line_) , hilite(hilite_) @@ -212,6 +215,7 @@ public: , identifier_quoting_style(identifier_quoting_style_) , show_secrets(show_secrets_) , nl_or_ws(one_line ? ' ' : '\n') + , literal_escaping_style(literal_escaping_style_) { } @@ -223,6 +227,7 @@ public: , identifier_quoting_style(other.identifier_quoting_style) , show_secrets(other.show_secrets) , nl_or_ws(other.nl_or_ws) + , literal_escaping_style(other.literal_escaping_style) { } diff --git a/src/Parsers/LiteralEscapingStyle.h b/src/Parsers/LiteralEscapingStyle.h new file mode 100644 index 00000000000..10d4d84a85d --- /dev/null +++ b/src/Parsers/LiteralEscapingStyle.h @@ -0,0 +1,14 @@ +#pragma once + + +namespace DB +{ + +/// Method to escape single quotes. +enum class LiteralEscapingStyle +{ + Regular, /// Escape backslashes with backslash (\\) and quotes with backslash (\') + PostgreSQL, /// Do not escape backslashes (\), escape quotes with quote ('') +}; + +} diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index b0a220eb1d2..76a439eabaf 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -104,6 +104,7 @@ Pipe StorageMySQL::read( column_names_, storage_snapshot->metadata->getColumns().getOrdinary(), IdentifierQuotingStyle::BackticksMySQL, + LiteralEscapingStyle::Regular, remote_database_name, remote_table_name, context_); diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 11558b39ad3..f233d4ff213 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -123,11 +123,7 @@ Pipe StoragePostgreSQL::read( query_info_, column_names_, storage_snapshot->metadata->getColumns().getOrdinary(), - IdentifierQuotingStyle::DoubleQuotes, remote_table_schema, remote_table_name, context_); - - /// Single quotes in PostgreSQL are escaped through repetition - boost::replace_all(query, "\\'", "''"); - + IdentifierQuotingStyle::DoubleQuotes, LiteralEscapingStyle::PostgreSQL, remote_table_schema, remote_table_name, context_); LOG_TRACE(log, "Query: {}", query); Block sample_block; diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index d5ae6f2383f..d5db5763da9 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -91,6 +91,7 @@ Pipe StorageSQLite::read( column_names, storage_snapshot->metadata->getColumns().getOrdinary(), IdentifierQuotingStyle::DoubleQuotes, + LiteralEscapingStyle::Regular, "", remote_table_name, context_); diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index b532d1c91f0..1715cde9d1e 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -79,6 +79,7 @@ std::function StorageXDBC::getReadPOSTDataCallback( column_names, columns_description.getOrdinary(), bridge_helper->getIdentifierQuotingStyle(), + LiteralEscapingStyle::Regular, remote_database_name, remote_table_name, local_context); diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 375510e62bf..84a696a1e9c 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -258,6 +258,7 @@ String transformQueryForExternalDatabaseImpl( Names used_columns, const NamesAndTypesList & available_columns, IdentifierQuotingStyle identifier_quoting_style, + LiteralEscapingStyle literal_escaping_style, const String & database, const String & table, ContextPtr context) @@ -337,7 +338,8 @@ String transformQueryForExternalDatabaseImpl( IAST::FormatSettings settings( out, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ identifier_quoting_style != IdentifierQuotingStyle::None, - /*identifier_quoting_style*/ identifier_quoting_style); + /*identifier_quoting_style*/ identifier_quoting_style, /*show_secrets_*/ true, + /*literal_escaping_style*/ literal_escaping_style); select->format(settings); @@ -351,6 +353,7 @@ String transformQueryForExternalDatabase( const Names & column_names, const NamesAndTypesList & available_columns, IdentifierQuotingStyle identifier_quoting_style, + LiteralEscapingStyle literal_escaping_style, const String & database, const String & table, ContextPtr context) @@ -375,6 +378,7 @@ String transformQueryForExternalDatabase( column_names, available_columns, identifier_quoting_style, + literal_escaping_style, database, table, context); @@ -386,6 +390,7 @@ String transformQueryForExternalDatabase( query_info.syntax_analyzer_result->requiredSourceColumns(), available_columns, identifier_quoting_style, + literal_escaping_style, database, table, context); diff --git a/src/Storages/transformQueryForExternalDatabase.h b/src/Storages/transformQueryForExternalDatabase.h index 0f2b0a5822f..fb6af21907e 100644 --- a/src/Storages/transformQueryForExternalDatabase.h +++ b/src/Storages/transformQueryForExternalDatabase.h @@ -31,6 +31,7 @@ String transformQueryForExternalDatabase( const Names & column_names, const NamesAndTypesList & available_columns, IdentifierQuotingStyle identifier_quoting_style, + LiteralEscapingStyle literal_escaping_style, const String & database, const String & table, ContextPtr context); diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 3a36d050f17..d4f8fab3a82 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -726,18 +726,20 @@ def test_auto_close_connection(started_cluster): assert count == 2 -def test_single_quotes(started_cluster): +def test_literal_escaping(started_cluster): cursor = started_cluster.postgres_conn.cursor() - cursor.execute(f"DROP TABLE IF EXISTS single_quote_fails") - cursor.execute(f"CREATE TABLE single_quote_fails(text varchar(255))") + cursor.execute(f"DROP TABLE IF EXISTS escaping") + cursor.execute(f"CREATE TABLE escaping(text varchar(255))") node1.query( - "CREATE TABLE default.single_quote_fails (text String) ENGINE = PostgreSQL('postgres1:5432', 'postgres', 'single_quote_fails', 'postgres', 'mysecretpassword')" + "CREATE TABLE default.escaping (text String) ENGINE = PostgreSQL('postgres1:5432', 'postgres', 'escaping', 'postgres', 'mysecretpassword')" ) - node1.query("SELECT * FROM single_quote_fails WHERE text = ''''") - node1.query("SELECT * FROM single_quote_fails WHERE text = '\\''") - node1.query("SELECT * FROM single_quote_fails WHERE text like '%a''a%'") - node1.query("SELECT * FROM single_quote_fails WHERE text like '%a\\'a%'") - cursor.execute(f"DROP TABLE single_quote_fails") + node1.query("SELECT * FROM escaping WHERE text = ''''") # ' -> '' + node1.query("SELECT * FROM escaping WHERE text = '\\''") # ' -> '' + node1.query("SELECT * FROM escaping WHERE text = '\\\\\\''") # \' -> \'' + node1.query("SELECT * FROM escaping WHERE text = '\\\\\\''") # \' -> \'' + node1.query("SELECT * FROM escaping WHERE text like '%a''a%'") # %a'a% -> %a''a% + node1.query("SELECT * FROM escaping WHERE text like '%a\\'a%'") # %a'a% -> %a''a% + cursor.execute(f"DROP TABLE escaping") if __name__ == "__main__": From c65f3d1eb870b0f5ce4272ee699edda6bed1423d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 10 Aug 2023 10:09:59 +0000 Subject: [PATCH 312/777] Mutex for name filter --- src/Storages/StorageFile.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index a0dc98d4312..0c720f2d7da 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -747,9 +747,10 @@ public: return !name_filter; } - IArchiveReader::NameFilter getNameFilter() const + bool passesFilter(const std::string & name) const { - return name_filter; + std::lock_guard lock(filter_mutex); + return name_filter(name); } const String & getFileName() @@ -763,6 +764,7 @@ public: std::vector files; std::vector archives; + mutable std::mutex filter_mutex; IArchiveReader::NameFilter name_filter; std::atomic index = 0; @@ -892,8 +894,6 @@ public: } else { - auto name_filter = files_iterator->getNameFilter(); - chassert(name_filter); while (true) { if (file_enumerator == nullptr) @@ -912,7 +912,7 @@ public: } bool file_found = true; - while (!name_filter(file_enumerator->getFileName())) + while (!files_iterator->passesFilter(file_enumerator->getFileName())) { if (!file_enumerator->nextFile()) { From 5cdeacf4cf61c0ac228eb63b7178392a6436d41c Mon Sep 17 00:00:00 2001 From: Ruslan Mardugalliamov Date: Sun, 6 Aug 2023 15:33:36 -0400 Subject: [PATCH 313/777] Add hints for HTTP handlers Add hints to HTTP handlers to help users avoid misspellings. For example, if a user mistakenly writes `/dashboad` instead of `/dashboard`, they will now get a hint that /dashboard is the correct handler. This change will improve the user experience by making it easier for users to find the correct handlers. #47662 --- programs/keeper/CMakeLists.txt | 1 + src/Server/HTTPHandlerFactory.cpp | 4 ++++ src/Server/HTTPPathHints.cpp | 16 ++++++++++++++ src/Server/HTTPPathHints.h | 22 +++++++++++++++++++ src/Server/HTTPRequestHandlerFactoryMain.cpp | 2 +- src/Server/HTTPRequestHandlerFactoryMain.h | 4 ++++ src/Server/NotFoundHandler.cpp | 3 ++- src/Server/NotFoundHandler.h | 3 +++ ...ggest_http_page_in_error_message.reference | 4 ++++ ...2842_suggest_http_page_in_error_message.sh | 12 ++++++++++ 10 files changed, 69 insertions(+), 2 deletions(-) create mode 100644 src/Server/HTTPPathHints.cpp create mode 100644 src/Server/HTTPPathHints.h create mode 100644 tests/queries/0_stateless/02842_suggest_http_page_in_error_message.reference create mode 100755 tests/queries/0_stateless/02842_suggest_http_page_in_error_message.sh diff --git a/programs/keeper/CMakeLists.txt b/programs/keeper/CMakeLists.txt index 43a8d84b513..a43a312ba54 100644 --- a/programs/keeper/CMakeLists.txt +++ b/programs/keeper/CMakeLists.txt @@ -57,6 +57,7 @@ if (BUILD_STANDALONE_KEEPER) ${CMAKE_CURRENT_SOURCE_DIR}/../../src/IO/ReadBuffer.cpp + ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/HTTPPathHints.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/KeeperTCPHandler.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/TCPServer.cpp ${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/NotFoundHandler.cpp diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 78e374ee9e0..1c911034da1 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -132,21 +132,25 @@ void addCommonDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IS auto ping_handler = std::make_shared>(server, ping_response_expression); ping_handler->attachStrictPath("/ping"); ping_handler->allowGetAndHeadRequest(); + factory.addPathToHints("/ping"); factory.addHandler(ping_handler); auto replicas_status_handler = std::make_shared>(server); replicas_status_handler->attachNonStrictPath("/replicas_status"); replicas_status_handler->allowGetAndHeadRequest(); + factory.addPathToHints("/replicas_status"); factory.addHandler(replicas_status_handler); auto play_handler = std::make_shared>(server); play_handler->attachNonStrictPath("/play"); play_handler->allowGetAndHeadRequest(); + factory.addPathToHints("/play"); factory.addHandler(play_handler); auto dashboard_handler = std::make_shared>(server); dashboard_handler->attachNonStrictPath("/dashboard"); dashboard_handler->allowGetAndHeadRequest(); + factory.addPathToHints("/dashboard"); factory.addHandler(dashboard_handler); auto js_handler = std::make_shared>(server); diff --git a/src/Server/HTTPPathHints.cpp b/src/Server/HTTPPathHints.cpp new file mode 100644 index 00000000000..51ef3eabffe --- /dev/null +++ b/src/Server/HTTPPathHints.cpp @@ -0,0 +1,16 @@ +#include + +namespace DB +{ + +void HTTPPathHints::add(const String & http_path) +{ + http_paths.push_back(http_path); +} + +std::vector HTTPPathHints::getAllRegisteredNames() const +{ + return http_paths; +} + +} diff --git a/src/Server/HTTPPathHints.h b/src/Server/HTTPPathHints.h new file mode 100644 index 00000000000..708816ebf07 --- /dev/null +++ b/src/Server/HTTPPathHints.h @@ -0,0 +1,22 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +class HTTPPathHints : public IHints<1, HTTPPathHints> +{ +public: + std::vector getAllRegisteredNames() const override; + void add(const String & http_path); + +private: + std::vector http_paths; +}; + +using HTTPPathHintsPtr = std::shared_ptr; + +} diff --git a/src/Server/HTTPRequestHandlerFactoryMain.cpp b/src/Server/HTTPRequestHandlerFactoryMain.cpp index 61a2909d30f..5481bcd5083 100644 --- a/src/Server/HTTPRequestHandlerFactoryMain.cpp +++ b/src/Server/HTTPRequestHandlerFactoryMain.cpp @@ -29,7 +29,7 @@ std::unique_ptr HTTPRequestHandlerFactoryMain::createRequest || request.getMethod() == Poco::Net::HTTPRequest::HTTP_HEAD || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) { - return std::unique_ptr(new NotFoundHandler); + return std::unique_ptr(new NotFoundHandler(hints.getHints(request.getURI()))); } return nullptr; diff --git a/src/Server/HTTPRequestHandlerFactoryMain.h b/src/Server/HTTPRequestHandlerFactoryMain.h index b0e57bd6b3b..07b278d831c 100644 --- a/src/Server/HTTPRequestHandlerFactoryMain.h +++ b/src/Server/HTTPRequestHandlerFactoryMain.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -15,11 +16,14 @@ public: void addHandler(HTTPRequestHandlerFactoryPtr child_factory) { child_factories.emplace_back(child_factory); } + void addPathToHints(const std::string & http_path) { hints.add(http_path); } + std::unique_ptr createRequestHandler(const HTTPServerRequest & request) override; private: Poco::Logger * log; std::string name; + HTTPPathHints hints; std::vector child_factories; }; diff --git a/src/Server/NotFoundHandler.cpp b/src/Server/NotFoundHandler.cpp index 3181708b9b7..5b1db508551 100644 --- a/src/Server/NotFoundHandler.cpp +++ b/src/Server/NotFoundHandler.cpp @@ -10,7 +10,8 @@ void NotFoundHandler::handleRequest(HTTPServerRequest & request, HTTPServerRespo try { response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_NOT_FOUND); - *response.send() << "There is no handle " << request.getURI() << "\n\n" + *response.send() << "There is no handle " << request.getURI() + << (!hints.empty() ? fmt::format(". Maybe you meant {}.", hints.front()) : "") << "\n\n" << "Use / or /ping for health checks.\n" << "Or /replicas_status for more sophisticated health checks.\n\n" << "Send queries from your program with POST method or GET /?query=...\n\n" diff --git a/src/Server/NotFoundHandler.h b/src/Server/NotFoundHandler.h index 749ac388c4d..1cbfcd57f8f 100644 --- a/src/Server/NotFoundHandler.h +++ b/src/Server/NotFoundHandler.h @@ -9,7 +9,10 @@ namespace DB class NotFoundHandler : public HTTPRequestHandler { public: + NotFoundHandler(std::vector hints_) : hints(std::move(hints_)) {} void handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) override; +private: + std::vector hints; }; } diff --git a/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.reference b/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.reference new file mode 100644 index 00000000000..0025187be30 --- /dev/null +++ b/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.reference @@ -0,0 +1,4 @@ +There is no handle /sashboards. Maybe you meant /dashboard +There is no handle /sashboard. Maybe you meant /dashboard +There is no handle /sashboarb. Maybe you meant /dashboard +There is no handle /sashboaxb. Maybe you meant /dashboard diff --git a/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.sh b/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.sh new file mode 100755 index 00000000000..cf69c742777 --- /dev/null +++ b/tests/queries/0_stateless/02842_suggest_http_page_in_error_message.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +export CLICKHOUSE_URL="${CLICKHOUSE_PORT_HTTP_PROTO}://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTP}/" + +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}sashboards" | grep -o ".* Maybe you meant /dashboard" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}sashboard" | grep -o ".* Maybe you meant /dashboard" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}sashboarb" | grep -o ".* Maybe you meant /dashboard" +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}sashboaxb" | grep -o ".* Maybe you meant /dashboard" From a4701657695207ca6f93d3ef3d9379996b02b7ff Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Aug 2023 12:28:18 +0200 Subject: [PATCH 314/777] Better fs cache cleanup --- src/Interpreters/Cache/FileCache.cpp | 17 +-- src/Interpreters/Cache/FileCache_fwd.h | 1 + src/Interpreters/Cache/Metadata.cpp | 185 ++++++++++++++++--------- src/Interpreters/Cache/Metadata.h | 12 +- 4 files changed, 134 insertions(+), 81 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index c9bd39a0a2c..b7b36cebf83 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -823,23 +823,13 @@ bool FileCache::tryReserve(FileSegment & file_segment, const size_t size, FileCa void FileCache::removeKey(const Key & key) { assertInitialized(); - auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::THROW); - locked_key->removeAll(); + metadata.removeKey(key, /* if_exists */false, /* if_releasable */true); } void FileCache::removeKeyIfExists(const Key & key) { assertInitialized(); - - auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::RETURN_NULL); - if (!locked_key) - return; - - /// In ordinary case we remove data from cache when it's not used by anyone. - /// But if we have multiple replicated zero-copy tables on the same server - /// it became possible to start removing something from cache when it is used - /// by other "zero-copy" tables. That is why it's not an error. - locked_key->removeAll(/* if_releasable */true); + metadata.removeKey(key, /* if_exists */true, /* if_releasable */true); } void FileCache::removeFileSegment(const Key & key, size_t offset) @@ -857,8 +847,7 @@ void FileCache::removePathIfExists(const String & path) void FileCache::removeAllReleasable() { assertInitialized(); - - metadata.iterate([](LockedKey & locked_key) { locked_key.removeAll(/* if_releasable */true); }); + metadata.removeAllKeys(/* if_releasable */true); if (stash) { diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index 4d4a1c1429c..6cb2a53684d 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -11,6 +11,7 @@ static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 10000000; static constexpr int FILECACHE_DEFAULT_HITS_THRESHOLD = 0; static constexpr size_t FILECACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024; static constexpr size_t FILECACHE_DELAYED_CLEANUP_INTERVAL_MS = 1000 * 60; /// 1 min +static constexpr size_t FILECACHE_DELAYED_CLEANUP_BATCH_SIZE = 1000; class FileCache; using FileCachePtr = std::shared_ptr; diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 42fedc8aa3d..13a158ce35d 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -1,7 +1,6 @@ #include #include #include -#include "Common/Exception.h" #include #include #include @@ -89,15 +88,19 @@ LockedKeyPtr KeyMetadata::lock() LockedKeyPtr KeyMetadata::tryLock() { - ProfileEventTimeIncrement watch(ProfileEvents::FilesystemCacheLockKeyMicroseconds); - - auto locked = std::make_unique(shared_from_this()); + auto locked = lockNoStateCheck(); if (key_state == KeyMetadata::KeyState::ACTIVE) return locked; return nullptr; } +LockedKeyPtr KeyMetadata::lockNoStateCheck() +{ + ProfileEventTimeIncrement watch(ProfileEvents::FilesystemCacheLockKeyMicroseconds); + return std::make_unique(shared_from_this()); +} + bool KeyMetadata::createBaseDirectory() { if (!created_base_directory.exchange(true)) @@ -214,13 +217,9 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata( } { - LockedKeyPtr locked_metadata; - { - ProfileEventTimeIncrement watch(ProfileEvents::FilesystemCacheLockKeyMicroseconds); - locked_metadata = std::make_unique(key_metadata); - } - + auto locked_metadata = key_metadata->lockNoStateCheck(); const auto key_state = locked_metadata->getKeyState(); + if (key_state == KeyMetadata::KeyState::ACTIVE) return locked_metadata; @@ -249,17 +248,12 @@ LockedKeyPtr CacheMetadata::lockKeyMetadata( return lockKeyMetadata(key, key_not_found_policy); } -void CacheMetadata::iterate(IterateCacheMetadataFunc && func) +void CacheMetadata::iterate(IterateFunc && func) { auto lock = lockMetadata(); - for (const auto & [key, key_metadata] : *this) + for (auto & [key, key_metadata] : *this) { - LockedKeyPtr locked_key; - { - ProfileEventTimeIncrement watch(ProfileEvents::FilesystemCacheLockKeyMicroseconds); - locked_key = std::make_unique(key_metadata); - } - + auto locked_key = key_metadata->lockNoStateCheck(); const auto key_state = locked_key->getKeyState(); if (key_state == KeyMetadata::KeyState::ACTIVE) @@ -267,8 +261,7 @@ void CacheMetadata::iterate(IterateCacheMetadataFunc && func) func(*locked_key); continue; } - - if (key_state == KeyMetadata::KeyState::REMOVING) + else if (key_state == KeyMetadata::KeyState::REMOVING) continue; throw Exception( @@ -278,62 +271,123 @@ void CacheMetadata::iterate(IterateCacheMetadataFunc && func) void CacheMetadata::doCleanup() { + /// Firstly, this cleanup does not delete cache files, + /// but only empty keys from cache_metadata_map and key (prefix) directories from fs. + /// Secondly, it deletes those only if arised as a result of + /// (1) eviction in FileCache::tryReserve(); + /// (2) removal of cancelled non-downloaded file segments after FileSegment::complete(). + /// which does not include removal of cache files because of FileCache::removeKey/removeAllKeys, + /// triggered by removal of source files from objects storage. + /// E.g. number of elements submitted to background cleanup should remain low. + auto lock = lockMetadata(); + LOG_DEBUG(log, "Having {} keys to delete", cleanup_queue->getSize()); FileCacheKey cleanup_key; - while (cleanup_queue->tryPop(cleanup_key)) + size_t remaining_remove_num = FILECACHE_DELAYED_CLEANUP_BATCH_SIZE; + while (remaining_remove_num && cleanup_queue->tryPop(cleanup_key)) { auto it = find(cleanup_key); if (it == end()) continue; - LockedKeyPtr locked_metadata; + auto locked_key = it->second->lockNoStateCheck(); + if (locked_key->getKeyState() == KeyMetadata::KeyState::REMOVING) { - ProfileEventTimeIncrement watch(ProfileEvents::FilesystemCacheLockKeyMicroseconds); - locked_metadata = std::make_unique(it->second); - } - - const auto key_state = locked_metadata->getKeyState(); - if (key_state == KeyMetadata::KeyState::ACTIVE) - { - /// Key was added back to cache after we submitted it to removal queue. - continue; - } - - chassert(it->second->empty()); - locked_metadata->markAsRemoved(); - erase(it); - LOG_DEBUG(log, "Key {} is removed from metadata", cleanup_key); - - const fs::path key_directory = getPathForKey(cleanup_key); - const fs::path key_prefix_directory = key_directory.parent_path(); - - try - { - if (fs::exists(key_directory)) - fs::remove_all(key_directory); - } - catch (...) - { - LOG_ERROR(log, "Error while removing key {}: {}", cleanup_key, getCurrentExceptionMessage(true)); - chassert(false); - continue; - } - - 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 (...) - { - LOG_ERROR(log, "Error while removing key {}: {}", cleanup_key, getCurrentExceptionMessage(true)); - chassert(false); + removeKeyImpl(it, *locked_key, lock); + --remaining_remove_num; } } } +void CacheMetadata::removeAllKeys(bool if_releasable) +{ + auto lock = lockMetadata(); + for (auto it = begin(); it != end();) + { + auto locked_key = it->second->lockNoStateCheck(); + if (locked_key->getKeyState() == KeyMetadata::KeyState::ACTIVE) + { + bool removed_all = locked_key->removeAllFileSegments(if_releasable); + if (removed_all) + { + it = removeKeyImpl(it, *locked_key, lock); + continue; + } + } + ++it; + } +} + +void CacheMetadata::removeKey(const Key & key, bool if_exists, bool if_releasable) +{ + auto metadata_lock = lockMetadata(); + + auto it = find(key); + if (it == end()) + { + if (if_exists) + return; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key: {}", key); + } + + auto locked_key = it->second->lockNoStateCheck(); + if (locked_key->getKeyState() != KeyMetadata::KeyState::ACTIVE) + { + if (if_exists) + return; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key: {}", key); + } + + bool removed_all = locked_key->removeAllFileSegments(if_releasable); + if (removed_all) + removeKeyImpl(it, *locked_key, metadata_lock); +} + +CacheMetadata::iterator CacheMetadata::removeKeyImpl(iterator it, LockedKey & locked_key, const CacheMetadataGuard::Lock &) +{ + const auto & key = locked_key.getKey(); + + if (!it->second->empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot remove non-empty key: {}", key); + + locked_key.markAsRemoved(); + auto next_it = erase(it); + + LOG_DEBUG(log, "Key {} is removed from metadata", key); + + const fs::path key_directory = getPathForKey(key); + const fs::path key_prefix_directory = key_directory.parent_path(); + + try + { + if (fs::exists(key_directory)) + fs::remove_all(key_directory); + } + catch (...) + { + LOG_ERROR(log, "Error while removing key {}: {}", key, getCurrentExceptionMessage(true)); + chassert(false); + return next_it; + } + + 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 (...) + { + LOG_ERROR(log, "Error while removing key {}: {}", key, getCurrentExceptionMessage(true)); + chassert(false); + } + return next_it; +} + + class DownloadQueue { friend struct CacheMetadata; @@ -566,13 +620,15 @@ bool LockedKey::isLastOwnerOfFileSegment(size_t offset) const return file_segment_metadata->file_segment.use_count() == 2; } -void LockedKey::removeAll(bool if_releasable) +bool LockedKey::removeAllFileSegments(bool if_releasable) { + bool removed_all = true; for (auto it = key_metadata->begin(); it != key_metadata->end();) { if (if_releasable && !it->second->releasable()) { ++it; + removed_all = false; continue; } else if (it->second->evicting()) @@ -589,6 +645,7 @@ void LockedKey::removeAll(bool if_releasable) auto file_segment = it->second->file_segment; it = removeFileSegment(file_segment->offset(), file_segment->lock()); } + return removed_all; } KeyMetadata::iterator LockedKey::removeFileSegment(size_t offset) diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 57187f2715b..9e5fc20d9e2 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -69,6 +69,8 @@ struct KeyMetadata : public std::map, /// Return nullptr if key has non-ACTIVE state. LockedKeyPtr tryLock(); + LockedKeyPtr lockNoStateCheck(); + bool createBaseDirectory(); std::string getFileSegmentPath(const FileSegment & file_segment); @@ -90,7 +92,7 @@ struct CacheMetadata : public std::unordered_map, { public: using Key = FileCacheKey; - using IterateCacheMetadataFunc = std::function; + using IterateFunc = std::function; explicit CacheMetadata(const std::string & path_); @@ -104,7 +106,7 @@ public: String getPathForKey(const Key & key) const; static String getFileNameForFileSegment(size_t offset, FileSegmentKind segment_kind); - void iterate(IterateCacheMetadataFunc && func); + void iterate(IterateFunc && func); enum class KeyNotFoundPolicy { @@ -119,6 +121,9 @@ public: KeyNotFoundPolicy key_not_found_policy, bool is_initial_load = false); + void removeKey(const Key & key, bool if_exists, bool is_releasable); + void removeAllKeys(bool is_releasable); + void doCleanup(); void downloadThreadFunc(); @@ -135,6 +140,7 @@ private: Poco::Logger * log; void downloadImpl(FileSegment & file_segment, std::optional> & memory); + iterator removeKeyImpl(iterator it, LockedKey &, const CacheMetadataGuard::Lock &); }; @@ -174,7 +180,7 @@ struct LockedKey : private boost::noncopyable std::shared_ptr getKeyMetadata() const { return key_metadata; } std::shared_ptr getKeyMetadata() { return key_metadata; } - void removeAll(bool if_releasable = true); + bool removeAllFileSegments(bool if_releasable = true); KeyMetadata::iterator removeFileSegment(size_t offset, const FileSegmentGuard::Lock &); KeyMetadata::iterator removeFileSegment(size_t offset); From 0ff5d12788f1656f61c5b8df2a716675aef02f88 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 11:14:55 +0000 Subject: [PATCH 315/777] Added decription to the test + race condition fix --- .../test_profile_max_sessions_for_user/test.py | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 925fa05881d..78e201f88b9 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -214,7 +214,21 @@ def test_profile_max_sessions_for_user_setting_in_query(started_cluster): def test_profile_max_sessions_for_user_client_suggestions_connection(started_cluster): command_text = f"{started_cluster.get_client_cmd()} --host {instance.ip_address} --port 9000 -u {TEST_USER} --password {TEST_PASSWORD}" - with client(name="client1>", log=None, command=command_text) as client1: + command_text_without_suggestions = command_text + " --disable_suggestion" + + # Launch client1 without suggestions to avoid a race condition: + # Client1 opens a session. + # Client1 opens a session for suggestion connection. + # Client2 fails to open a session and gets the USER_SESSION_LIMIT_EXCEEDED error. + # + # Expected order: + # Client1 opens a session. + # Client2 opens a session. + # Client2 fails to open a session for suggestions and with USER_SESSION_LIMIT_EXCEEDED (No error printed). + # Client3 fails to open a session. + # Client1 executes the query. + # Client2 loads suggestions from the server using the main connection and executes a query. + with client(name="client1>", log=None, command=command_text_without_suggestions) as client1: client1.expect(prompt) with client(name="client2>", log=None, command=command_text) as client2: client2.expect(prompt) From 7ed7707ab7e6ccd6b2f26675f3349b29e703b442 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 10 Aug 2023 11:19:16 +0000 Subject: [PATCH 316/777] black run --- tests/integration/test_profile_max_sessions_for_user/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 78e201f88b9..c5c33b1cddb 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -228,7 +228,9 @@ def test_profile_max_sessions_for_user_client_suggestions_connection(started_clu # Client3 fails to open a session. # Client1 executes the query. # Client2 loads suggestions from the server using the main connection and executes a query. - with client(name="client1>", log=None, command=command_text_without_suggestions) as client1: + with client( + name="client1>", log=None, command=command_text_without_suggestions + ) as client1: client1.expect(prompt) with client(name="client2>", log=None, command=command_text) as client2: client2.expect(prompt) From bb38918a263dd59307c463bf038ebf0c4d28d184 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 10 Aug 2023 13:21:11 +0200 Subject: [PATCH 317/777] Apply suggestions from code review MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 2 +- src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp | 2 +- src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 9092c7fceba..52f9571f962 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -417,7 +417,7 @@ std::optional, DataTypes>> CSVSchemaReader::readRo auto fields = reader.readRow(); auto data_types = tryInferDataTypesByEscapingRule(fields, format_settings, FormatSettings::EscapingRule::CSV); - return std::make_pair(fields, data_types); + return std::make_pair(std::move(fields), std::move(data_types)); } std::optional CSVSchemaReader::readRowAndGetDataTypesImpl() diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp index ff3d6d49199..17cc88425f5 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp @@ -388,7 +388,7 @@ std::optional, DataTypes>> CustomSeparatedSchemaRe auto fields = reader.readRow(); auto data_types = tryInferDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule(), &json_inference_info); - return std::make_pair(fields, data_types); + return std::make_pair(std::move(fields), std::move(data_types)); } std::optional CustomSeparatedSchemaReader::readRowAndGetDataTypesImpl() diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index 4000bd14ddc..fc2b5cd8207 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -425,7 +425,7 @@ void FormatWithNamesAndTypesSchemaReader::tryDetectHeader(std::vector & if (!first_row) return; - auto [first_row_values, first_row_types] = *first_row; + const auto & [first_row_values, first_row_types] = *first_row; /// The first row contains non String elements, it cannot be a header. if (!checkIfAllTypesAreString(first_row_types)) @@ -443,7 +443,7 @@ void FormatWithNamesAndTypesSchemaReader::tryDetectHeader(std::vector & return; } - auto [second_row_values, second_row_types] = *second_row; + const auto & [second_row_values, second_row_types] = *second_row; DataTypes data_types; bool second_row_can_be_type_names = checkIfAllTypesAreString(second_row_types) && checkIfAllValuesAreTypeNames(readNamesFromFields(second_row_values)); From 6eb6c8a320c919ad33d5e9b1b5f7093eb071be1c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 10 Aug 2023 11:43:26 +0000 Subject: [PATCH 318/777] Fix: correct execution over cluster with multiple shards respect shard number during parallel replicas query execution --- .../ClusterProxy/executeQuery.cpp | 25 ++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index e4a48fdcac0..fbc7bbd5bbb 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -270,7 +270,28 @@ void executeQueryWithParallelReplicas( const ClusterPtr & not_optimized_cluster) { const auto & settings = context->getSettingsRef(); - ClusterPtr new_cluster = not_optimized_cluster->getClusterWithReplicasAsShards(settings); + auto new_context = Context::createCopy(context); + auto scalars = new_context->hasQueryContext() ? new_context->getQueryContext()->getScalars() : Scalars{}; + + Int64 shard_num = 0; /// shard_num is 1-based, so 0 - no shard specified + auto it = scalars.find("_shard_num"); + if (it != scalars.end()) + { + const Block & block = it->second; + shard_num = block.getColumns()[0]->get64(0); + } + + ClusterPtr new_cluster; + /// if got valid shard_num from query initiator, then parallel replicas scope is the specified shard + /// shards are numbered in order of appearance in the cluster config + if (shard_num > 0) + { + LOG_DEBUG(&Poco::Logger::get("executeQueryWithParallelReplicas"), "Parallel replicas query in shard scope: shard_num={}", shard_num); + /// shard_num is 1-based, but getClusterWithSingleShard expects 0-based index + new_cluster = not_optimized_cluster->getClusterWithSingleShard(shard_num - 1); + } + else + new_cluster = not_optimized_cluster->getClusterWithReplicasAsShards(settings); auto all_replicas_count = std::min(static_cast(settings.max_parallel_replicas), new_cluster->getShardCount()); auto coordinator = std::make_shared(all_replicas_count); @@ -284,8 +305,6 @@ void executeQueryWithParallelReplicas( /// to then tell it about the reading method we chose. query_info.coordinator = coordinator; - auto new_context = Context::createCopy(context); - auto scalars = new_context->hasQueryContext() ? new_context->getQueryContext()->getScalars() : Scalars{}; auto external_tables = new_context->getExternalTables(); auto read_from_remote = std::make_unique( From 4b30900fed3dcafa015a7d6dc4d6d91ea362a966 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Aug 2023 14:44:16 +0300 Subject: [PATCH 319/777] Revert "Improve CHECK TABLE system query" --- .../sql-reference/statements/check-table.md | 107 +++------- src/Common/FileChecker.cpp | 42 ++-- src/Common/FileChecker.h | 37 +--- src/Interpreters/InterpreterCheckQuery.cpp | 201 ++++-------------- src/Storages/CheckResults.h | 2 + src/Storages/IStorage.cpp | 11 - src/Storages/IStorage.h | 41 +--- src/Storages/StorageLog.cpp | 9 +- src/Storages/StorageLog.h | 16 +- src/Storages/StorageMergeTree.cpp | 26 +-- src/Storages/StorageMergeTree.h | 29 +-- src/Storages/StorageProxy.h | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 37 ++-- src/Storages/StorageReplicatedMergeTree.h | 31 +-- src/Storages/StorageStripeLog.cpp | 10 +- src/Storages/StorageStripeLog.h | 17 +- .../0_stateless/00063_check_query.reference | 1 - .../queries/0_stateless/00063_check_query.sql | 3 - .../queries/0_stateless/00961_check_table.sql | 12 +- ...1042_check_query_and_last_granule_size.sql | 12 +- ...02235_check_table_sparse_serialization.sql | 3 +- .../02841_check_table_progress.reference | 2 - .../0_stateless/02841_check_table_progress.sh | 29 --- 23 files changed, 138 insertions(+), 544 deletions(-) delete mode 100644 tests/queries/0_stateless/02841_check_table_progress.reference delete mode 100755 tests/queries/0_stateless/02841_check_table_progress.sh diff --git a/docs/en/sql-reference/statements/check-table.md b/docs/en/sql-reference/statements/check-table.md index db8c32249ef..0209d59b018 100644 --- a/docs/en/sql-reference/statements/check-table.md +++ b/docs/en/sql-reference/statements/check-table.md @@ -5,38 +5,19 @@ sidebar_label: CHECK TABLE title: "CHECK TABLE Statement" --- -The `CHECK TABLE` query in ClickHouse is used to perform a validation check on a specific table or its partitions. It ensures the integrity of the data by verifying the checksums and other internal data structures. +Checks if the data in the table is corrupted. -Particularly it compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution. - -:::note -The `CHECK TABLE`` query may read all the data in the table and hold some resources, making it resource-intensive. -Consider the potential impact on performance and resource utilization before executing this query. -::: - -## Syntax - -The basic syntax of the query is as follows: - -```sql -CHECK TABLE table_name [PARTITION partition_expression] [FORMAT format] [SETTINGS check_query_single_value_result = (0|1) [, other_settings] ] +``` sql +CHECK TABLE [db.]name [PARTITION partition_expr] ``` -- `table_name`: Specifies the name of the table that you want to check. -- `partition_expression`: (Optional) If you want to check a specific partition of the table, you can use this expression to specify the partition. -- `FORMAT format`: (Optional) Allows you to specify the output format of the result. -- `SETTINGS`: (Optional) Allows additional settings. - - **`check_query_single_value_result`**: (Optional) This setting allows you to toggle between a detailed result (`0`) or a summarized result (`1`). - - Other settings (e.g. `max_threads` can be applied as well). +The `CHECK TABLE` query compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution. +The query response contains the `result` column with a single row. The row has a value of +[Boolean](../../sql-reference/data-types/boolean.md) type: -The query response depends on the value of contains `check_query_single_value_result` setting. -In case of `check_query_single_value_result = 1` only `result` column with a single row is returned. Value inside this row is `1` if the integrity check is passed and `0` if data is corrupted. - -With `check_query_single_value_result = 0` the query returns the following columns: - - `part_path`: Indicates the path to the data part or file name. - - `is_passed`: Returns 1 if the check for this part is successful, 0 otherwise. - - `message`: Any additional messages related to the check, such as errors or success messages. +- 0 - The data in the table is corrupted. +- 1 - The data maintains integrity. The `CHECK TABLE` query supports the following table engines: @@ -45,15 +26,30 @@ The `CHECK TABLE` query supports the following table engines: - [StripeLog](../../engines/table-engines/log-family/stripelog.md) - [MergeTree family](../../engines/table-engines/mergetree-family/mergetree.md) -Performed over the tables with another table engines causes an `NOT_IMPLEMETED` exception. +Performed over the tables with another table engines causes an exception. Engines from the `*Log` family do not provide automatic data recovery on failure. Use the `CHECK TABLE` query to track data loss in a timely manner. -## Examples +## Checking the MergeTree Family Tables -By default `CHECK TABLE` query shows the general table check status: +For `MergeTree` family engines, if [check_query_single_value_result](../../operations/settings/settings.md#check_query_single_value_result) = 0, the `CHECK TABLE` query shows a check status for every individual data part of a table on the local server. ```sql +SET check_query_single_value_result = 0; +CHECK TABLE test_table; +``` + +```text +┌─part_path─┬─is_passed─┬─message─┐ +│ all_1_4_1 │ 1 │ │ +│ all_1_4_2 │ 1 │ │ +└───────────┴───────────┴─────────┘ +``` + +If `check_query_single_value_result` = 1, the `CHECK TABLE` query shows the general table check status. + +```sql +SET check_query_single_value_result = 1; CHECK TABLE test_table; ``` @@ -63,60 +59,11 @@ CHECK TABLE test_table; └────────┘ ``` -If you want to see the check status for every individual data part you may use `check_query_single_value_result` setting. - -Also, to check a specific partition of the table, you can use the `PARTITION` keyword. - -```sql -CHECK TABLE t0 PARTITION ID '201003' -FORMAT PrettyCompactMonoBlock -SETTINGS check_query_single_value_result = 0 -``` - -Output: - -```text -┌─part_path────┬─is_passed─┬─message─┐ -│ 201003_7_7_0 │ 1 │ │ -│ 201003_3_3_0 │ 1 │ │ -└──────────────┴───────────┴─────────┘ -``` - -### Receiving a 'Corrupted' Result - -:::warning -Disclaimer: The procedure described here, including the manual manipulating or removing files directly from the data directory, is for experimental or development environments only. Do **not** attempt this on a production server, as it may lead to data loss or other unintended consequences. -::: - -Remove the existing checksum file: - -```bash -rm /var/lib/clickhouse-server/data/default/t0/201003_3_3_0/checksums.txt -``` - -```sql -CHECK TABLE t0 PARTITION ID '201003' -FORMAT PrettyCompactMonoBlock -SETTINGS check_query_single_value_result = 0 - - -Output: - -```text -┌─part_path────┬─is_passed─┬─message──────────────────────────────────┐ -│ 201003_7_7_0 │ 1 │ │ -│ 201003_3_3_0 │ 1 │ Checksums recounted and written to disk. │ -└──────────────┴───────────┴──────────────────────────────────────────┘ -``` - -If the checksums.txt file is missing, it can be restored. It will be recalculated and rewritten during the execution of the CHECK TABLE command for the specific partition, and the status will still be reported as 'success.'" - - ## If the Data Is Corrupted If the table is corrupted, you can copy the non-corrupted data to another table. To do this: 1. Create a new table with the same structure as damaged table. To do this execute the query `CREATE TABLE AS `. -2. Set the `max_threads` value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`. +2. Set the [max_threads](../../operations/settings/settings.md#settings-max_threads) value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`. 3. Execute the query `INSERT INTO SELECT * FROM `. This request copies the non-corrupted data from the damaged table to another table. Only the data before the corrupted part will be copied. 4. Restart the `clickhouse-client` to reset the `max_threads` value. diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index 122ea83835d..876bc4e641c 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -82,35 +82,33 @@ size_t FileChecker::getTotalSize() const } -FileChecker::DataValidationTasksPtr FileChecker::getDataValidationTasks() +CheckResults FileChecker::check() const { - return std::make_unique(map); -} - -CheckResult FileChecker::checkNextEntry(DataValidationTasksPtr & check_data_tasks, bool & has_nothing_to_do) const -{ - String name; - size_t expected_size; - bool is_finished = check_data_tasks->next(name, expected_size); - if (is_finished) - { - has_nothing_to_do = true; + if (map.empty()) return {}; - } - String path = parentPath(files_info_path) + name; - bool exists = fileReallyExists(path); - auto real_size = exists ? getRealFileSize(path) : 0; /// No race condition assuming no one else is working with these files. + CheckResults results; - if (real_size != expected_size) + for (const auto & name_size : map) { - String failure_message = exists - ? ("Size of " + path + " is wrong. Size is " + toString(real_size) + " but should be " + toString(expected_size)) - : ("File " + path + " doesn't exist"); - return CheckResult(name, false, failure_message); + const String & name = name_size.first; + String path = parentPath(files_info_path) + name; + bool exists = fileReallyExists(path); + auto real_size = exists ? getRealFileSize(path) : 0; /// No race condition assuming no one else is working with these files. + + if (real_size != name_size.second) + { + String failure_message = exists + ? ("Size of " + path + " is wrong. Size is " + toString(real_size) + " but should be " + toString(name_size.second)) + : ("File " + path + " doesn't exist"); + results.emplace_back(name, false, failure_message); + break; + } + + results.emplace_back(name, true, ""); } - return CheckResult(name, true, ""); + return results; } void FileChecker::repair() diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index 8ffc310b84d..bb0383e4b56 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -3,7 +3,6 @@ #include #include #include -#include namespace Poco { class Logger; } @@ -29,11 +28,7 @@ public: bool empty() const { return map.empty(); } /// Check the files whose parameters are specified in sizes.json - /// See comment in IStorage::checkDataNext - struct DataValidationTasks; - using DataValidationTasksPtr = std::unique_ptr; - DataValidationTasksPtr getDataValidationTasks(); - CheckResult checkNextEntry(DataValidationTasksPtr & check_data_tasks, bool & has_nothing_to_do) const; + CheckResults check() const; /// Truncate files that have excessive size to the expected size. /// Throw exception if the file size is less than expected. @@ -46,36 +41,6 @@ public: /// Returns total size of all files. size_t getTotalSize() const; - struct DataValidationTasks - { - DataValidationTasks(const std::map & map_) - : map(map_), it(map.begin()) - {} - - bool next(String & out_name, size_t & out_size) - { - std::lock_guard lock(mutex); - if (it == map.end()) - return true; - out_name = it->first; - out_size = it->second; - ++it; - return false; - } - - size_t size() const - { - std::lock_guard lock(mutex); - return std::distance(it, map.end()); - } - - const std::map & map; - - mutable std::mutex mutex; - using Iterator = std::map::const_iterator; - Iterator it; - }; - private: void load(); diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index bd530654dd2..333aed84873 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -8,201 +8,72 @@ #include #include #include -#include #include -#include - -#include -#include - namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - namespace { -Block getSingleValueBlock(UInt8 value) +NamesAndTypes getBlockStructure() { - return Block{{ColumnUInt8::create(1, value), std::make_shared(), "result"}}; -} - -Block getHeaderForCheckResult() -{ - auto names_and_types = NamesAndTypes{ + return { {"part_path", std::make_shared()}, {"is_passed", std::make_shared()}, {"message", std::make_shared()}, }; - - return Block({ - {names_and_types[0].type->createColumn(), names_and_types[0].type, names_and_types[0].name}, - {names_and_types[1].type->createColumn(), names_and_types[1].type, names_and_types[1].name}, - {names_and_types[2].type->createColumn(), names_and_types[2].type, names_and_types[2].name}, - }); } -Chunk getChunkFromCheckResult(const CheckResult & check_result) -{ - MutableColumns columns = getHeaderForCheckResult().cloneEmptyColumns(); - columns[0]->insert(check_result.fs_path); - columns[1]->insert(static_cast(check_result.success)); - columns[2]->insert(check_result.failure_message); - return Chunk(std::move(columns), 1); -} - -class TableCheckWorkerProcessor : public ISource -{ - -public: - TableCheckWorkerProcessor(IStorage::DataValidationTasksPtr check_data_tasks_, StoragePtr table_) - : ISource(getHeaderForCheckResult()) - , table(table_) - , check_data_tasks(check_data_tasks_) - { - } - - String getName() const override { return "TableCheckWorkerProcessor"; } - -protected: - - std::optional tryGenerate() override - { - bool has_nothing_to_do = false; - auto check_result = table->checkDataNext(check_data_tasks, has_nothing_to_do); - if (has_nothing_to_do) - return {}; - - /// We can omit manual `progess` call, ISource will may count it automatically by returned chunk - /// However, we want to report only rows in progress - progress(1, 0); - - if (!check_result.success) - { - LOG_WARNING(&Poco::Logger::get("InterpreterCheckQuery"), - "Check query for table {} failed, path {}, reason: {}", - table->getStorageID().getNameForLogs(), - check_result.fs_path, - check_result.failure_message); - } - - return getChunkFromCheckResult(check_result); - } - -private: - StoragePtr table; - IStorage::DataValidationTasksPtr check_data_tasks; -}; - -class TableCheckResultEmitter : public IAccumulatingTransform -{ -public: - TableCheckResultEmitter() : IAccumulatingTransform(getHeaderForCheckResult(), getSingleValueBlock(1).cloneEmpty()) {} - - String getName() const override { return "TableCheckResultEmitter"; } - - void consume(Chunk chunk) override - { - if (result_value == 0) - return; - - auto columns = chunk.getColumns(); - if (columns.size() != 3) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong number of columns: {}", columns.size()); - - const auto * col = checkAndGetColumn(columns[1].get()); - for (size_t i = 0; i < col->size(); ++i) - { - if (col->getElement(i) == 0) - { - result_value = 0; - return; - } - } - } - - Chunk generate() override - { - if (is_value_emitted.exchange(true)) - return {}; - auto block = getSingleValueBlock(result_value); - return Chunk(block.getColumns(), block.rows()); - } - -private: - std::atomic result_value{1}; - std::atomic_bool is_value_emitted{false}; -}; - } -InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_) - : WithContext(context_) - , query_ptr(query_ptr_) + +InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) { } + BlockIO InterpreterCheckQuery::execute() { const auto & check = query_ptr->as(); - const auto & context = getContext(); - auto table_id = context->resolveStorageID(check, Context::ResolveOrdinary); + auto table_id = getContext()->resolveStorageID(check, Context::ResolveOrdinary); - context->checkAccess(AccessType::SHOW_TABLES, table_id); - StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); + getContext()->checkAccess(AccessType::SHOW_TABLES, table_id); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext()); + auto check_results = table->checkData(query_ptr, getContext()); - auto check_data_tasks = table->getCheckTaskList(query_ptr, context); + Block block; + if (getContext()->getSettingsRef().check_query_single_value_result) + { + bool result = std::all_of(check_results.begin(), check_results.end(), [] (const CheckResult & res) { return res.success; }); + auto column = ColumnUInt8::create(); + column->insertValue(static_cast(result)); + block = Block{{std::move(column), std::make_shared(), "result"}}; + } + else + { + auto block_structure = getBlockStructure(); + auto path_column = block_structure[0].type->createColumn(); + auto is_passed_column = block_structure[1].type->createColumn(); + auto message_column = block_structure[2].type->createColumn(); - const auto & settings = context->getSettingsRef(); + for (const auto & check_result : check_results) + { + path_column->insert(check_result.fs_path); + is_passed_column->insert(static_cast(check_result.success)); + message_column->insert(check_result.failure_message); + } + + block = Block({ + {std::move(path_column), block_structure[0].type, block_structure[0].name}, + {std::move(is_passed_column), block_structure[1].type, block_structure[1].name}, + {std::move(message_column), block_structure[2].type, block_structure[2].name}}); + } BlockIO res; - { - auto processors = std::make_shared(); + res.pipeline = QueryPipeline(std::make_shared(std::move(block))); - std::vector worker_ports; - - size_t num_streams = std::max(1, settings.max_threads); - - for (size_t i = 0; i < num_streams; ++i) - { - auto worker_processor = std::make_shared(check_data_tasks, table); - if (i == 0) - worker_processor->addTotalRowsApprox(check_data_tasks->size()); - worker_ports.emplace_back(&worker_processor->getPort()); - processors->emplace_back(worker_processor); - } - - OutputPort * resize_outport; - { - auto resize_processor = std::make_shared(getHeaderForCheckResult(), worker_ports.size(), 1); - - auto & resize_inputs = resize_processor->getInputs(); - auto resize_inport_it = resize_inputs.begin(); - for (size_t i = 0; i < worker_ports.size(); ++i, ++resize_inport_it) - connect(*worker_ports[i], *resize_inport_it); - - resize_outport = &resize_processor->getOutputs().front(); - processors->emplace_back(resize_processor); - } - - if (settings.check_query_single_value_result) - { - auto emitter_processor = std::make_shared(); - auto * input_port = &emitter_processor->getInputPort(); - processors->emplace_back(emitter_processor); - - connect(*resize_outport, *input_port); - } - - res.pipeline = QueryPipeline(Pipe(std::move(processors))); - res.pipeline.setNumThreads(num_streams); - } return res; } diff --git a/src/Storages/CheckResults.h b/src/Storages/CheckResults.h index 2e4652fea29..b342b014fa4 100644 --- a/src/Storages/CheckResults.h +++ b/src/Storages/CheckResults.h @@ -22,4 +22,6 @@ struct CheckResult {} }; +using CheckResults = std::vector; + } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 85299f63165..ae7659e074f 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -273,17 +273,6 @@ bool IStorage::isStaticStorage() const return false; } -IStorage::DataValidationTasksPtr IStorage::getCheckTaskList(const ASTPtr & /* query */, ContextPtr /* context */) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Check query is not supported for {} storage", getName()); -} - -CheckResult IStorage::checkDataNext(DataValidationTasksPtr & /* check_task_list */, bool & has_nothing_to_do) -{ - has_nothing_to_do = true; - return {}; -} - void IStorage::adjustCreateQueryForBackup(ASTPtr &) const { } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index b1e20c55782..ec92f57aeda 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -595,45 +595,8 @@ public: /// Provides a hint that the storage engine may evaluate the IN-condition by using an index. virtual bool mayBenefitFromIndexForIn(const ASTPtr & /* left_in_operand */, ContextPtr /* query_context */, const StorageMetadataPtr & /* metadata_snapshot */) const { return false; } - - /** A list of tasks to check a validity of data. - * Each IStorage implementation may interpret this task in its own way. - * E.g. for some storages it to check data it need to check a list of files in filesystem, for others it can be a list of parts. - * Also it may hold resources (e.g. locks) required during check. - */ - struct DataValidationTasksBase - { - /// Number of entries left to check. - /// It decreases after each call to checkDataNext(). - virtual size_t size() const = 0; - virtual ~DataValidationTasksBase() = default; - }; - - using DataValidationTasksPtr = std::shared_ptr; - - virtual DataValidationTasksPtr getCheckTaskList(const ASTPtr & /* query */, ContextPtr /* context */); - - /** Executes one task from the list. - * If no tasks left, sets has_nothing_to_do to true. - * Note: Function `checkDataNext` is accessing `check_task_list` thread-safely, - * and can be called simultaneously for the same `getCheckTaskList` result - * to process different tasks in parallel. - * Usage: - * - * auto check_task_list = storage.getCheckTaskList(query, context); - * size_t total_tasks = check_task_list->size(); - * while (true) - * { - * size_t tasks_left = check_task_list->size(); - * std::cout << "Checking data: " << (total_tasks - tasks_left) << " / " << total_tasks << " tasks done." << std::endl; - * bool has_nothing_to_do = false; - * auto result = storage.checkDataNext(check_task_list, has_nothing_to_do); - * if (has_nothing_to_do) - * break; - * doSomething(result); - * } - */ - virtual CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do); + /// Checks validity of the data + virtual CheckResults checkData(const ASTPtr & /* query */, ContextPtr /* context */) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Check query is not supported for {} storage", getName()); } /// Checks that table could be dropped right now /// Otherwise - throws an exception with detailed information. diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 0e9f83e886a..87aa71f3e8d 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -866,18 +866,15 @@ SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetada return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -IStorage::DataValidationTasksPtr StorageLog::getCheckTaskList(const ASTPtr & /* query */, ContextPtr local_context) +CheckResults StorageLog::checkData(const ASTPtr & /* query */, ContextPtr local_context) { ReadLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded"); - return std::make_unique(file_checker.getDataValidationTasks(), std::move(lock)); + + return file_checker.check(); } -CheckResult StorageLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) -{ - return file_checker.checkNextEntry(assert_cast(check_task_list.get())->file_checker_tasks, has_nothing_to_do); -} IStorage::ColumnSizeByName StorageLog::getColumnSizes() const { diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 95f95088aa2..f1d05ed39ac 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -59,8 +59,7 @@ public: void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + CheckResults checkData(const ASTPtr & query, ContextPtr local_context) override; void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; @@ -143,19 +142,6 @@ private: std::atomic total_rows = 0; std::atomic total_bytes = 0; - struct DataValidationTasks : public IStorage::DataValidationTasksBase - { - DataValidationTasks(FileChecker::DataValidationTasksPtr file_checker_tasks_, ReadLock && lock_) - : file_checker_tasks(std::move(file_checker_tasks_)), lock(std::move(lock_)) - {} - - size_t size() const override { return file_checker_tasks->size(); } - - FileChecker::DataValidationTasksPtr file_checker_tasks; - /// Lock to prevent table modification while checking - ReadLock lock; - }; - FileChecker file_checker; const size_t max_compress_block_size; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 97fc7a6731f..ad9013d9f13 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2197,8 +2197,9 @@ void StorageMergeTree::onActionLockRemove(StorageActionBlockType action_type) background_moves_assignee.trigger(); } -IStorage::DataValidationTasksPtr StorageMergeTree::getCheckTaskList(const ASTPtr & query, ContextPtr local_context) +CheckResults StorageMergeTree::checkData(const ASTPtr & query, ContextPtr local_context) { + CheckResults results; DataPartsVector data_parts; if (const auto & check_query = query->as(); check_query.partition) { @@ -2208,14 +2209,7 @@ IStorage::DataValidationTasksPtr StorageMergeTree::getCheckTaskList(const ASTPtr else data_parts = getVisibleDataPartsVector(local_context); - return std::make_unique(std::move(data_parts), local_context); -} - -CheckResult StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) -{ - auto * data_validation_tasks = assert_cast(check_task_list.get()); - auto local_context = data_validation_tasks->context; - if (auto part = data_validation_tasks->next()) + for (auto & part : data_parts) { /// If the checksums file is not present, calculate the checksums and write them to disk. static constexpr auto checksums_path = "checksums.txt"; @@ -2230,12 +2224,12 @@ CheckResult StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_ part_mutable.writeChecksums(part->checksums, local_context->getWriteSettings()); part->checkMetadata(); - return CheckResult(part->name, true, "Checksums recounted and written to disk."); + results.emplace_back(part->name, true, "Checksums recounted and written to disk."); } catch (const Exception & ex) { tryLogCurrentException(log, __PRETTY_FUNCTION__); - return CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); } } else @@ -2244,19 +2238,15 @@ CheckResult StorageMergeTree::checkDataNext(DataValidationTasksPtr & check_task_ { checkDataPart(part, true); part->checkMetadata(); - return CheckResult(part->name, true, ""); + results.emplace_back(part->name, true, ""); } catch (const Exception & ex) { - return CheckResult(part->name, false, ex.message()); + results.emplace_back(part->name, false, ex.message()); } } } - else - { - has_nothing_to_do = true; - return {}; - } + return results; } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index ec4796e4941..c77e5140d75 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -108,8 +108,7 @@ public: void onActionLockRemove(StorageActionBlockType action_type) override; - DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + CheckResults checkData(const ASTPtr & query, ContextPtr context) override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) override; @@ -279,32 +278,6 @@ private: friend class MergePlainMergeTreeTask; friend class MutatePlainMergeTreeTask; - struct DataValidationTasks : public IStorage::DataValidationTasksBase - { - DataValidationTasks(DataPartsVector && parts_, ContextPtr context_) - : parts(std::move(parts_)), it(parts.begin()), context(std::move(context_)) - {} - - DataPartPtr next() - { - std::lock_guard lock(mutex); - if (it == parts.end()) - return nullptr; - return *(it++); - } - - size_t size() const override - { - std::lock_guard lock(mutex); - return std::distance(it, parts.end()); - } - - mutable std::mutex mutex; - DataPartsVector parts; - DataPartsVector::const_iterator it; - - ContextPtr context; - }; protected: std::map getAlterMutationCommandsForPart(const DataPartPtr & part) const override; diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index ea908bea032..21ed4b91c62 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -149,9 +149,7 @@ public: return getNested()->mayBenefitFromIndexForIn(left_in_operand, query_context, metadata_snapshot); } - DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override { return getNested()->getCheckTaskList(query, context); } - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override { return getNested()->checkDataNext(check_task_list, has_nothing_to_do); } - + CheckResults checkData(const ASTPtr & query, ContextPtr context) override { return getNested()->checkData(query, context); } void checkTableCanBeDropped() const override { getNested()->checkTableCanBeDropped(); } bool storesDataOnDisk() const override { return getNested()->storesDataOnDisk(); } Strings getDataPaths() const override { return getNested()->getDataPaths(); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c08f1ebcc48..7fce373e26b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -8481,8 +8481,9 @@ void StorageReplicatedMergeTree::enqueuePartForCheck(const String & part_name, t part_check_thread.enqueuePart(part_name, delay_to_check_seconds); } -IStorage::DataValidationTasksPtr StorageReplicatedMergeTree::getCheckTaskList(const ASTPtr & query, ContextPtr local_context) +CheckResults StorageReplicatedMergeTree::checkData(const ASTPtr & query, ContextPtr local_context) { + CheckResults results; DataPartsVector data_parts; if (const auto & check_query = query->as(); check_query.partition) { @@ -8492,30 +8493,24 @@ IStorage::DataValidationTasksPtr StorageReplicatedMergeTree::getCheckTaskList(co else data_parts = getVisibleDataPartsVector(local_context); - auto part_check_lock = part_check_thread.pausePartsCheck(); - return std::make_unique(std::move(data_parts), std::move(part_check_lock)); -} - -CheckResult StorageReplicatedMergeTree::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) -{ - - if (auto part = assert_cast(check_task_list.get())->next()) { - try + auto part_check_lock = part_check_thread.pausePartsCheck(); + + for (auto & part : data_parts) { - return CheckResult(part_check_thread.checkPartAndFix(part->name)); - } - catch (const Exception & ex) - { - tryLogCurrentException(log, __PRETTY_FUNCTION__); - return CheckResult(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + try + { + results.push_back(part_check_thread.checkPartAndFix(part->name)); + } + catch (const Exception & ex) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); + } } } - else - { - has_nothing_to_do = true; - return {}; - } + + return results; } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 2bc18aa3b0a..78ef39f032f 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -230,8 +230,7 @@ public: /// Add a part to the queue of parts whose data you want to check in the background thread. void enqueuePartForCheck(const String & part_name, time_t delay_to_check_seconds = 0); - DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + CheckResults checkData(const ASTPtr & query, ContextPtr context) override; /// Checks ability to use granularity bool canUseAdaptiveGranularity() const override; @@ -991,34 +990,6 @@ private: bool waitZeroCopyLockToDisappear(const ZeroCopyLock & lock, size_t milliseconds_to_wait) override; void startupImpl(bool from_attach_thread); - - struct DataValidationTasks : public IStorage::DataValidationTasksBase - { - explicit DataValidationTasks(DataPartsVector && parts_, std::unique_lock && parts_check_lock_) - : parts_check_lock(std::move(parts_check_lock_)), parts(std::move(parts_)), it(parts.begin()) - {} - - DataPartPtr next() - { - std::lock_guard lock(mutex); - if (it == parts.end()) - return nullptr; - return *(it++); - } - - size_t size() const override - { - std::lock_guard lock(mutex); - return std::distance(it, parts.end()); - } - - std::unique_lock parts_check_lock; - - mutable std::mutex mutex; - DataPartsVector parts; - DataPartsVector::const_iterator it; - }; - }; String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index a3cbff96199..0bfef5ed5e5 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -403,18 +403,16 @@ SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const Storage return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -IStorage::DataValidationTasksPtr StorageStripeLog::getCheckTaskList(const ASTPtr & /* query */, ContextPtr local_context) + +CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr local_context) { ReadLock lock{rwlock, getLockTimeout(local_context)}; if (!lock) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded"); - return std::make_unique(file_checker.getDataValidationTasks(), std::move(lock)); + + return file_checker.check(); } -CheckResult StorageStripeLog::checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) -{ - return file_checker.checkNextEntry(assert_cast(check_task_list.get())->file_checker_tasks, has_nothing_to_do); -} void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 5d4e2fcbd3a..f889a1de71b 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -53,8 +53,7 @@ public: void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - DataValidationTasksPtr getCheckTaskList(const ASTPtr & query, ContextPtr context) override; - CheckResult checkDataNext(DataValidationTasksPtr & check_task_list, bool & has_nothing_to_do) override; + CheckResults checkData(const ASTPtr & query, ContextPtr ocal_context) override; bool storesDataOnDisk() const override { return true; } Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } @@ -94,20 +93,6 @@ private: const DiskPtr disk; String table_path; - struct DataValidationTasks : public IStorage::DataValidationTasksBase - { - DataValidationTasks(FileChecker::DataValidationTasksPtr file_checker_tasks_, ReadLock && lock_) - : file_checker_tasks(std::move(file_checker_tasks_)), lock(std::move(lock_)) - {} - - size_t size() const override { return file_checker_tasks->size(); } - - FileChecker::DataValidationTasksPtr file_checker_tasks; - - /// Lock to prevent table modification while checking - ReadLock lock; - }; - String data_file_path; String index_file_path; FileChecker file_checker; diff --git a/tests/queries/0_stateless/00063_check_query.reference b/tests/queries/0_stateless/00063_check_query.reference index e8183f05f5d..6ed281c757a 100644 --- a/tests/queries/0_stateless/00063_check_query.reference +++ b/tests/queries/0_stateless/00063_check_query.reference @@ -1,3 +1,2 @@ 1 1 -1 diff --git a/tests/queries/0_stateless/00063_check_query.sql b/tests/queries/0_stateless/00063_check_query.sql index 263cf94fb4a..e7362074a05 100644 --- a/tests/queries/0_stateless/00063_check_query.sql +++ b/tests/queries/0_stateless/00063_check_query.sql @@ -8,9 +8,6 @@ INSERT INTO check_query_tiny_log VALUES (1, 'A'), (2, 'B'), (3, 'C'); CHECK TABLE check_query_tiny_log; --- Settings and FORMAT are supported -CHECK TABLE check_query_tiny_log SETTINGS max_threads = 16; -CHECK TABLE check_query_tiny_log FORMAT Null SETTINGS max_threads = 8, check_query_single_value_result = 0; DROP TABLE IF EXISTS check_query_log; diff --git a/tests/queries/0_stateless/00961_check_table.sql b/tests/queries/0_stateless/00961_check_table.sql index 079acc8cdbb..0e0b2c3b483 100644 --- a/tests/queries/0_stateless/00961_check_table.sql +++ b/tests/queries/0_stateless/00961_check_table.sql @@ -3,29 +3,29 @@ DROP TABLE IF EXISTS mt_table; CREATE TABLE mt_table (d Date, key UInt64, data String) ENGINE = MergeTree() PARTITION BY toYYYYMM(d) ORDER BY key; -CHECK TABLE mt_table SETTINGS max_threads = 1; +CHECK TABLE mt_table; INSERT INTO mt_table VALUES (toDate('2019-01-02'), 1, 'Hello'), (toDate('2019-01-02'), 2, 'World'); -CHECK TABLE mt_table SETTINGS max_threads = 1; +CHECK TABLE mt_table; INSERT INTO mt_table VALUES (toDate('2019-01-02'), 3, 'quick'), (toDate('2019-01-02'), 4, 'brown'); SELECT '========'; -CHECK TABLE mt_table SETTINGS max_threads = 1; +CHECK TABLE mt_table; OPTIMIZE TABLE mt_table FINAL; SELECT '========'; -CHECK TABLE mt_table SETTINGS max_threads = 1; +CHECK TABLE mt_table; SELECT '========'; INSERT INTO mt_table VALUES (toDate('2019-02-03'), 5, '!'), (toDate('2019-02-03'), 6, '?'); -CHECK TABLE mt_table SETTINGS max_threads = 1; +CHECK TABLE mt_table; SELECT '========'; @@ -33,6 +33,6 @@ INSERT INTO mt_table VALUES (toDate('2019-02-03'), 7, 'jump'), (toDate('2019-02- OPTIMIZE TABLE mt_table FINAL; -CHECK TABLE mt_table PARTITION 201902 SETTINGS max_threads = 1; +CHECK TABLE mt_table PARTITION 201902; DROP TABLE IF EXISTS mt_table; diff --git a/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql b/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql index eccb2d25878..b66aff8384d 100644 --- a/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql +++ b/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql @@ -7,11 +7,11 @@ CREATE TABLE check_query_test (SomeKey UInt64, SomeValue String) ENGINE = MergeT -- Rows in this table are short, so granularity will be 8192. INSERT INTO check_query_test SELECT number, toString(number) FROM system.numbers LIMIT 81920; -CHECK TABLE check_query_test SETTINGS max_threads = 1; +CHECK TABLE check_query_test; OPTIMIZE TABLE check_query_test; -CHECK TABLE check_query_test SETTINGS max_threads = 1; +CHECK TABLE check_query_test; DROP TABLE IF EXISTS check_query_test; @@ -21,18 +21,18 @@ CREATE TABLE check_query_test_non_adaptive (SomeKey UInt64, SomeValue String) EN INSERT INTO check_query_test_non_adaptive SELECT number, toString(number) FROM system.numbers LIMIT 81920; -CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; +CHECK TABLE check_query_test_non_adaptive; OPTIMIZE TABLE check_query_test_non_adaptive; -CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; +CHECK TABLE check_query_test_non_adaptive; INSERT INTO check_query_test_non_adaptive SELECT number, toString(number) FROM system.numbers LIMIT 77; -CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; +CHECK TABLE check_query_test_non_adaptive; OPTIMIZE TABLE check_query_test_non_adaptive; -CHECK TABLE check_query_test_non_adaptive SETTINGS max_threads = 1; +CHECK TABLE check_query_test_non_adaptive; DROP TABLE IF EXISTS check_query_test_non_adaptive; diff --git a/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql b/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql index 625be63e0c0..0ac97404c46 100644 --- a/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql +++ b/tests/queries/0_stateless/02235_check_table_sparse_serialization.sql @@ -12,6 +12,7 @@ SELECT name, column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_sparse_02235' ORDER BY name, column; -CHECK TABLE t_sparse_02235 SETTINGS check_query_single_value_result = 0, max_threads = 1; +SET check_query_single_value_result = 0; +CHECK TABLE t_sparse_02235; DROP TABLE t_sparse_02235; diff --git a/tests/queries/0_stateless/02841_check_table_progress.reference b/tests/queries/0_stateless/02841_check_table_progress.reference deleted file mode 100644 index 541dab48def..00000000000 --- a/tests/queries/0_stateless/02841_check_table_progress.reference +++ /dev/null @@ -1,2 +0,0 @@ -Ok -Ok diff --git a/tests/queries/0_stateless/02841_check_table_progress.sh b/tests/queries/0_stateless/02841_check_table_progress.sh deleted file mode 100755 index 166386b999b..00000000000 --- a/tests/queries/0_stateless/02841_check_table_progress.sh +++ /dev/null @@ -1,29 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t0"; -${CLICKHOUSE_CLIENT} -q "CREATE TABLE t0 (x UInt64, val String) ENGINE = MergeTree ORDER BY x PARTITION BY x % 100"; -${CLICKHOUSE_CLIENT} -q "INSERT INTO t0 SELECT sipHash64(number), randomPrintableASCII(1000) FROM numbers(1000)"; - - -# Check that we have at least 3 different values for read_rows -UNIQUE_VALUES=$( - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d @- <<< "CHECK TABLE t0" -v |& { - grep -F -e X-ClickHouse-Progress: -e X-ClickHouse-Summary: | grep -o '"read_rows"\s*:\s*"[0-9]*"' - } | uniq | wc -l -) - -[ "$UNIQUE_VALUES" -ge "3" ] && echo "Ok" || echo "Fail: got $UNIQUE_VALUES" - - -# Check that we have we have at least 100 total_rows_to_read (at least one check task per partition) -MAX_TOTAL_VALUE=$( - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&send_progress_in_http_headers=1&http_headers_progress_interval_ms=0" -d @- <<< "CHECK TABLE t0" -v |& { - grep -F -e X-ClickHouse-Progress: -e X-ClickHouse-Summary: | grep -o '"total_rows_to_read"\s*:\s*"[0-9]*"' | grep -o '[0-9]*' - } | sort -n | tail -1 -) - -[ "$MAX_TOTAL_VALUE" -ge "100" ] && echo "Ok" || echo "Fail: got $MAX_TOTAL_VALUE" From 82aff97dd04605233371c9c6de1e59933961cb78 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 10 Aug 2023 11:51:36 +0000 Subject: [PATCH 320/777] Add comment, more test --- src/Processors/Formats/RowInputFormatWithNamesAndTypes.h | 1 + ...2834_formats_with_variable_number_of_columns.reference | 8 ++++++++ .../02834_formats_with_variable_number_of_columns.sql | 2 ++ 3 files changed, 11 insertions(+) diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index 7b3e2cbea67..377341da685 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -119,6 +119,7 @@ public: /// Check suffix. virtual bool checkForSuffix() { return in->eof(); } + /// Check if we are at the end of row, not between fields. virtual bool checkForEndOfRow() { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method checkForEndOfRow is not implemented"); } virtual bool allowVariableNumberOfColumns() const { return false; } diff --git a/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.reference b/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.reference index e9ff548e05c..50173c150c0 100644 --- a/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.reference +++ b/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.reference @@ -37,6 +37,14 @@ JSONCompactEachRow 2 0 0 0 3 3 +1 1 +2 0 +0 0 +3 3 +1 [1,2,3] +2 [] +0 [] +3 [3] 1 1 \N \N 2 \N \N \N \N \N \N \N diff --git a/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.sql b/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.sql index dea4c20db8a..7c55cf2e9a7 100644 --- a/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.sql +++ b/tests/queries/0_stateless/02834_formats_with_variable_number_of_columns.sql @@ -10,6 +10,8 @@ select * from format(TSVWithNames, 'x\ty\n1\t1\n2\n\n3\t3\t3\t3') settings input select * from format(TSVWithNames, 'x UInt32, z UInt32', 'x\ty\n1\t1\n2\n\n3\t3\t3\t3') settings input_format_tsv_allow_variable_number_of_columns=1; select 'JSONCompactEachRow'; select * from format(JSONCompactEachRow, 'x UInt32, y UInt32', '[1,1]\n[2]\n[]\n[3,3,3,3]') settings input_format_json_compact_allow_variable_number_of_columns=1; +select * from format(JSONCompactEachRow, 'x UInt32, y UInt32', '[1,1,[1,2,3]]\n[2]\n[]\n[3,3,3,3,[1,2,3]]') settings input_format_json_compact_allow_variable_number_of_columns=1; +select * from format(JSONCompactEachRow, 'x UInt32, y Array(UInt32)', '[1,[1,2,3],1]\n[2]\n[]\n[3,[3],3,3,[1,2,3]]') settings input_format_json_compact_allow_variable_number_of_columns=1; select * from format(JSONCompactEachRow, '[1,1]\n[2]\n[]\n[3,3,3,3]') settings input_format_json_compact_allow_variable_number_of_columns=1; select * from format(JSONCompactEachRowWithNames, '["x","y"]\n[1,1]\n[2]\n[]\n[3,3,3,3]') settings input_format_json_compact_allow_variable_number_of_columns=1; select * from format(JSONCompactEachRowWithNames, 'x UInt32, z UInt32', '["x","y"]\n[1,1]\n[2]\n[]\n[3,3,3,3]') settings input_format_json_compact_allow_variable_number_of_columns=1; From 48fe9605a8b0720ed6fad7240c73005cb78b7a26 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Aug 2023 13:56:13 +0200 Subject: [PATCH 321/777] Better --- src/Common/CurrentMetrics.cpp | 1 + src/Interpreters/Cache/FileCache.cpp | 31 +--- src/Interpreters/Cache/FileCache.h | 10 +- src/Interpreters/Cache/FileCacheSettings.cpp | 2 - src/Interpreters/Cache/FileCacheSettings.h | 1 - src/Interpreters/Cache/FileCache_fwd.h | 2 - src/Interpreters/Cache/Metadata.cpp | 159 +++++++++--------- src/Interpreters/Cache/Metadata.h | 12 +- .../InterpreterDescribeCacheQuery.cpp | 2 - .../tests/gtest_lru_file_cache.cpp | 3 - 10 files changed, 100 insertions(+), 123 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 970919d1514..b3dee29e02f 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -194,6 +194,7 @@ M(FilesystemCacheSizeLimit, "Filesystem cache size limit in bytes") \ M(FilesystemCacheElements, "Filesystem cache elements (file segments)") \ M(FilesystemCacheDownloadQueueElements, "Filesystem cache elements in download queue") \ + M(FilesystemCacheDelayedCleanupElements, "Filesystem cache elements in background cleanup queue") \ M(AsyncInsertCacheSize, "Number of async insert hash id in cache") \ M(S3Requests, "S3 requests") \ M(KeeperAliveConnections, "Number of alive connections") \ diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index b7b36cebf83..2a445cffff9 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -10,6 +10,7 @@ #include #include #include +#include "Common/ThreadPool_fwd.h" #include #include #include @@ -54,7 +55,6 @@ namespace ErrorCodes FileCache::FileCache(const FileCacheSettings & settings) : max_file_segment_size(settings.max_file_segment_size) , bypass_cache_threshold(settings.enable_bypass_cache_with_threashold ? settings.bypass_cache_threashold : 0) - , delayed_cleanup_interval_ms(settings.delayed_cleanup_interval_ms) , boundary_alignment(settings.boundary_alignment) , background_download_threads(settings.background_download_threads) , log(&Poco::Logger::get("FileCache")) @@ -134,9 +134,7 @@ void FileCache::initialize() for (size_t i = 0; i < background_download_threads; ++i) download_threads.emplace_back([this] { metadata.downloadThreadFunc(); }); - cleanup_task = Context::getGlobalContextInstance()->getSchedulePool().createTask("FileCacheCleanup", [this]{ cleanupThreadFunc(); }); - cleanup_task->activate(); - cleanup_task->scheduleAfter(delayed_cleanup_interval_ms); + cleanup_thread = std::make_unique(std::function{ [this]{ metadata.cleanupThreadFunc(); }}); } CacheGuard::Lock FileCache::lockCache() const @@ -1028,33 +1026,14 @@ FileCache::~FileCache() void FileCache::deactivateBackgroundOperations() { - if (cleanup_task) - cleanup_task->deactivate(); - metadata.cancelDownload(); for (auto & thread : download_threads) if (thread.joinable()) thread.join(); -} -void FileCache::cleanup() -{ - metadata.doCleanup(); -} - -void FileCache::cleanupThreadFunc() -{ - try - { - cleanup(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - chassert(false); - } - - cleanup_task->scheduleAfter(delayed_cleanup_interval_ms); + metadata.cancelCleanup(); + if (cleanup_thread && cleanup_thread->joinable()) + cleanup_thread->join(); } FileSegmentsHolderPtr FileCache::getSnapshot() diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index d020f6d35f7..de0923bdbd1 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -12,7 +12,7 @@ #include -#include +#include #include #include #include @@ -130,8 +130,6 @@ public: FileSegmentsHolderPtr dumpQueue(); - void cleanup(); - void deactivateBackgroundOperations(); /// For per query cache limit. @@ -157,7 +155,6 @@ private: const size_t max_file_segment_size; const size_t bypass_cache_threshold = 0; - const size_t delayed_cleanup_interval_ms; const size_t boundary_alignment; const size_t background_download_threads; @@ -202,9 +199,8 @@ private: * A background cleanup task. * Clears removed cache entries from metadata. */ - BackgroundSchedulePool::TaskHolder cleanup_task; - std::vector download_threads; + std::unique_ptr cleanup_thread; void assertInitialized() const; @@ -235,8 +231,6 @@ private: FileSegment::State state, const CreateFileSegmentSettings & create_settings, const CacheGuard::Lock *); - - void cleanupThreadFunc(); }; } diff --git a/src/Interpreters/Cache/FileCacheSettings.cpp b/src/Interpreters/Cache/FileCacheSettings.cpp index 455e9b44d0b..426bbbc6ee9 100644 --- a/src/Interpreters/Cache/FileCacheSettings.cpp +++ b/src/Interpreters/Cache/FileCacheSettings.cpp @@ -49,8 +49,6 @@ void FileCacheSettings::loadFromConfig(const Poco::Util::AbstractConfiguration & if (config.has(config_prefix + ".background_download_threads")) background_download_threads = config.getUInt(config_prefix + ".background_download_threads"); - - delayed_cleanup_interval_ms = config.getUInt64(config_prefix + ".delayed_cleanup_interval_ms", FILECACHE_DELAYED_CLEANUP_INTERVAL_MS); } } diff --git a/src/Interpreters/Cache/FileCacheSettings.h b/src/Interpreters/Cache/FileCacheSettings.h index e56d6fcc54d..fab3e8744f6 100644 --- a/src/Interpreters/Cache/FileCacheSettings.h +++ b/src/Interpreters/Cache/FileCacheSettings.h @@ -24,7 +24,6 @@ struct FileCacheSettings bool enable_bypass_cache_with_threashold = false; size_t bypass_cache_threashold = FILECACHE_BYPASS_THRESHOLD; - size_t delayed_cleanup_interval_ms = FILECACHE_DELAYED_CLEANUP_INTERVAL_MS; size_t boundary_alignment = FILECACHE_DEFAULT_FILE_SEGMENT_ALIGNMENT; size_t background_download_threads = FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_THREADS; diff --git a/src/Interpreters/Cache/FileCache_fwd.h b/src/Interpreters/Cache/FileCache_fwd.h index 6cb2a53684d..939cb676e3a 100644 --- a/src/Interpreters/Cache/FileCache_fwd.h +++ b/src/Interpreters/Cache/FileCache_fwd.h @@ -10,8 +10,6 @@ static constexpr int FILECACHE_DEFAULT_BACKGROUND_DOWNLOAD_THREADS = 2; static constexpr int FILECACHE_DEFAULT_MAX_ELEMENTS = 10000000; static constexpr int FILECACHE_DEFAULT_HITS_THRESHOLD = 0; static constexpr size_t FILECACHE_BYPASS_THRESHOLD = 256 * 1024 * 1024; -static constexpr size_t FILECACHE_DELAYED_CLEANUP_INTERVAL_MS = 1000 * 60; /// 1 min -static constexpr size_t FILECACHE_DELAYED_CLEANUP_BATCH_SIZE = 1000; class FileCache; using FileCachePtr = std::shared_ptr; diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 13a158ce35d..8fcb59c0259 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -10,6 +10,7 @@ namespace fs = std::filesystem; namespace CurrentMetrics { extern const Metric FilesystemCacheDownloadQueueElements; + extern const Metric FilesystemCacheDelayedCleanupElements; } namespace ProfileEvents @@ -134,22 +135,6 @@ std::string KeyMetadata::getFileSegmentPath(const FileSegment & file_segment) } -class CleanupQueue -{ - friend struct CacheMetadata; -public: - void add(const FileCacheKey & key); - void remove(const FileCacheKey & key); - size_t getSize() const; - -private: - bool tryPop(FileCacheKey & key); - - std::unordered_set keys; - mutable std::mutex mutex; -}; - - CacheMetadata::CacheMetadata(const std::string & path_) : path(path_) , cleanup_queue(std::make_unique()) @@ -269,37 +254,6 @@ void CacheMetadata::iterate(IterateFunc && func) } } -void CacheMetadata::doCleanup() -{ - /// Firstly, this cleanup does not delete cache files, - /// but only empty keys from cache_metadata_map and key (prefix) directories from fs. - /// Secondly, it deletes those only if arised as a result of - /// (1) eviction in FileCache::tryReserve(); - /// (2) removal of cancelled non-downloaded file segments after FileSegment::complete(). - /// which does not include removal of cache files because of FileCache::removeKey/removeAllKeys, - /// triggered by removal of source files from objects storage. - /// E.g. number of elements submitted to background cleanup should remain low. - - auto lock = lockMetadata(); - LOG_DEBUG(log, "Having {} keys to delete", cleanup_queue->getSize()); - - FileCacheKey cleanup_key; - size_t remaining_remove_num = FILECACHE_DELAYED_CLEANUP_BATCH_SIZE; - while (remaining_remove_num && cleanup_queue->tryPop(cleanup_key)) - { - auto it = find(cleanup_key); - if (it == end()) - continue; - - auto locked_key = it->second->lockNoStateCheck(); - if (locked_key->getKeyState() == KeyMetadata::KeyState::REMOVING) - { - removeKeyImpl(it, *locked_key, lock); - --remaining_remove_num; - } - } -} - void CacheMetadata::removeAllKeys(bool if_releasable) { auto lock = lockMetadata(); @@ -387,6 +341,86 @@ CacheMetadata::iterator CacheMetadata::removeKeyImpl(iterator it, LockedKey & lo return next_it; } +class CleanupQueue +{ + friend struct CacheMetadata; +public: + void add(const FileCacheKey & key) + { + { + std::lock_guard lock(mutex); + keys.insert(key); + } + CurrentMetrics::add(CurrentMetrics::FilesystemCacheDelayedCleanupElements); + cv.notify_one(); + } + + void cancel() + { + { + std::lock_guard lock(mutex); + cancelled = true; + } + cv.notify_all(); + } + +private: + std::unordered_set keys; + mutable std::mutex mutex; + std::condition_variable cv; + bool cancelled = false; +}; + +void CacheMetadata::cleanupThreadFunc() +{ + while (true) + { + Key key; + { + std::unique_lock lock(cleanup_queue->mutex); + + if (cleanup_queue->cancelled) + return; + + auto & keys = cleanup_queue->keys; + if (keys.empty()) + { + cleanup_queue->cv.wait(lock); + continue; + } + + auto it = keys.begin(); + key = *it; + keys.erase(it); + } + + CurrentMetrics::sub(CurrentMetrics::FilesystemCacheDelayedCleanupElements); + + try + { + auto lock = lockMetadata(); + + auto it = find(key); + if (it == end()) + continue; + + auto locked_key = it->second->lockNoStateCheck(); + if (locked_key->getKeyState() == KeyMetadata::KeyState::REMOVING) + { + removeKeyImpl(it, *locked_key, lock); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } +} + +void CacheMetadata::cancelCleanup() +{ + cleanup_queue->cancel(); +} class DownloadQueue { @@ -814,35 +848,4 @@ std::string LockedKey::toString() const return result; } -void CleanupQueue::add(const FileCacheKey & key) -{ - std::lock_guard lock(mutex); - keys.insert(key); -} - -void CleanupQueue::remove(const FileCacheKey & key) -{ - std::lock_guard lock(mutex); - bool erased = keys.erase(key); - if (!erased) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key {} in removal queue", key); -} - -bool CleanupQueue::tryPop(FileCacheKey & key) -{ - std::lock_guard lock(mutex); - if (keys.empty()) - return false; - auto it = keys.begin(); - key = *it; - keys.erase(it); - return true; -} - -size_t CleanupQueue::getSize() const -{ - std::lock_guard lock(mutex); - return keys.size(); -} - } diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 9e5fc20d9e2..caf6563aa9d 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -124,7 +124,17 @@ public: void removeKey(const Key & key, bool if_exists, bool is_releasable); void removeAllKeys(bool is_releasable); - void doCleanup(); + void cancelCleanup(); + + /// Firstly, this cleanup does not delete cache files, + /// but only empty keys from cache_metadata_map and key (prefix) directories from fs. + /// Secondly, it deletes those only if arised as a result of + /// (1) eviction in FileCache::tryReserve(); + /// (2) removal of cancelled non-downloaded file segments after FileSegment::complete(). + /// which does not include removal of cache files because of FileCache::removeKey/removeAllKeys, + /// triggered by removal of source files from objects storage. + /// E.g. number of elements submitted to background cleanup should remain low. + void cleanupThreadFunc(); void downloadThreadFunc(); diff --git a/src/Interpreters/InterpreterDescribeCacheQuery.cpp b/src/Interpreters/InterpreterDescribeCacheQuery.cpp index 7822ecdb8be..f9c61afb4d8 100644 --- a/src/Interpreters/InterpreterDescribeCacheQuery.cpp +++ b/src/Interpreters/InterpreterDescribeCacheQuery.cpp @@ -25,7 +25,6 @@ static Block getSampleBlock() ColumnWithTypeAndName{std::make_shared(), "current_size"}, ColumnWithTypeAndName{std::make_shared(), "current_elements"}, ColumnWithTypeAndName{std::make_shared(), "path"}, - ColumnWithTypeAndName{std::make_shared>(), "delayed_cleanup_interval_ms"}, ColumnWithTypeAndName{std::make_shared>(), "background_download_threads"}, ColumnWithTypeAndName{std::make_shared>(), "enable_bypass_cache_with_threshold"}, }; @@ -54,7 +53,6 @@ BlockIO InterpreterDescribeCacheQuery::execute() res_columns[i++]->insert(cache->getUsedCacheSize()); res_columns[i++]->insert(cache->getFileSegmentsNum()); res_columns[i++]->insert(cache->getBasePath()); - res_columns[i++]->insert(settings.delayed_cleanup_interval_ms); res_columns[i++]->insert(settings.background_download_threads); res_columns[i++]->insert(settings.enable_bypass_cache_with_threashold); diff --git a/src/Interpreters/tests/gtest_lru_file_cache.cpp b/src/Interpreters/tests/gtest_lru_file_cache.cpp index dab14a66ed7..cbaf3733464 100644 --- a/src/Interpreters/tests/gtest_lru_file_cache.cpp +++ b/src/Interpreters/tests/gtest_lru_file_cache.cpp @@ -604,7 +604,6 @@ TEST_F(FileCacheTest, get) auto cache = FileCache(settings); cache.initialize(); - cache.cleanup(); const auto key = cache.createKeyForPath("key10"); const auto key_path = cache.getPathInLocalCache(key); @@ -622,7 +621,6 @@ TEST_F(FileCacheTest, get) ASSERT_TRUE(fs::exists(key_path)); ASSERT_TRUE(!fs::exists(cache.getPathInLocalCache(key, 0, FileSegmentKind::Regular))); - cache.cleanup(); ASSERT_TRUE(!fs::exists(key_path)); ASSERT_TRUE(!fs::exists(fs::path(key_path).parent_path())); } @@ -632,7 +630,6 @@ TEST_F(FileCacheTest, get) /// Test background thread delated cleanup auto settings2{settings}; - settings2.delayed_cleanup_interval_ms = 0; auto cache = DB::FileCache(settings2); cache.initialize(); const auto key = cache.createKeyForPath("key10"); From 3c26d51220be909991d7690873a3faa09d3e360a Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Aug 2023 14:13:26 +0200 Subject: [PATCH 322/777] Better name --- src/Interpreters/Cache/Metadata.cpp | 8 ++++---- src/Interpreters/Cache/Metadata.h | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 8fcb59c0259..62110e487dc 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -265,7 +265,7 @@ void CacheMetadata::removeAllKeys(bool if_releasable) bool removed_all = locked_key->removeAllFileSegments(if_releasable); if (removed_all) { - it = removeKeyImpl(it, *locked_key, lock); + it = removeEmptyKey(it, *locked_key, lock); continue; } } @@ -297,10 +297,10 @@ void CacheMetadata::removeKey(const Key & key, bool if_exists, bool if_releasabl bool removed_all = locked_key->removeAllFileSegments(if_releasable); if (removed_all) - removeKeyImpl(it, *locked_key, metadata_lock); + removeEmptyKey(it, *locked_key, metadata_lock); } -CacheMetadata::iterator CacheMetadata::removeKeyImpl(iterator it, LockedKey & locked_key, const CacheMetadataGuard::Lock &) +CacheMetadata::iterator CacheMetadata::removeEmptyKey(iterator it, LockedKey & locked_key, const CacheMetadataGuard::Lock &) { const auto & key = locked_key.getKey(); @@ -407,7 +407,7 @@ void CacheMetadata::cleanupThreadFunc() auto locked_key = it->second->lockNoStateCheck(); if (locked_key->getKeyState() == KeyMetadata::KeyState::REMOVING) { - removeKeyImpl(it, *locked_key, lock); + removeEmptyKey(it, *locked_key, lock); } } catch (...) diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index caf6563aa9d..041318c7210 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -128,7 +128,7 @@ public: /// Firstly, this cleanup does not delete cache files, /// but only empty keys from cache_metadata_map and key (prefix) directories from fs. - /// Secondly, it deletes those only if arised as a result of + /// Secondly, it deletes those only if arose as a result of /// (1) eviction in FileCache::tryReserve(); /// (2) removal of cancelled non-downloaded file segments after FileSegment::complete(). /// which does not include removal of cache files because of FileCache::removeKey/removeAllKeys, @@ -150,7 +150,7 @@ private: Poco::Logger * log; void downloadImpl(FileSegment & file_segment, std::optional> & memory); - iterator removeKeyImpl(iterator it, LockedKey &, const CacheMetadataGuard::Lock &); + iterator removeEmptyKey(iterator it, LockedKey &, const CacheMetadataGuard::Lock &); }; From 1377d86ed9d6aaf17878b8c7d2960a0053b1111d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 17:01:09 +0300 Subject: [PATCH 323/777] Update src/Functions/array/arrayAUC.cpp MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- src/Functions/array/arrayAUC.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index caf929ba038..b7bd7dcc0ad 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -127,8 +127,8 @@ private: } static void vector( - const IColumn & data1, - const IColumn & data2, + const IColumn & scores, + const IColumn & labels, const ColumnArray::Offsets & offsets, PaddedPODArray & result) { From 635e7e74a86b80519544167eeb2d771a612d6a34 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Aug 2023 16:52:57 +0200 Subject: [PATCH 324/777] add garbage --- .../test_s3_zero_copy_ttl/__init__.py | 0 .../configs/max_delayed_streams.xml | 9 ++ .../test_s3_zero_copy_ttl/configs/s3.xml | 39 ++++++++ .../integration/test_s3_zero_copy_ttl/test.py | 94 +++++++++++++++++++ .../test_vertical_merge_memory_usage.py | 46 +++++++++ 5 files changed, 188 insertions(+) create mode 100644 tests/integration/test_s3_zero_copy_ttl/__init__.py create mode 100644 tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml create mode 100644 tests/integration/test_s3_zero_copy_ttl/configs/s3.xml create mode 100644 tests/integration/test_s3_zero_copy_ttl/test.py create mode 100644 tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py diff --git a/tests/integration/test_s3_zero_copy_ttl/__init__.py b/tests/integration/test_s3_zero_copy_ttl/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml b/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml new file mode 100644 index 00000000000..54f7152690b --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/configs/max_delayed_streams.xml @@ -0,0 +1,9 @@ + + + + + + 10 + + + diff --git a/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml b/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml new file mode 100644 index 00000000000..7bb7fa875e4 --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/configs/s3.xml @@ -0,0 +1,39 @@ + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + + + + +
+ default +
+ + s3_disk + +
+
+ + +
+ s3_disk +
+
+
+
+
+ + + true + 1.0 + + + true +
diff --git a/tests/integration/test_s3_zero_copy_ttl/test.py b/tests/integration/test_s3_zero_copy_ttl/test.py new file mode 100644 index 00000000000..04bff4a44fb --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/test.py @@ -0,0 +1,94 @@ +#!/usr/bin/env python3 +import time + +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True +) +node2 = cluster.add_instance( + "node2", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True +) +node3 = cluster.add_instance( + "node3", main_configs=["configs/s3.xml"], with_minio=True, with_zookeeper=True +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_ttl_move_and_s3(started_cluster): + for i, node in enumerate([node1, node2, node3]): + node.query( + """ + CREATE TABLE s3_test_with_ttl (date DateTime, id UInt32, value String) + ENGINE=ReplicatedMergeTree('/clickhouse/tables/s3_test', '{}') + ORDER BY id + PARTITION BY id + TTL date TO DISK 's3_disk' + SETTINGS storage_policy='s3_and_default', temporary_directories_lifetime=1 + """.format( + i + ) + ) + + node1.query("SYSTEM STOP MOVES s3_test_with_ttl") + + node2.query("SYSTEM STOP MOVES s3_test_with_ttl") + + for i in range(30): + if i % 2 == 0: + node = node1 + else: + node = node2 + + node.query( + f"INSERT INTO s3_test_with_ttl SELECT now() + 5, {i}, randomPrintableASCII(1048570)" + ) + + node1.query("SYSTEM SYNC REPLICA s3_test_with_ttl") + node2.query("SYSTEM SYNC REPLICA s3_test_with_ttl") + node3.query("SYSTEM SYNC REPLICA s3_test_with_ttl") + + assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + + node1.query("SYSTEM START MOVES s3_test_with_ttl") + node2.query("SYSTEM START MOVES s3_test_with_ttl") + + assert node1.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + assert node2.query("SELECT COUNT() FROM s3_test_with_ttl") == "30\n" + + for attempt in reversed(range(5)): + time.sleep(5) + + print( + node1.query( + "SELECT * FROM system.parts WHERE table = 's3_test_with_ttl' FORMAT Vertical" + ) + ) + + minio = cluster.minio_client + objects = minio.list_objects(cluster.minio_bucket, "data/", recursive=True) + counter = 0 + for obj in objects: + print(f"Objectname: {obj.object_name}, metadata: {obj.metadata}") + counter += 1 + + print(f"Total objects: {counter}") + + if counter == 330: + break + + print(f"Attempts remaining: {attempt}") + + assert counter == 330 diff --git a/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py b/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py new file mode 100644 index 00000000000..fb9f3eb67b9 --- /dev/null +++ b/tests/integration/test_s3_zero_copy_ttl/test_vertical_merge_memory_usage.py @@ -0,0 +1,46 @@ +#!/usr/bin/env python3 +import time + +import pytest +from helpers.cluster import ClickHouseCluster + + +single_node_cluster = ClickHouseCluster(__file__) +small_node = single_node_cluster.add_instance( + "small_node", + main_configs=["configs/s3.xml"], + user_configs=["configs/max_delayed_streams.xml"], + with_minio=True, +) + + +@pytest.fixture(scope="module") +def started_single_node_cluster(): + try: + single_node_cluster.start() + + yield single_node_cluster + finally: + single_node_cluster.shutdown() + + +def test_vertical_merge_memory_usage(started_single_node_cluster): + if small_node.is_built_with_sanitizer() or small_node.is_debug_build(): + pytest.skip("Disabled for debug and sanitizers. Too slow.") + + small_node.query( + "create table tvm2 (c0 UInt64, c1 UInt64, c2 UInt64, c3 UInt64, c4 UInt64, c5 UInt64, c6 UInt64, c7 UInt64, c8 UInt64, c9 UInt64, c10 UInt64, c11 UInt64, c12 UInt64, c13 UInt64, c14 UInt64, c15 UInt64, c16 UInt64, c17 UInt64, c18 UInt64, c19 UInt64, c20 UInt64, c21 UInt64, c22 UInt64, c23 UInt64, c24 UInt64, c25 UInt64, c26 UInt64, c27 UInt64, c28 UInt64, c29 UInt64, c30 UInt64, c31 UInt64, c32 UInt64, c33 UInt64, c34 UInt64, c35 UInt64, c36 UInt64, c37 UInt64, c38 UInt64, c39 UInt64, c40 UInt64, c41 UInt64, c42 UInt64, c43 UInt64, c44 UInt64, c45 UInt64, c46 UInt64, c47 UInt64, c48 UInt64, c49 UInt64, c50 UInt64, c51 UInt64, c52 UInt64, c53 UInt64, c54 UInt64, c55 UInt64, c56 UInt64, c57 UInt64, c58 UInt64, c59 UInt64, c60 UInt64, c61 UInt64, c62 UInt64, c63 UInt64, c64 UInt64, c65 UInt64, c66 UInt64, c67 UInt64, c68 UInt64, c69 UInt64, c70 UInt64, c71 UInt64, c72 UInt64, c73 UInt64, c74 UInt64, c75 UInt64, c76 UInt64, c77 UInt64, c78 UInt64, c79 UInt64, c80 UInt64, c81 UInt64, c82 UInt64, c83 UInt64, c84 UInt64, c85 UInt64, c86 UInt64, c87 UInt64, c88 UInt64, c89 UInt64, c90 UInt64, c91 UInt64, c92 UInt64, c93 UInt64, c94 UInt64, c95 UInt64, c96 UInt64, c97 UInt64, c98 UInt64, c99 UInt64, c100 UInt64, c101 UInt64, c102 UInt64, c103 UInt64, c104 UInt64, c105 UInt64, c106 UInt64, c107 UInt64, c108 UInt64, c109 UInt64, c110 UInt64, c111 UInt64, c112 UInt64, c113 UInt64, c114 UInt64, c115 UInt64, c116 UInt64, c117 UInt64, c118 UInt64, c119 UInt64, c120 UInt64, c121 UInt64, c122 UInt64, c123 UInt64, c124 UInt64, c125 UInt64, c126 UInt64, c127 UInt64, c128 UInt64, c129 UInt64, c130 UInt64, c131 UInt64, c132 UInt64, c133 UInt64, c134 UInt64, c135 UInt64, c136 UInt64, c137 UInt64, c138 UInt64, c139 UInt64, c140 UInt64, c141 UInt64, c142 UInt64, c143 UInt64, c144 UInt64, c145 UInt64, c146 UInt64, c147 UInt64, c148 UInt64, c149 UInt64, c150 UInt64, c151 UInt64, c152 UInt64, c153 UInt64, c154 UInt64, c155 UInt64, c156 UInt64, c157 UInt64, c158 UInt64, c159 UInt64, c160 UInt64, c161 UInt64, c162 UInt64, c163 UInt64, c164 UInt64, c165 UInt64, c166 UInt64, c167 UInt64, c168 UInt64, c169 UInt64, c170 UInt64, c171 UInt64, c172 UInt64, c173 UInt64, c174 UInt64, c175 UInt64, c176 UInt64, c177 UInt64, c178 UInt64, c179 UInt64, c180 UInt64, c181 UInt64, c182 UInt64, c183 UInt64, c184 UInt64, c185 UInt64, c186 UInt64, c187 UInt64, c188 UInt64, c189 UInt64, c190 UInt64, c191 UInt64, c192 UInt64, c193 UInt64, c194 UInt64, c195 UInt64, c196 UInt64, c197 UInt64, c198 UInt64, c199 UInt64, c200 UInt64, c201 UInt64, c202 UInt64, c203 UInt64, c204 UInt64, c205 UInt64, c206 UInt64, c207 UInt64, c208 UInt64, c209 UInt64, c210 UInt64, c211 UInt64, c212 UInt64, c213 UInt64, c214 UInt64, c215 UInt64, c216 UInt64, c217 UInt64, c218 UInt64, c219 UInt64, c220 UInt64, c221 UInt64, c222 UInt64, c223 UInt64, c224 UInt64, c225 UInt64, c226 UInt64, c227 UInt64, c228 UInt64, c229 UInt64, c230 UInt64, c231 UInt64, c232 UInt64, c233 UInt64, c234 UInt64, c235 UInt64, c236 UInt64, c237 UInt64, c238 UInt64, c239 UInt64, c240 UInt64, c241 UInt64, c242 UInt64, c243 UInt64, c244 UInt64, c245 UInt64, c246 UInt64, c247 UInt64, c248 UInt64, c249 UInt64, c250 UInt64, c251 UInt64, c252 UInt64, c253 UInt64, c254 UInt64, c255 UInt64, c256 UInt64, c257 UInt64, c258 UInt64, c259 UInt64, c260 UInt64, c261 UInt64, c262 UInt64, c263 UInt64, c264 UInt64, c265 UInt64, c266 UInt64, c267 UInt64, c268 UInt64, c269 UInt64, c270 UInt64, c271 UInt64, c272 UInt64, c273 UInt64, c274 UInt64, c275 UInt64, c276 UInt64, c277 UInt64, c278 UInt64, c279 UInt64, c280 UInt64, c281 UInt64, c282 UInt64, c283 UInt64, c284 UInt64, c285 UInt64, c286 UInt64, c287 UInt64, c288 UInt64, c289 UInt64, c290 UInt64, c291 UInt64, c292 UInt64, c293 UInt64, c294 UInt64, c295 UInt64, c296 UInt64, c297 UInt64, c298 UInt64, c299 UInt64) engine = MergeTree order by tuple() settings min_rows_for_wide_part = 10, min_bytes_for_wide_part=0, storage_policy = 's3', vertical_merge_algorithm_min_rows_to_activate=1" + ) + + small_node.query( + "insert into tvm2 select number + 0, number + 1, number + 2, number + 3, number + 4, number + 5, number + 6, number + 7, number + 8, number + 9, number + 10, number + 11, number + 12, number + 13, number + 14, number + 15, number + 16, number + 17, number + 18, number + 19, number + 20, number + 21, number + 22, number + 23, number + 24, number + 25, number + 26, number + 27, number + 28, number + 29, number + 30, number + 31, number + 32, number + 33, number + 34, number + 35, number + 36, number + 37, number + 38, number + 39, number + 40, number + 41, number + 42, number + 43, number + 44, number + 45, number + 46, number + 47, number + 48, number + 49, number + 50, number + 51, number + 52, number + 53, number + 54, number + 55, number + 56, number + 57, number + 58, number + 59, number + 60, number + 61, number + 62, number + 63, number + 64, number + 65, number + 66, number + 67, number + 68, number + 69, number + 70, number + 71, number + 72, number + 73, number + 74, number + 75, number + 76, number + 77, number + 78, number + 79, number + 80, number + 81, number + 82, number + 83, number + 84, number + 85, number + 86, number + 87, number + 88, number + 89, number + 90, number + 91, number + 92, number + 93, number + 94, number + 95, number + 96, number + 97, number + 98, number + 99, number + 100, number + 101, number + 102, number + 103, number + 104, number + 105, number + 106, number + 107, number + 108, number + 109, number + 110, number + 111, number + 112, number + 113, number + 114, number + 115, number + 116, number + 117, number + 118, number + 119, number + 120, number + 121, number + 122, number + 123, number + 124, number + 125, number + 126, number + 127, number + 128, number + 129, number + 130, number + 131, number + 132, number + 133, number + 134, number + 135, number + 136, number + 137, number + 138, number + 139, number + 140, number + 141, number + 142, number + 143, number + 144, number + 145, number + 146, number + 147, number + 148, number + 149, number + 150, number + 151, number + 152, number + 153, number + 154, number + 155, number + 156, number + 157, number + 158, number + 159, number + 160, number + 161, number + 162, number + 163, number + 164, number + 165, number + 166, number + 167, number + 168, number + 169, number + 170, number + 171, number + 172, number + 173, number + 174, number + 175, number + 176, number + 177, number + 178, number + 179, number + 180, number + 181, number + 182, number + 183, number + 184, number + 185, number + 186, number + 187, number + 188, number + 189, number + 190, number + 191, number + 192, number + 193, number + 194, number + 195, number + 196, number + 197, number + 198, number + 199, number + 200, number + 201, number + 202, number + 203, number + 204, number + 205, number + 206, number + 207, number + 208, number + 209, number + 210, number + 211, number + 212, number + 213, number + 214, number + 215, number + 216, number + 217, number + 218, number + 219, number + 220, number + 221, number + 222, number + 223, number + 224, number + 225, number + 226, number + 227, number + 228, number + 229, number + 230, number + 231, number + 232, number + 233, number + 234, number + 235, number + 236, number + 237, number + 238, number + 239, number + 240, number + 241, number + 242, number + 243, number + 244, number + 245, number + 246, number + 247, number + 248, number + 249, number + 250, number + 251, number + 252, number + 253, number + 254, number + 255, number + 256, number + 257, number + 258, number + 259, number + 260, number + 261, number + 262, number + 263, number + 264, number + 265, number + 266, number + 267, number + 268, number + 269, number + 270, number + 271, number + 272, number + 273, number + 274, number + 275, number + 276, number + 277, number + 278, number + 279, number + 280, number + 281, number + 282, number + 283, number + 284, number + 285, number + 286, number + 287, number + 288, number + 289, number + 290, number + 291, number + 292, number + 293, number + 294, number + 295, number + 296, number + 297, number + 298, number + 299 from numbers(20)" + ) + small_node.query("optimize table tvm2 final") + small_node.query("system flush logs") + + # Should be about 25M + res = small_node.query( + "select formatReadableSize(peak_memory_usage), * from system.part_log where table = 'tvm2' and database = currentDatabase() and event_date >= today() - 1 and event_type = 'MergeParts' and peak_memory_usage > 100000000 format Vertical" + ) + + assert res == "" From 708fd914bc708345fcc46fadd19b47eceddc786f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 5 Aug 2023 13:47:57 +0200 Subject: [PATCH 325/777] Fix 02263_format_insert_settings flakiness I guess the problem was with the async nature of the process substitution ("2> >(cmd)"), let's avoid using this feature of bash. CI: https://s3.amazonaws.com/clickhouse-test-reports/52683/b98cb7fa145d1a92c2c78421be1eeb8fe8353d53/stateless_tests__aarch64_.html Signed-off-by: Azat Khuzhin Co-authored-by: Alexey Milovidov Update 02263_format_insert_settings.sh --- .../02263_format_insert_settings.reference | 6 +++--- .../0_stateless/02263_format_insert_settings.sh | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02263_format_insert_settings.reference b/tests/queries/0_stateless/02263_format_insert_settings.reference index 721e7960875..e2d1ec3980e 100644 --- a/tests/queries/0_stateless/02263_format_insert_settings.reference +++ b/tests/queries/0_stateless/02263_format_insert_settings.reference @@ -1,6 +1,6 @@ -insert into foo settings max_threads=1 +[multi] insert into foo settings max_threads=1 Syntax error (query): failed at position 40 (end of query): -insert into foo format tsv settings max_threads=1 +[multi] insert into foo format tsv settings max_threads=1 Can't format ASTInsertQuery with data, since data will be lost. [multi] insert into foo format tsv settings max_threads=1 INSERT INTO foo @@ -8,7 +8,7 @@ SETTINGS max_threads = 1 FORMAT tsv [oneline] insert into foo format tsv settings max_threads=1 INSERT INTO foo SETTINGS max_threads = 1 FORMAT tsv -insert into foo settings max_threads=1 format tsv settings max_threads=1 +[multi] insert into foo settings max_threads=1 format tsv settings max_threads=1 You have SETTINGS before and after FORMAT Cannot parse input: expected '\n' before: 'settings max_threads=1 1' 1 diff --git a/tests/queries/0_stateless/02263_format_insert_settings.sh b/tests/queries/0_stateless/02263_format_insert_settings.sh index efb3d39ab6c..8b156ffec83 100755 --- a/tests/queries/0_stateless/02263_format_insert_settings.sh +++ b/tests/queries/0_stateless/02263_format_insert_settings.sh @@ -8,7 +8,7 @@ function run_format() { local q="$1" && shift - echo "$q" + echo "[multi] $q" $CLICKHOUSE_FORMAT "$@" <<<"$q" } function run_format_both() @@ -22,20 +22,20 @@ function run_format_both() } # NOTE: that those queries may work slow, due to stack trace obtaining -run_format 'insert into foo settings max_threads=1' 2> >(grep -m1 -o "Syntax error (query): failed at position .* (end of query):") +run_format 'insert into foo settings max_threads=1' |& grep --max-count 2 --only-matching -e "Syntax error (query): failed at position .* (end of query):" -e '^\[.*$' # compatibility -run_format 'insert into foo format tsv settings max_threads=1' 2> >(grep -m1 -F -o "Can't format ASTInsertQuery with data, since data will be lost.") +run_format 'insert into foo format tsv settings max_threads=1' |& grep --max-count 2 --only-matching -e "Can't format ASTInsertQuery with data, since data will be lost." -e '^\[.*$' run_format_both 'insert into foo format tsv settings max_threads=1' --allow_settings_after_format_in_insert -run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT") +run_format 'insert into foo settings max_threads=1 format tsv settings max_threads=1' --allow_settings_after_format_in_insert |& grep --max-count 2 --only-matching -e "You have SETTINGS before and after FORMAT" -e '^\[.*$' # and via server (since this is a separate code path) $CLICKHOUSE_CLIENT -q 'drop table if exists data_02263' $CLICKHOUSE_CLIENT -q 'create table data_02263 (key Int) engine=Memory()' -$CLICKHOUSE_CLIENT -q 'insert into data_02263 format TSV settings max_threads=1 1' 2> >(grep -m1 -F -o "Cannot parse input: expected '\n' before: 'settings max_threads=1 1'") +$CLICKHOUSE_CLIENT -q 'insert into data_02263 format TSV settings max_threads=1 1' |& grep --max-count 1 -F --only-matching "Cannot parse input: expected '\n' before: 'settings max_threads=1 1'" $CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 format TSV settings max_threads=1 1' $CLICKHOUSE_CLIENT -q 'select * from data_02263' -$CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 settings max_threads=1 format tsv settings max_threads=1' 2> >(grep -m1 -F -o "You have SETTINGS before and after FORMAT") +$CLICKHOUSE_CLIENT --allow_settings_after_format_in_insert=1 -q 'insert into data_02263 settings max_threads=1 format tsv settings max_threads=1' |& grep --max-count 1 -F --only-matching "You have SETTINGS before and after FORMAT" $CLICKHOUSE_CLIENT -q 'drop table data_02263' run_format_both 'insert into foo values' From cda633a1f66d68f282722d9cdfa7ab445a50f9cc Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Aug 2023 15:48:51 +0000 Subject: [PATCH 326/777] different linker --- cmake/linux/toolchain-s390x.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 23d14d6ed26..4d4d1fab473 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,9 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=bfd") -set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=bfd") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=bfd") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From bcc0fbbf9150de48fdec116f62c0664b84c13c9f Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 10 Aug 2023 16:10:33 +0000 Subject: [PATCH 327/777] Add EXCEPT clause to SYSTEM STOP LISTEN query --- docs/en/sql-reference/statements/system.md | 6 +- programs/server/Server.cpp | 6 +- src/Parsers/ASTSystemQuery.cpp | 48 +++++++++-- src/Parsers/ParserSystemQuery.cpp | 81 ++++++++++++++----- src/Server/ServerType.cpp | 58 +++++++++---- src/Server/ServerType.h | 18 ++++- .../test_system_start_stop_listen/test.py | 70 ++++++++++++++++ 7 files changed, 238 insertions(+), 49 deletions(-) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 59970dbeccd..766dd2348ee 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -443,9 +443,9 @@ SYSTEM STOP LISTEN [ON CLUSTER cluster_name] [QUERIES ALL | QUERIES DEFAULT | QU ``` - If `CUSTOM 'protocol'` modifier is specified, the custom protocol with the specified name defined in the protocols section of the server configuration will be stopped. -- If `QUERIES ALL` modifier is specified, all protocols are stopped. -- If `QUERIES DEFAULT` modifier is specified, all default protocols are stopped. -- If `QUERIES CUSTOM` modifier is specified, all custom protocols are stopped. +- If `QUERIES ALL [EXCEPT .. [,..]]` modifier is specified, all protocols are stopped, unless specified with `EXCEPT` clause. +- If `QUERIES DEFAULT [EXCEPT .. [,..]]` modifier is specified, all default protocols are stopped, unless specified with `EXCEPT` clause. +- If `QUERIES CUSTOM [EXCEPT .. [,..]]` modifier is specified, all custom protocols are stopped, unless specified with `EXCEPT` clause. ### SYSTEM START LISTEN diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index e6d5837dd0e..bdff3b79a99 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -2045,6 +2045,9 @@ void Server::createServers( for (const auto & protocol : protocols) { + if (!server_type.shouldStart(ServerType::Type::CUSTOM, protocol)) + continue; + std::string prefix = "protocols." + protocol + "."; std::string port_name = prefix + "port"; std::string description {" protocol"}; @@ -2054,9 +2057,6 @@ void Server::createServers( if (!config.has(prefix + "port")) continue; - if (!server_type.shouldStart(ServerType::Type::CUSTOM, port_name)) - continue; - std::vector hosts; if (config.has(prefix + "host")) hosts.push_back(config.getString(prefix + "host")); diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index fb10474a4d4..9be01719d8c 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -204,7 +204,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, } else if (type == Type::SUSPEND) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " FOR " + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FOR " << (settings.hilite ? hilite_none : "") << seconds << (settings.hilite ? hilite_keyword : "") << " SECOND" << (settings.hilite ? hilite_none : ""); @@ -232,12 +232,50 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, } else if (type == Type::START_LISTEN || type == Type::STOP_LISTEN) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " " << ServerType::serverTypeToString(server_type.type) - << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " " + << ServerType::serverTypeToString(server_type.type) << (settings.hilite ? hilite_none : ""); - if (server_type.type == ServerType::CUSTOM) + if (server_type.type == ServerType::Type::CUSTOM) { - settings.ostr << (settings.hilite ? hilite_identifier : "") << " " << backQuoteIfNeed(server_type.custom_name); + settings.ostr << " " << quoteString(server_type.custom_name); + } + + bool comma = false; + + if (!server_type.exclude_types.empty()) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") + << " EXCEPT" << (settings.hilite ? hilite_none : ""); + + for (auto cur_type : server_type.exclude_types) + { + if (cur_type == ServerType::Type::CUSTOM) + continue; + + if (comma) + settings.ostr << ","; + else + comma = true; + + settings.ostr << (settings.hilite ? hilite_keyword : "") << " " + << ServerType::serverTypeToString(cur_type) << (settings.hilite ? hilite_none : ""); + } + + if (server_type.exclude_types.contains(ServerType::Type::CUSTOM)) + { + for (const auto & cur_name : server_type.exclude_custom_names) + { + if (comma) + settings.ostr << ","; + else + comma = true; + + settings.ostr << (settings.hilite ? hilite_keyword : "") << " " + << ServerType::serverTypeToString(ServerType::Type::CUSTOM) << (settings.hilite ? hilite_none : ""); + + settings.ostr << " " << quoteString(cur_name); + } + } } } diff --git a/src/Parsers/ParserSystemQuery.cpp b/src/Parsers/ParserSystemQuery.cpp index 40fc1acae69..ac3aa41048c 100644 --- a/src/Parsers/ParserSystemQuery.cpp +++ b/src/Parsers/ParserSystemQuery.cpp @@ -458,32 +458,71 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & if (!parseQueryWithOnCluster(res, pos, expected)) return false; - ServerType::Type current_type = ServerType::Type::END; - std::string current_custom_name; - - for (const auto & type : magic_enum::enum_values()) + auto parse_server_type = [&](ServerType::Type & type, std::string & custom_name) -> bool { - if (ParserKeyword{ServerType::serverTypeToString(type)}.ignore(pos, expected)) + type = ServerType::Type::END; + custom_name = ""; + + for (const auto & cur_type : magic_enum::enum_values()) { - current_type = type; - break; + if (ParserKeyword{ServerType::serverTypeToString(cur_type)}.ignore(pos, expected)) + { + type = cur_type; + break; + } + } + + if (type == ServerType::Type::END) + return false; + + if (type == ServerType::CUSTOM) + { + ASTPtr ast; + + if (!ParserStringLiteral{}.parse(pos, ast, expected)) + return false; + + custom_name = ast->as().value.get(); + } + + return true; + }; + + ServerType::Type base_type; + std::string base_custom_name; + + ServerType::Types exclude_type; + ServerType::CustomNames exclude_custom_names; + + if (!parse_server_type(base_type, base_custom_name)) + return false; + + if (ParserKeyword{"EXCEPT"}.ignore(pos, expected)) + { + if (base_type != ServerType::Type::QUERIES_ALL && + base_type != ServerType::Type::QUERIES_DEFAULT && + base_type != ServerType::Type::QUERIES_CUSTOM) + return false; + + ServerType::Type current_type; + std::string current_custom_name; + + while (true) + { + if (!exclude_type.empty() && !ParserToken(TokenType::Comma).ignore(pos, expected)) + break; + + if (!parse_server_type(current_type, current_custom_name)) + return false; + + exclude_type.insert(current_type); + + if (current_type == ServerType::Type::CUSTOM) + exclude_custom_names.insert(current_custom_name); } } - if (current_type == ServerType::Type::END) - return false; - - if (current_type == ServerType::CUSTOM) - { - ASTPtr ast; - - if (!ParserStringLiteral{}.parse(pos, ast, expected)) - return false; - - current_custom_name = ast->as().value.get(); - } - - res->server_type = ServerType(current_type, current_custom_name); + res->server_type = ServerType(base_type, base_custom_name, exclude_type, exclude_custom_names); break; } diff --git a/src/Server/ServerType.cpp b/src/Server/ServerType.cpp index 4952cd1bd24..4199a5fd042 100644 --- a/src/Server/ServerType.cpp +++ b/src/Server/ServerType.cpp @@ -42,12 +42,9 @@ const char * ServerType::serverTypeToString(ServerType::Type type) bool ServerType::shouldStart(Type server_type, const std::string & server_custom_name) const { - if (type == Type::QUERIES_ALL) - return true; - - if (type == Type::QUERIES_DEFAULT) + auto is_type_default = [](Type current_type) { - switch (server_type) + switch (current_type) { case Type::TCP: case Type::TCP_WITH_PROXY: @@ -64,21 +61,42 @@ bool ServerType::shouldStart(Type server_type, const std::string & server_custom default: return false; } + }; + + auto is_type_custom = [](Type current_type) + { + return current_type == Type::CUSTOM; + }; + + if (exclude_types.contains(Type::QUERIES_ALL)) + return false; + + if (exclude_types.contains(Type::QUERIES_DEFAULT) && is_type_default(server_type)) + return false; + + if (exclude_types.contains(Type::QUERIES_CUSTOM) && is_type_custom(server_type)) + return false; + + if (exclude_types.contains(server_type)) + { + if (server_type != Type::CUSTOM) + return false; + + if (exclude_custom_names.contains(server_custom_name)) + return false; } + if (type == Type::QUERIES_ALL) + return true; + + if (type == Type::QUERIES_DEFAULT) + return is_type_default(server_type); + if (type == Type::QUERIES_CUSTOM) - { - switch (server_type) - { - case Type::CUSTOM: - return true; - default: - return false; - } - } + return is_type_custom(server_type); if (type == Type::CUSTOM) - return server_type == type && server_custom_name == "protocols." + custom_name + ".port"; + return server_type == type && server_custom_name == custom_name; return server_type == type; } @@ -86,6 +104,7 @@ bool ServerType::shouldStart(Type server_type, const std::string & server_custom bool ServerType::shouldStop(const std::string & port_name) const { Type port_type; + std::string port_custom_name; if (port_name == "http_port") port_type = Type::HTTP; @@ -121,12 +140,19 @@ bool ServerType::shouldStop(const std::string & port_name) const port_type = Type::INTERSERVER_HTTPS; else if (port_name.starts_with("protocols.") && port_name.ends_with(".port")) + { port_type = Type::CUSTOM; + constexpr size_t protocols_size = std::string_view("protocols.").size(); + constexpr size_t ports_size = std::string_view(".ports").size(); + + port_custom_name = port_name.substr(protocols_size, port_name.size() - protocols_size - ports_size + 1); + } + else return false; - return shouldStart(port_type, port_name); + return shouldStart(port_type, port_custom_name); } } diff --git a/src/Server/ServerType.h b/src/Server/ServerType.h index 1fab492222a..bfbe692f5bd 100644 --- a/src/Server/ServerType.h +++ b/src/Server/ServerType.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -28,8 +29,20 @@ public: END }; + using Types = std::unordered_set; + using CustomNames = std::unordered_set; + ServerType() = default; - explicit ServerType(Type type_, const std::string & custom_name_ = "") : type(type_), custom_name(custom_name_) {} + + explicit ServerType( + Type type_, + const std::string & custom_name_ = "", + const Types & exclude_types_ = {}, + const CustomNames exclude_custom_names_ = {}) + : type(type_), + custom_name(custom_name_), + exclude_types(exclude_types_), + exclude_custom_names(exclude_custom_names_) {} static const char * serverTypeToString(Type type); @@ -39,6 +52,9 @@ public: Type type; std::string custom_name; + + Types exclude_types; + CustomNames exclude_custom_names; }; } diff --git a/tests/integration/test_system_start_stop_listen/test.py b/tests/integration/test_system_start_stop_listen/test.py index 1925685af03..8a3081e0c15 100644 --- a/tests/integration/test_system_start_stop_listen/test.py +++ b/tests/integration/test_system_start_stop_listen/test.py @@ -143,3 +143,73 @@ def test_all_protocols(started_cluster): backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES ALL") assert_everything_works() + + +def test_except(started_cluster): + custom_client = Client(main_node.ip_address, 9001, command=cluster.client_bin_path) + assert_everything_works() + + # STOP LISTEN QUERIES ALL EXCEPT + main_node.query("SYSTEM STOP LISTEN QUERIES ALL EXCEPT MYSQL, CUSTOM 'tcp'") + assert "Connection refused" in main_node.query_and_get_error(QUERY) + custom_client.query(MYSQL_QUERY) + assert http_works() == False + assert http_works(8124) == False + + # START LISTEN QUERIES ALL EXCEPT + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES ALL EXCEPT TCP") + assert "Connection refused" in main_node.query_and_get_error(QUERY) + custom_client.query(MYSQL_QUERY) + assert http_works() == True + assert http_works(8124) == True + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES ALL") + + assert_everything_works() + + # STOP LISTEN QUERIES DEFAULT EXCEPT + main_node.query("SYSTEM STOP LISTEN QUERIES DEFAULT EXCEPT TCP") + main_node.query(QUERY) + assert "Connections to mysql failed" in custom_client.query_and_get_error( + MYSQL_QUERY + ) + custom_client.query(QUERY) + assert http_works() == False + assert http_works(8124) == True + + # START LISTEN QUERIES DEFAULT EXCEPT + backup_node.query( + "SYSTEM START LISTEN ON CLUSTER default QUERIES DEFAULT EXCEPT HTTP" + ) + main_node.query(QUERY) + main_node.query(MYSQL_QUERY) + custom_client.query(QUERY) + assert http_works() == False + assert http_works(8124) == True + + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES ALL") + + assert_everything_works() + + # STOP LISTEN QUERIES CUSTOM EXCEPT + main_node.query("SYSTEM STOP LISTEN QUERIES CUSTOM EXCEPT CUSTOM 'tcp'") + main_node.query(QUERY) + custom_client.query(MYSQL_QUERY) + custom_client.query(QUERY) + assert http_works() == True + assert http_works(8124) == False + + main_node.query("SYSTEM STOP LISTEN QUERIES CUSTOM") + + # START LISTEN QUERIES DEFAULT EXCEPT + backup_node.query( + "SYSTEM START LISTEN ON CLUSTER default QUERIES CUSTOM EXCEPT CUSTOM 'tcp'" + ) + main_node.query(QUERY) + main_node.query(MYSQL_QUERY) + assert "Connection refused" in custom_client.query_and_get_error(QUERY) + assert http_works() == True + assert http_works(8124) == True + + backup_node.query("SYSTEM START LISTEN ON CLUSTER default QUERIES ALL") + + assert_everything_works() From 4f0be777c5e186f3ebeabd5c1d8fd9adc2fe761b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 10 Aug 2023 13:18:32 +0000 Subject: [PATCH 328/777] Fix build --- .../tests/gtest_transform_query_for_external_database.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 5c1442ece11..749a154c19d 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -127,7 +127,8 @@ static void checkOld( std::string transformed_query = transformQueryForExternalDatabase( query_info, query_info.syntax_analyzer_result->requiredSourceColumns(), - state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, "test", "table", state.context); + state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, + LiteralEscapingStyle::Regular, "test", "table", state.context); EXPECT_EQ(transformed_query, expected) << query; } @@ -180,7 +181,8 @@ static void checkNewAnalyzer( query_info.table_expression = findTableExpression(query_node->getJoinTree(), "table"); std::string transformed_query = transformQueryForExternalDatabase( - query_info, column_names, state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, "test", "table", state.context); + query_info, column_names, state.getColumns(0), IdentifierQuotingStyle::DoubleQuotes, + LiteralEscapingStyle::Regular, "test", "table", state.context); EXPECT_EQ(transformed_query, expected) << query; } From 3d59ebe108016a83bba161751f728b08d5f94d70 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Aug 2023 20:11:22 +0200 Subject: [PATCH 329/777] fix --- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeDataPartInMemory.h | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 5 ++- .../test.py | 45 ++++++++++--------- 5 files changed, 31 insertions(+), 25 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 1df091ab1a3..195fdbc4d05 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -369,7 +369,7 @@ public: /// Makes clone of a part in detached/ directory via hard links virtual DataPartStoragePtr makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot, - const DiskTransactionPtr & disk_transaction = {}) const; + const DiskTransactionPtr & disk_transaction = {}) const; /// NOLINT /// Makes full clone of part in specified subdirectory (relative to storage data directory, e.g. "detached") on another disk MutableDataPartStoragePtr makeCloneOnDisk(const DiskPtr & disk, const String & directory_name) const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ed9127de977..395b480a84f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2636,7 +2636,7 @@ size_t MergeTreeData::clearOldBrokenPartsFromDetachedDirectory() if (!part_info.dir_name.starts_with("deleting_")) continue; - time_t startup_time = current_time + static_cast(Context::getGlobalContextInstance()->getUptimeSeconds()); + time_t startup_time = current_time - static_cast(Context::getGlobalContextInstance()->getUptimeSeconds()); time_t last_touch_time = get_last_touched_time(part_info); /// Maybe it's being deleted right now (for example, in ALTER DROP DETACHED) diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 29506a54fdc..95a17cbf589 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -43,7 +43,7 @@ public: String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return ""; } void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) override; DataPartStoragePtr makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot, - const DiskTransactionPtr & disk_transaction = {}) const override; + const DiskTransactionPtr & disk_transaction = {}) const override; /// NOLINT std::optional getColumnModificationTime(const String & /* column_name */) const override { return {}; } MutableDataPartStoragePtr flushToDisk(const String & new_relative_path, const StorageMetadataPtr & metadata_snapshot) const; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index bf0acef89c2..fa5a40cf27a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -651,7 +651,10 @@ bool ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData::Mutabl part->version.setCreationTID(Tx::PrehistoricTID, nullptr); String block_id = deduplicate ? fmt::format("{}_{}", part->info.partition_id, part->checksums.getTotalChecksumHex()) : ""; bool deduplicated = commitPart(zookeeper, part, block_id, replicas_num, /* writing_existing_part */ true).second; - PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, watch.elapsed(), profile_events_scope.getSnapshot())); + + /// Set a special error code if the block is duplicate + int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; + PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, watch.elapsed(), profile_events_scope.getSnapshot()), ExecutionStatus(error)); return deduplicated; } catch (...) diff --git a/tests/integration/test_broken_detached_part_clean_up/test.py b/tests/integration/test_broken_detached_part_clean_up/test.py index e7341deae35..bdf993ddedf 100644 --- a/tests/integration/test_broken_detached_part_clean_up/test.py +++ b/tests/integration/test_broken_detached_part_clean_up/test.py @@ -57,7 +57,11 @@ def remove_broken_detached_part_impl(table, node, expect_broken_prefix): ] ) - for name in ['unexpected_all_42_1337_5', 'deleting_all_123_456_7', 'tmp-fetch_all_12_34_5']: + for name in [ + "unexpected_all_42_1337_5", + "deleting_all_123_456_7", + "covered-by-broken_all_12_34_5", + ]: node.exec_in_container(["mkdir", f"{path_to_detached}../{name}"]) node.exec_in_container( [ @@ -67,9 +71,7 @@ def remove_broken_detached_part_impl(table, node, expect_broken_prefix): f"{path_to_detached}../{name}", ] ) - result = node.exec_in_container( - ["stat", f"{path_to_detached}../{name}"] - ) + result = node.exec_in_container(["stat", f"{path_to_detached}../{name}"]) print(result) assert "Modify: 2013-12-03" in result node.exec_in_container( @@ -89,21 +91,19 @@ def remove_broken_detached_part_impl(table, node, expect_broken_prefix): node.query(f"ATTACH TABLE {table}") node.wait_for_log_line( - "Removing detached part deleting_all_123_456_7", timeout=90, look_behind_lines=1000000 + "Removing detached part deleting_all_123_456_7", + timeout=90, + look_behind_lines=1000000, ) - - result = node.exec_in_container(["ls", path_to_detached]) - print(result) - assert f"{expect_broken_prefix}_all_3_3_0" in result - assert "all_1_1_0" in result - assert "trash" in result - assert "broken_all_fake" in result - assert "unexpected_all_42_1337_5" in result - assert "deleting_all_123_456_7" not in result - - time.sleep(15) - assert node.contains_in_log( - "Removed broken detached part unexpected_all_42_1337_5 due to a timeout" + node.wait_for_log_line( + f"Removed broken detached part {expect_broken_prefix}_all_3_3_0 due to a timeout", + timeout=10, + look_behind_lines=1000000, + ) + node.wait_for_log_line( + "Removed broken detached part unexpected_all_42_1337_5 due to a timeout", + timeout=10, + look_behind_lines=1000000, ) result = node.exec_in_container(["ls", path_to_detached]) @@ -112,13 +112,16 @@ def remove_broken_detached_part_impl(table, node, expect_broken_prefix): assert "all_1_1_0" in result assert "trash" in result assert "broken_all_fake" in result - assert "tmp-fetch_all_12_34_5" in result + assert "covered-by-broken_all_12_34_5" in result assert "unexpected_all_42_1337_5" not in result assert "deleting_all_123_456_7" not in result - node.query(f"ALTER TABLE {table} DROP DETACHED PART 'tmp-fetch_all_12_34_5'", settings={"allow_drop_detached": 1}) + node.query( + f"ALTER TABLE {table} DROP DETACHED PART 'covered-by-broken_all_12_34_5'", + settings={"allow_drop_detached": 1}, + ) result = node.exec_in_container(["ls", path_to_detached]) - assert "tmp-fetch_all_12_34_5" not in result + assert "covered-by-broken_all_12_34_5" not in result node.query(f"DROP TABLE {table} SYNC") From 3acb10b8bc08077c9ac39149517709ddc2c3163a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 20:51:03 +0200 Subject: [PATCH 330/777] Inhibit randomization in `00906_low_cardinality_cache` --- tests/ci/stress.py | 2 +- tests/queries/0_stateless/00906_low_cardinality_cache.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/stress.py b/tests/ci/stress.py index eb829cf519c..2c566144f2c 100755 --- a/tests/ci/stress.py +++ b/tests/ci/stress.py @@ -302,7 +302,7 @@ if __name__ == "__main__": have_long_running_queries = prepare_for_hung_check(args.drop_databases) except Exception as ex: have_long_running_queries = True - logging.error("Failed to prepare for hung check %s", str(ex)) + logging.error("Failed to prepare for hung check: %s", str(ex)) logging.info("Checking if some queries hung") cmd = " ".join( [ diff --git a/tests/queries/0_stateless/00906_low_cardinality_cache.sql b/tests/queries/0_stateless/00906_low_cardinality_cache.sql index cd2ceabcf6d..55eacd0db44 100644 --- a/tests/queries/0_stateless/00906_low_cardinality_cache.sql +++ b/tests/queries/0_stateless/00906_low_cardinality_cache.sql @@ -1,5 +1,5 @@ drop table if exists lc_00906; -create table lc_00906 (b LowCardinality(String)) engine=MergeTree order by b; +create table lc_00906 (b LowCardinality(String)) engine=MergeTree order by b SETTINGS index_granularity = 8192, index_granularity_bytes = '10Mi'; insert into lc_00906 select '0123456789' from numbers(100000000); select count(), b from lc_00906 group by b; drop table if exists lc_00906; From 3aca2408548bc149f933379506250e49238a24de Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 10 Aug 2023 20:59:33 +0200 Subject: [PATCH 331/777] Change the default of max_concurrent_queries from 100 to 1000 --- programs/server/config.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 14b8954fc39..85cdda63558 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -317,7 +317,7 @@ 0 - 100 + 1000 - false - -
\ No newline at end of file diff --git a/tests/integration/test_session_log/configs/session_log.xml b/tests/integration/test_session_log/configs/session_log.xml deleted file mode 100644 index a0e4e3e2216..00000000000 --- a/tests/integration/test_session_log/configs/session_log.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - system - session_log
- - toYYYYMM(event_date) - 7500 -
-
diff --git a/tests/integration/test_session_log/configs/users.xml b/tests/integration/test_session_log/configs/users.xml deleted file mode 100644 index 0416dfadc8a..00000000000 --- a/tests/integration/test_session_log/configs/users.xml +++ /dev/null @@ -1,23 +0,0 @@ - - - - 0 - - - - - - - pass - - - pass - - - pass - - - pass - - - \ No newline at end of file diff --git a/tests/integration/test_session_log/protos/clickhouse_grpc.proto b/tests/integration/test_session_log/protos/clickhouse_grpc.proto deleted file mode 120000 index 25d15f11e3b..00000000000 --- a/tests/integration/test_session_log/protos/clickhouse_grpc.proto +++ /dev/null @@ -1 +0,0 @@ -../../../../src/Server/grpc_protos/clickhouse_grpc.proto \ No newline at end of file diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py deleted file mode 100644 index dbb39993ce3..00000000000 --- a/tests/integration/test_session_log/test.py +++ /dev/null @@ -1,289 +0,0 @@ -import os - -import grpc -import pymysql.connections -import psycopg2 as py_psql -import pytest -import random -import sys -import threading - -from helpers.cluster import ClickHouseCluster, run_and_check - -POSTGRES_SERVER_PORT = 5433 -MYSQL_SERVER_PORT = 9001 -GRPC_PORT = 9100 -SESSION_LOG_MATCHING_FIELDS = "auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -DEFAULT_ENCODING = "utf-8" - -# Use grpcio-tools to generate *pb2.py files from *.proto. -proto_dir = os.path.join(SCRIPT_DIR, "./protos") -gen_dir = os.path.join(SCRIPT_DIR, "./_gen") -os.makedirs(gen_dir, exist_ok=True) -run_and_check( - f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", - shell=True, -) - -sys.path.append(gen_dir) - -import clickhouse_grpc_pb2 -import clickhouse_grpc_pb2_grpc - -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance( - "node", - main_configs=[ - "configs/ports.xml", - "configs/log.xml", - "configs/session_log.xml", - ], - user_configs=["configs/users.xml"], - # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 - env_variables={ - "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="") - }, -) - - -def grpc_get_url(): - return f"{instance.ip_address}:{GRPC_PORT}" - - -def grpc_create_insecure_channel(): - channel = grpc.insecure_channel(grpc_get_url()) - grpc.channel_ready_future(channel).result(timeout=2) - return channel - - -session_id_counter = 0 - - -def next_session_id(): - global session_id_counter - session_id = session_id_counter - session_id_counter += 1 - return str(session_id) - - -def grpc_query(query, user_, pass_, raise_exception): - try: - query_info = clickhouse_grpc_pb2.QueryInfo( - query=query, - session_id=next_session_id(), - user_name=user_, - password=pass_, - ) - channel = grpc_create_insecure_channel() - stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) - result = stub.ExecuteQuery(query_info) - if result and result.HasField("exception"): - raise Exception(result.exception.display_text) - - return result.output.decode(DEFAULT_ENCODING) - except Exception: - assert raise_exception - - -def postgres_query(query, user_, pass_, raise_exception): - try: - client = py_psql.connect( - host=instance.ip_address, - port=POSTGRES_SERVER_PORT, - user=user_, - password=pass_, - database="default", - ) - cursor = client.cursor() - cursor.execute(query) - cursor.fetchall() - except Exception: - assert raise_exception - - -def mysql_query(query, user_, pass_, raise_exception): - try: - client = pymysql.connections.Connection( - host=instance.ip_address, - user=user_, - password=pass_, - database="default", - port=MYSQL_SERVER_PORT, - ) - cursor = client.cursor(pymysql.cursors.DictCursor) - if raise_exception: - with pytest.raises(Exception): - cursor.execute(query) - else: - cursor.execute(query) - cursor.fetchall() - except Exception: - assert raise_exception - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_grpc_session(started_cluster): - grpc_query("SELECT 1", "grpc_user", "pass", False) - grpc_query("SELECT 2", "grpc_user", "wrong_pass", True) - grpc_query("SELECT 3", "wrong_grpc_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "grpc_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'Logout'" - ) - assert logout_records == "grpc_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "grpc_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_mysql_session(started_cluster): - mysql_query("SELECT 1", "mysql_user", "pass", False) - mysql_query("SELECT 2", "mysql_user", "wrong_pass", True) - mysql_query("SELECT 3", "wrong_mysql_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "mysql_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'Logout'" - ) - assert logout_records == "mysql_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "mysql_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_postgres_session(started_cluster): - postgres_query("SELECT 1", "postgres_user", "pass", False) - postgres_query("SELECT 2", "postgres_user", "wrong_pass", True) - postgres_query("SELECT 3", "wrong_postgres_user", "pass", True) - - instance.query("SYSTEM FLUSH LOGS") - login_success_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginSuccess'" - ) - assert login_success_records == "postgres_user\t1\t1\n" - logout_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'Logout'" - ) - assert logout_records == "postgres_user\t1\t1\n" - login_failure_records = instance.query( - "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginFailure'" - ) - assert login_failure_records == "postgres_user\t1\t1\n" - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "1\n" - - -def test_parallel_sessions(started_cluster): - thread_list = [] - for _ in range(10): - # Sleep time does not significantly matter here, - # test should pass even without sleeping. - for function in [postgres_query, grpc_query, mysql_query]: - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "parallel_user", - "pass", - False, - ), - ) - thread.start() - thread_list.append(thread) - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "parallel_user", - "wrong_pass", - True, - ), - ) - thread.start() - thread_list.append(thread) - thread = threading.Thread( - target=function, - args=( - f"SELECT sleep({random.uniform(0.03, 0.04)})", - "wrong_parallel_user", - "pass", - True, - ), - ) - thread.start() - thread_list.append(thread) - - for thread in thread_list: - thread.join() - - instance.query("SYSTEM FLUSH LOGS") - port_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user'" - ) - assert port_0_sessions == "90\n" - - port_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_port = 0" - ) - assert port_0_sessions == "0\n" - - address_0_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_address = toIPv6('::')" - ) - assert address_0_sessions == "0\n" - - grpc_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'gRPC'" - ) - assert grpc_sessions == "30\n" - - mysql_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'MySQL'" - ) - assert mysql_sessions == "30\n" - - postgres_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'PostgreSQL'" - ) - assert postgres_sessions == "30\n" - - logins_and_logouts = instance.query( - f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'Logout')" - ) - assert logins_and_logouts == "30\n" - - logout_failure_sessions = instance.query( - f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'" - ) - assert logout_failure_sessions == "30\n" diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.reference b/tests/queries/0_stateless/02833_concurrrent_sessions.reference deleted file mode 100644 index bfe507e8eac..00000000000 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.reference +++ /dev/null @@ -1,34 +0,0 @@ -sessions: -150 -port_0_sessions: -0 -address_0_sessions: -0 -tcp_sessions -60 -http_sessions -30 -http_with_session_id_sessions -30 -my_sql_sessions -30 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 -Corresponding LoginSuccess/Logout -10 -LoginFailure -10 diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh deleted file mode 100755 index 26b48462a76..00000000000 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.sh +++ /dev/null @@ -1,138 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest, no-debug - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ - -# Each user uses a separate thread. -readonly TCP_USERS=( "02833_TCP_USER_${PID}"_{1,2} ) # 2 concurrent TCP users -readonly HTTP_USERS=( "02833_HTTP_USER_${PID}" ) -readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "02833_HTTP_WITH_SESSION_ID_USER_${PID}" ) -readonly MYSQL_USERS=( "02833_MYSQL_USER_${PID}") -readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) - -readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" - -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -for user in "${ALL_USERS[@]}"; do - ${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${user} IDENTIFIED WITH plaintext_password BY 'pass'" - ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${user}" - ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${user}"; -done - -# All _session functions execute in separate threads. -# These functions try to create a session with successful login and logout. -# Sleep a small, random amount of time to make concurrency more intense. -# and try to login with an invalid password. -function tcp_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM})" --user="${user}" --password="pass" - # login failure - ${CLICKHOUSE_CLIENT} -q "SELECT 2" --user="${user}" --password 'invalid' - done -} - -function http_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep(0.01${RANDOM})" - - # login failure - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=wrong" -d "SELECT 4" - done -} - -function http_with_session_id_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.01${RANDOM}" - - # login failure - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=wrong" -d "SELECT 6" - done -} - -function mysql_session() -{ - local user=$1 - local i=0 - while (( (i++) < 10 )); do - # login logout - ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" - - # login failure - ${CLICKHOUSE_CLIENT} -q "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'wrong', SETTINGS connection_max_tries=1)" - done -} - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" - -export -f tcp_session; -export -f http_session; -export -f http_with_session_id_session; -export -f mysql_session; - -for user in "${TCP_USERS[@]}"; do - timeout 60s bash -c "tcp_session ${user}" >/dev/null 2>&1 & -done - -for user in "${HTTP_USERS[@]}"; do - timeout 60s bash -c "http_session ${user}" >/dev/null 2>&1 & -done - -for user in "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}"; do - timeout 60s bash -c "http_with_session_id_session ${user}" >/dev/null 2>&1 & -done - -for user in "${MYSQL_USERS[@]}"; do - timeout 60s bash -c "mysql_session ${user}" >/dev/null 2>&1 & -done - -wait - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" - -echo "port_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_port = 0" - -echo "address_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_address = toIPv6('::')" - -echo "tcp_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${TCP_USERS_SQL_COLLECTION_STRING}) AND interface = 'TCP'" -echo "http_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" -echo "http_with_session_id_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" -echo "my_sql_sessions" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${MYSQL_USERS_SQL_COLLECTION_STRING}) AND interface = 'MySQL'" - -for user in "${ALL_USERS[@]}"; do - ${CLICKHOUSE_CLIENT} -q "DROP USER ${user}" - echo "Corresponding LoginSuccess/Logout" - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'Logout')" - echo "LoginFailure" - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" - done diff --git a/tests/queries/0_stateless/02834_remote_session_log.reference b/tests/queries/0_stateless/02834_remote_session_log.reference deleted file mode 100644 index e2680982ab0..00000000000 --- a/tests/queries/0_stateless/02834_remote_session_log.reference +++ /dev/null @@ -1,13 +0,0 @@ -0 -0 -0 -0 -client_port 0 connections: -0 -client_address '::' connections: -0 -login failures: -0 -TCP Login and logout count is equal -HTTP Login and logout count is equal -MySQL Login and logout count is equal diff --git a/tests/queries/0_stateless/02834_remote_session_log.sh b/tests/queries/0_stateless/02834_remote_session_log.sh deleted file mode 100755 index 3bedfb6c9ee..00000000000 --- a/tests/queries/0_stateless/02834_remote_session_log.sh +++ /dev/null @@ -1,56 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ -readonly TEST_USER=$"02834_USER_${PID}" -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER} IDENTIFIED WITH plaintext_password BY 'pass'" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT CREATE TEMPORARY TABLE, MYSQL, REMOTE ON *.* TO ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" - -${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ - -d "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" - -${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ - -d "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" - -${CLICKHOUSE_CLIENT} -q "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" -${CLICKHOUSE_CLIENT} -q "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "client_port 0 connections:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_port = 0" - -echo "client_address '::' connections:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_address = toIPv6('::')" - -echo "login failures:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and type = 'LoginFailure'" - -# remote(...) function sometimes reuses old cached sessions for query execution. -# This makes LoginSuccess/Logout entries count unstable, but success and logouts must always match. - -for interface in 'TCP' 'HTTP' 'MySQL' -do - LOGIN_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}'"` - CORRESPONDING_LOGOUT_RECORDS_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}')"` - - if [ "$LOGIN_COUNT" == "$CORRESPONDING_LOGOUT_RECORDS_COUNT" ]; then - echo "${interface} Login and logout count is equal" - else - TOTAL_LOGOUT_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}'"` - echo "${interface} Login count ${LOGIN_COUNT} != corresponding logout count ${CORRESPONDING_LOGOUT_RECORDS_COUNT}. TOTAL_LOGOUT_COUNT ${TOTAL_LOGOUT_COUNT}" - fi -done - -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.reference b/tests/queries/0_stateless/02835_drop_user_during_session.reference deleted file mode 100644 index 7252faab8c6..00000000000 --- a/tests/queries/0_stateless/02835_drop_user_during_session.reference +++ /dev/null @@ -1,8 +0,0 @@ -port_0_sessions: -0 -address_0_sessions: -0 -Corresponding LoginSuccess/Logout -9 -LoginFailure -0 diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh deleted file mode 100755 index 347ebd22f96..00000000000 --- a/tests/queries/0_stateless/02835_drop_user_during_session.sh +++ /dev/null @@ -1,114 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-debug - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -readonly PID=$$ - -readonly TEST_USER="02835_USER_${PID}" -readonly TEST_ROLE="02835_ROLE_${PID}" -readonly TEST_PROFILE="02835_PROFILE_${PID}" -readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" - -function tcp_session() -{ - local user=$1 - ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.numbers" --user="${user}" -} - -function http_session() -{ - local user=$1 - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" -} - -function http_with_session_id_session() -{ - local user=$1 - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" -} - -# Busy-waits until user $1, specified amount of queries ($2) will run simultaneously. -function wait_for_queries_start() -{ - local user=$1 - local queries_count=$2 - # 10 seconds waiting - counter=0 retries=100 - while [[ $counter -lt $retries ]]; do - result=$($CLICKHOUSE_CLIENT --query "SELECT COUNT(*) FROM system.processes WHERE user = '${user}'") - if [[ $result == "${queries_count}" ]]; then - break; - fi - sleep 0.1 - ((++counter)) - done -} - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" -${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" - -# DROP USE CASE -${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -export -f tcp_session; -export -f http_session; -export -f http_with_session_id_session; - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -# DROP ROLE CASE -${CLICKHOUSE_CLIENT} -q "CREATE ROLE IF NOT EXISTS ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} DEFAULT ROLE ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP ROLE ${TEST_ROLE}" -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -# DROP PROFILE CASE -${CLICKHOUSE_CLIENT} -q "CREATE SETTINGS PROFILE IF NOT EXISTS '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} SETTINGS PROFILE '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" - -timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & -timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & - -wait_for_queries_start $TEST_USER 3 -${CLICKHOUSE_CLIENT} -q "DROP SETTINGS PROFILE '${TEST_PROFILE}'" -${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" - -${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & - -wait - -${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" - -echo "port_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_port = 0" -echo "address_0_sessions:" -${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_address = toIPv6('::')" -echo "Corresponding LoginSuccess/Logout" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout')" -echo "LoginFailure" -${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" From 0d9e3ca34501123cce18c495eb9cc5f0b9b4a9ff Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 11 Aug 2023 00:00:36 +0200 Subject: [PATCH 347/777] remove no-parallel tag from some tests --- .../InterpreterShowTablesQuery.cpp | 4 +-- src/Interpreters/executeQuery.cpp | 4 ++- src/Parsers/ASTShowTablesQuery.cpp | 19 +++++++++++--- src/Parsers/ASTShowTablesQuery.h | 5 +++- src/Parsers/ParserShowTablesQuery.cpp | 4 +-- src/Parsers/QueryParameterVisitor.cpp | 2 +- tests/clickhouse-test | 2 +- ...80_show_tables_and_system_tables.reference | 6 ++--- .../00080_show_tables_and_system_tables.sql | 19 +++++++------- .../00158_buffer_and_nonexistent_table.sql | 17 ++++++------- .../00508_materialized_view_to.sql | 22 ++++++---------- ...t_database_when_create_materializ_view.sql | 25 +++++++------------ .../00604_show_create_database.reference | 2 +- .../00604_show_create_database.sql | 8 +++--- ...3_max_block_size_system_tables_columns.sql | 1 - .../00740_database_in_nested_view.sql | 21 ++++++---------- .../00815_left_join_on_stepanel.sql | 8 ++---- .../queries/0_stateless/01015_attach_part.sql | 1 - .../0_stateless/01021_only_tuple_columns.sql | 1 - 19 files changed, 80 insertions(+), 91 deletions(-) diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index 2f1a4a32bee..5fe0a862e05 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -116,10 +116,10 @@ String InterpreterShowTablesQuery::getRewrittenQuery() return rewritten_query.str(); } - if (query.temporary && !query.from.empty()) + if (query.temporary && !query.getFrom().empty()) throw Exception(ErrorCodes::SYNTAX_ERROR, "The `FROM` and `TEMPORARY` cannot be used together in `SHOW TABLES`"); - String database = getContext()->resolveDatabase(query.from); + String database = getContext()->resolveDatabase(query.getFrom()); DatabaseCatalog::instance().assertDatabaseExists(database); WriteBufferFromOwnString rewritten_query; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 578ca3b41f9..f66326f570a 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -728,7 +728,9 @@ static std::tuple executeQueryImpl( is_create_parameterized_view = create_query->isParameterizedView(); /// Replace ASTQueryParameter with ASTLiteral for prepared statements. - if (!is_create_parameterized_view && context->hasQueryParameters()) + /// Even if we don't have parameters in query_context, check that AST doesn't have unknown parameters + bool probably_has_params = find_first_symbols<'{'>(begin, end) != end; + if (!is_create_parameterized_view && probably_has_params) { ReplaceQueryParameterVisitor visitor(context->getQueryParameters()); visitor.visit(ast); diff --git a/src/Parsers/ASTShowTablesQuery.cpp b/src/Parsers/ASTShowTablesQuery.cpp index 7b4e052bc86..03ae96fa288 100644 --- a/src/Parsers/ASTShowTablesQuery.cpp +++ b/src/Parsers/ASTShowTablesQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -10,10 +11,20 @@ ASTPtr ASTShowTablesQuery::clone() const { auto res = std::make_shared(*this); res->children.clear(); + if (from) + res->set(res->from, from->clone()); + cloneOutputOptions(*res); return res; } +String ASTShowTablesQuery::getFrom() const +{ + String name; + tryGetIdentifierNameInto(from, name); + return name; +} + void ASTShowTablesQuery::formatLike(const FormatSettings & settings) const { if (!like.empty()) @@ -72,9 +83,11 @@ void ASTShowTablesQuery::formatQueryImpl(const FormatSettings & settings, Format settings.ostr << (settings.hilite ? hilite_keyword : "") << "SHOW " << (temporary ? "TEMPORARY " : "") << (dictionaries ? "DICTIONARIES" : "TABLES") << (settings.hilite ? hilite_none : ""); - if (!from.empty()) - settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") - << backQuoteIfNeed(from); + if (from) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : ""); + from->formatImpl(settings, state, frame); + } formatLike(settings); diff --git a/src/Parsers/ASTShowTablesQuery.h b/src/Parsers/ASTShowTablesQuery.h index 2878df54bcc..2fd4c3dac96 100644 --- a/src/Parsers/ASTShowTablesQuery.h +++ b/src/Parsers/ASTShowTablesQuery.h @@ -24,8 +24,9 @@ public: bool caches = false; bool full = false; + IAST * from; + String cluster_str; - String from; String like; bool not_like = false; @@ -38,6 +39,8 @@ public: ASTPtr clone() const override; QueryKind getQueryKind() const override { return QueryKind::Show; } + String getFrom() const; + protected: void formatLike(const FormatSettings & settings) const; void formatLimit(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const; diff --git a/src/Parsers/ParserShowTablesQuery.cpp b/src/Parsers/ParserShowTablesQuery.cpp index 3540a6d3fc8..e3728eb2cd6 100644 --- a/src/Parsers/ParserShowTablesQuery.cpp +++ b/src/Parsers/ParserShowTablesQuery.cpp @@ -36,7 +36,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserKeyword s_where("WHERE"); ParserKeyword s_limit("LIMIT"); ParserStringLiteral like_p; - ParserIdentifier name_p; + ParserIdentifier name_p(true); ParserExpressionWithOptionalAlias exp_elem(false); ASTPtr like; @@ -174,7 +174,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec return false; } - tryGetIdentifierNameInto(database, query->from); + query->set(query->from, database); if (like) query->like = like->as().value.safeGet(); diff --git a/src/Parsers/QueryParameterVisitor.cpp b/src/Parsers/QueryParameterVisitor.cpp index 1282c12cce6..b8679cc3b96 100644 --- a/src/Parsers/QueryParameterVisitor.cpp +++ b/src/Parsers/QueryParameterVisitor.cpp @@ -31,7 +31,7 @@ private: void visitQueryParameter(const ASTQueryParameter & query_parameter) { - query_parameters[query_parameter.name]= query_parameter.type; + query_parameters[query_parameter.name] = query_parameter.type; } }; diff --git a/tests/clickhouse-test b/tests/clickhouse-test index fc175f2a05a..b428c291ba1 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1197,7 +1197,7 @@ class TestCase: os.environ["CLICKHOUSE_LOG_COMMENT"] = args.testcase_basename params = { - "client": client + " --database=" + database, + "client": client + " --database=" + database + " --param_CLICKHOUSE_DATABASE=" + database, "logs_level": server_logs_level, "options": client_options, "test": self.case_file, diff --git a/tests/queries/0_stateless/00080_show_tables_and_system_tables.reference b/tests/queries/0_stateless/00080_show_tables_and_system_tables.reference index f0b6f0e0c41..e4690f20d3e 100644 --- a/tests/queries/0_stateless/00080_show_tables_and_system_tables.reference +++ b/tests/queries/0_stateless/00080_show_tables_and_system_tables.reference @@ -2,8 +2,8 @@ A B numbers one -A 1 TinyLog CREATE TABLE test_show_tables.A (`A` UInt8) ENGINE = TinyLog -B 1 TinyLog CREATE TABLE test_show_tables.B (`A` UInt8) ENGINE = TinyLog +A 1 TinyLog CREATE TABLE default.A (`A` UInt8) ENGINE = TinyLog +B 1 TinyLog CREATE TABLE default.B (`A` UInt8) ENGINE = TinyLog test_temporary_table -['test_show_tables'] ['test_materialized'] +['default'] ['test_materialized'] 0 diff --git a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql index 137dfb5b6f0..40c26be31c0 100644 --- a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql +++ b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql @@ -1,25 +1,24 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS test_show_tables; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; -CREATE DATABASE test_show_tables; +CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier}; -CREATE TABLE test_show_tables.A (A UInt8) ENGINE = TinyLog; -CREATE TABLE test_show_tables.B (A UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.A (A UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.B (A UInt8) ENGINE = TinyLog; -SHOW TABLES from test_show_tables; +SHOW TABLES from {CLICKHOUSE_DATABASE:Identifier}; SHOW TABLES in system where engine like '%System%' and name in ('numbers', 'one'); -SELECT name, toUInt32(metadata_modification_time) > 0, engine_full, create_table_query FROM system.tables WHERE database = 'test_show_tables' ORDER BY name FORMAT TSVRaw; +SELECT name, toUInt32(metadata_modification_time) > 0, engine_full, create_table_query FROM system.tables WHERE database = currentDatabase() ORDER BY name FORMAT TSVRaw; CREATE TEMPORARY TABLE test_temporary_table (id UInt64); SELECT name FROM system.tables WHERE is_temporary = 1 AND name = 'test_temporary_table'; -CREATE TABLE test_show_tables.test_log(id UInt64) ENGINE = Log; -CREATE MATERIALIZED VIEW test_show_tables.test_materialized ENGINE = Log AS SELECT * FROM test_show_tables.test_log; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test_log(id UInt64) ENGINE = Log; +CREATE MATERIALIZED VIEW {CLICKHOUSE_DATABASE:Identifier}.test_materialized ENGINE = Log AS SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_log; SELECT dependencies_database, dependencies_table FROM system.tables WHERE name = 'test_log'; -DROP DATABASE test_show_tables; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; -- Check that create_table_query works for system tables and unusual Databases diff --git a/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql b/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql index 23c157db875..1d988b38b05 100644 --- a/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql +++ b/tests/queries/0_stateless/00158_buffer_and_nonexistent_table.sql @@ -1,11 +1,10 @@ --- Tags: no-parallel -CREATE DATABASE IF NOT EXISTS test2_00158; -DROP TABLE IF EXISTS test2_00158.mt_buffer_00158; -DROP TABLE IF EXISTS test2_00158.mt_00158; -CREATE TABLE test2_00158.mt_buffer_00158 (d Date DEFAULT today(), x UInt64) ENGINE = Buffer(test2_00158, mt_00158, 16, 100, 100, 1000000, 1000000, 1000000000, 1000000000); +CREATE DATABASE IF NOT EXISTS {CLICKHOUSE_DATABASE:Identifier}; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.mt_buffer_00158; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.mt_00158; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.mt_buffer_00158 (d Date DEFAULT today(), x UInt64) ENGINE = Buffer({CLICKHOUSE_DATABASE:Identifier}, mt_00158, 16, 100, 100, 1000000, 1000000, 1000000000, 1000000000); SET send_logs_level = 'fatal'; -- Supress "Destination table test2.mt doesn't exist. Block of data is discarded." -INSERT INTO test2_00158.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 100000; -INSERT INTO test2_00158.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 1000000; -DROP TABLE IF EXISTS test2_00158.mt_buffer_00158; -DROP DATABASE test2_00158; +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 100000; +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 1000000; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.mt_buffer_00158; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/00508_materialized_view_to.sql b/tests/queries/0_stateless/00508_materialized_view_to.sql index 522ceb40404..0d8fb85eeeb 100644 --- a/tests/queries/0_stateless/00508_materialized_view_to.sql +++ b/tests/queries/0_stateless/00508_materialized_view_to.sql @@ -1,9 +1,3 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS test_00508; -CREATE DATABASE test_00508; - -USE test_00508; CREATE TABLE src (x UInt8) ENGINE = Null; CREATE TABLE dst (x UInt8) ENGINE = Memory; @@ -20,17 +14,17 @@ SELECT * FROM dst ORDER BY x; USE default; -- Reattach MV (shortcut) -ATTACH TABLE test_00508.mv_00508; +ATTACH TABLE {CLICKHOUSE_DATABASE:Identifier}.mv_00508; -INSERT INTO test_00508.src VALUES (3); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.src VALUES (3); -SELECT * FROM test_00508.mv_00508 ORDER BY x; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.mv_00508 ORDER BY x; -- Drop the MV and see if the data is still readable -DROP TABLE test_00508.mv_00508; -SELECT * FROM test_00508.dst ORDER BY x; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.mv_00508; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.dst ORDER BY x; -DROP TABLE test_00508.src; -DROP TABLE test_00508.dst; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.src; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.dst; -DROP DATABASE test_00508; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql b/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql index 46fc0dd586d..87f2e2b5276 100644 --- a/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql +++ b/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql @@ -1,11 +1,6 @@ --- Tags: no-parallel - -CREATE DATABASE test_00571; - -USE test_00571; DROP DATABASE IF EXISTS none; -DROP TABLE IF EXISTS test_00571; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; DROP TABLE IF EXISTS test_materialized_00571; set allow_deprecated_syntax_for_merge_tree=1; @@ -15,18 +10,16 @@ CREATE MATERIALIZED VIEW test_materialized_00571 ENGINE = MergeTree(date, (platf USE none; -INSERT INTO test_00571.test_00571 VALUES('2018-02-16', 'a', 'a'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.test_00571 VALUES('2018-02-16', 'a', 'a'); -SELECT * FROM test_00571.test_00571; -SELECT * FROM test_00571.test_materialized_00571; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_00571; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; -DETACH TABLE test_00571.test_materialized_00571; -ATTACH TABLE test_00571.test_materialized_00571; +DETACH TABLE {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; +ATTACH TABLE {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; -SELECT * FROM test_00571.test_materialized_00571; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; DROP DATABASE IF EXISTS none; -DROP TABLE IF EXISTS test_00571.test_00571; -DROP TABLE IF EXISTS test_00571.test_materialized_00571; - -DROP DATABASE test_00571; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.test_00571; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; diff --git a/tests/queries/0_stateless/00604_show_create_database.reference b/tests/queries/0_stateless/00604_show_create_database.reference index c05b088280e..52fd2c48df1 100644 --- a/tests/queries/0_stateless/00604_show_create_database.reference +++ b/tests/queries/0_stateless/00604_show_create_database.reference @@ -1 +1 @@ -CREATE DATABASE test_00604\nENGINE = Atomic +CREATE DATABASE default\nENGINE = Atomic diff --git a/tests/queries/0_stateless/00604_show_create_database.sql b/tests/queries/0_stateless/00604_show_create_database.sql index c990e7abed3..d20d2cb3e53 100644 --- a/tests/queries/0_stateless/00604_show_create_database.sql +++ b/tests/queries/0_stateless/00604_show_create_database.sql @@ -1,5 +1,5 @@ --- Tags: no-ordinary-database, no-parallel +-- Tags: no-ordinary-database -create database if not exists test_00604; -show create database test_00604; -drop database test_00604; +create database if not exists {CLICKHOUSE_DATABASE:Identifier}; +show create database {CLICKHOUSE_DATABASE:Identifier}; +drop database {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql b/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql index 0c8e4ae237d..fe1876ed6f8 100644 --- a/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql +++ b/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql @@ -1,4 +1,3 @@ --- Tags: no-parallel -- NOTE: database = currentDatabase() is not mandatory diff --git a/tests/queries/0_stateless/00740_database_in_nested_view.sql b/tests/queries/0_stateless/00740_database_in_nested_view.sql index e2debe2859b..42c26a709b1 100644 --- a/tests/queries/0_stateless/00740_database_in_nested_view.sql +++ b/tests/queries/0_stateless/00740_database_in_nested_view.sql @@ -1,10 +1,5 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS test_00740; -CREATE DATABASE test_00740; -USE test_00740; - -DROP TABLE IF EXISTS test_00740; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; DROP TABLE IF EXISTS test_view_00740; DROP TABLE IF EXISTS test_nested_view_00740; DROP TABLE IF EXISTS test_joined_view_00740; @@ -19,12 +14,10 @@ SELECT * FROM test_nested_view_00740; SELECT * FROM test_joined_view_00740; USE default; -SELECT * FROM test_00740.test_view_00740; -SELECT * FROM test_00740.test_nested_view_00740; -SELECT * FROM test_00740.test_joined_view_00740; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_view_00740; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_nested_view_00740; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_joined_view_00740; -DROP TABLE IF EXISTS test_00740.test_00740; -DROP TABLE IF EXISTS test_00740.test_view_00740; -DROP TABLE IF EXISTS test_00740.test_nested_view_00740; - -DROP DATABASE test_00740; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.test_00740; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.test_view_00740; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.test_nested_view_00740; diff --git a/tests/queries/0_stateless/00815_left_join_on_stepanel.sql b/tests/queries/0_stateless/00815_left_join_on_stepanel.sql index 13172f8c18d..725c9523cd7 100644 --- a/tests/queries/0_stateless/00815_left_join_on_stepanel.sql +++ b/tests/queries/0_stateless/00815_left_join_on_stepanel.sql @@ -1,7 +1,3 @@ --- Tags: no-parallel - -CREATE DATABASE IF NOT EXISTS test_00815; -USE test_00815; DROP TABLE IF EXISTS fact_cpc_clicks; DROP TABLE IF EXISTS dim_model; @@ -16,6 +12,6 @@ select f.model_id from fact_cpc_clicks as f left join dim_model as d on f.model_ USE default; -select f.model_id from test_00815.fact_cpc_clicks as f left join test_00815.dim_model as d on f.model_id=d.model_id limit 10; +select f.model_id from {CLICKHOUSE_DATABASE:Identifier}.fact_cpc_clicks as f left join {CLICKHOUSE_DATABASE:Identifier}.dim_model as d on f.model_id=d.model_id limit 10; -DROP DATABASE test_00815; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/01015_attach_part.sql b/tests/queries/0_stateless/01015_attach_part.sql index a2f949d3499..9ff505efd8f 100644 --- a/tests/queries/0_stateless/01015_attach_part.sql +++ b/tests/queries/0_stateless/01015_attach_part.sql @@ -1,4 +1,3 @@ --- Tags: no-parallel DROP TABLE IF EXISTS table_01; diff --git a/tests/queries/0_stateless/01021_only_tuple_columns.sql b/tests/queries/0_stateless/01021_only_tuple_columns.sql index 02db21bc0b2..d8d146f59fd 100644 --- a/tests/queries/0_stateless/01021_only_tuple_columns.sql +++ b/tests/queries/0_stateless/01021_only_tuple_columns.sql @@ -1,4 +1,3 @@ --- Tags: no-parallel CREATE TABLE test ( From b9638c6387d46e85abe55a333049efb0d2c757e7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Aug 2023 22:16:30 +0000 Subject: [PATCH 348/777] path to libc --- cmake/linux/toolchain-s390x.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 4d4d1fab473..03314a9d2ba 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,9 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold") -set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From 9d68d50ae76b179c7cf448635fef2e494e7a4f7f Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 18 May 2023 21:26:57 +0000 Subject: [PATCH 349/777] system_kafka_consumers: initial --- .../System/StorageSystemKafkaConsumers.cpp | 166 ++++++++++++++++++ .../System/StorageSystemKafkaConsumers.h | 20 +++ src/Storages/System/attachSystemTables.cpp | 2 + 3 files changed, 188 insertions(+) create mode 100644 src/Storages/System/StorageSystemKafkaConsumers.cpp create mode 100644 src/Storages/System/StorageSystemKafkaConsumers.h diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp new file mode 100644 index 00000000000..c6f3fc3a4d9 --- /dev/null +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -0,0 +1,166 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include "base/types.h" + + +namespace DB +{ + +NamesAndTypesList StorageSystemKafkaConsumers::getNamesAndTypes() +{ + NamesAndTypesList names_and_types{ + {"database", std::make_shared()}, + {"table", std::make_shared()}, + {"consumer_id", std::make_shared()}, //(number? or string? - single clickhouse table can have many consumers) + {"assignments.topic", std::make_shared(std::make_shared())}, + {"assignments.partition_id", std::make_shared(std::make_shared())}, + {"assignments.current_offset", std::make_shared(std::make_shared())}, + // {"last_exception_time", std::make_shared()}, + // {"last_exception", std::make_shared()}, + // {"last_poll_time", std::make_shared()}, + // {"num_messages_read", std::make_shared()}, + // {"last_commit_time", std::make_shared()}, + // {"num_commits", std::make_shared()}, + // {"last_rebalance_time", std::make_shared()}, + // {"num_rebalance_revocations", std::make_shared()}, + // {"num_rebalance_assignments", std::make_shared()}, + // {"is_currently_used", std::make_shared()}, + }; + return names_and_types; +} + + + +void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +{ + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-variable" + + auto tables_mark_dropped = DatabaseCatalog::instance().getTablesMarkedDropped(); + + size_t index = 0; + + // auto & column_index = assert_cast(*res_columns[index++]); + // auto & column_database = assert_cast(*res_columns[index++]); + // auto & column_table = assert_cast(*res_columns[index++]); + // auto & column_uuid = assert_cast(*res_columns[index++]).getData(); + // auto & column_engine = assert_cast(*res_columns[index++]); + // auto & column_metadata_dropped_path = assert_cast(*res_columns[index++]); + // auto & column_table_dropped_time = assert_cast(*res_columns[index++]); + + + auto & database = assert_cast(*res_columns[index++]); + auto & table = assert_cast(*res_columns[index++]); + auto & consumer_id = assert_cast(*res_columns[index++]); //(number? or string? - single clickhouse table can have many consumers) + + auto & assigments_topics = assert_cast(assert_cast(*res_columns[index]).getData()); + auto & assigments_topics_offsets = assert_cast(*res_columns[index++]).getOffsets(); + + auto & assigments_partition_id = assert_cast(assert_cast(*res_columns[index]).getData()); + auto & assigments_partition_id_offsets = assert_cast(*res_columns[index++]).getOffsets(); + + auto & assigments_current_offset = assert_cast(assert_cast(*res_columns[index]).getData()); + auto & assigments_current_offset_offsets = assert_cast(*res_columns[index++]).getOffsets(); + + // auto & assignments.topic = assert_cast(std::make_shared())}, + // auto & assignments.partition_id = assert_cast(std::make_shared())}, + // auto & assignments.current_offset = assert_cast(std::make_shared())}, + + + + // auto & last_exception_time = assert_cast(*res_columns[index++]); + // auto & last_exception = assert_cast(*res_columns[index++]); + // auto & last_poll_time = assert_cast(*res_columns[index++]); + // auto & num_messages_read = assert_cast(*res_columns[index++]); + // auto & last_commit_time = assert_cast(*res_columns[index++]); + // auto & num_commits = assert_cast(*res_columns[index++]); + // auto & last_rebalance_time = assert_cast(*res_columns[index++]); + // auto & num_rebalance_revocations = assert_cast(*res_columns[index++]); + // auto & num_rebalance_assignments = assert_cast(*res_columns[index++]); + // auto & is_currently_used = assert_cast(*res_columns[index++]); + + + auto add_row = [&]() + { + // column_index.insertValue(idx); + // column_database.insertData(table_mark_dropped.table_id.getDatabaseName().data(), table_mark_dropped.table_id.getDatabaseName().size()); + // column_table.insertData(table_mark_dropped.table_id.getTableName().data(), table_mark_dropped.table_id.getTableName().size()); + // column_uuid.push_back(table_mark_dropped.table_id.uuid.toUnderType()); + // if (table_mark_dropped.table) + // column_engine.insertData(table_mark_dropped.table->getName().data(), table_mark_dropped.table->getName().size()); + // else + // column_engine.insertData({}, 0); + // column_metadata_dropped_path.insertData(table_mark_dropped.metadata_path.data(), table_mark_dropped.metadata_path.size()); + // column_table_dropped_time.insertValue(static_cast(table_mark_dropped.drop_time)); + + + + std::string fake_database = "fake_kafka_database"; + database.insertData(fake_database.data(), fake_database.size()); + + std::string fake_table = "fake_kafka_table"; + table.insertData(fake_table.data(), fake_table.size()); + + std::string fake_consumer_id = "fake_consumer_id"; + consumer_id.insertData(fake_consumer_id.data(), fake_consumer_id.size()); + + std::string fake_assigments_topic_1 = "fake_assigments_topic_1"; + std::string fake_assigments_topic_2 = "fake_assigments_topic_2"; + assigments_topics.insertData(fake_assigments_topic_1.data(), fake_assigments_topic_1.size()); + assigments_topics.insertData(fake_assigments_topic_2.data(), fake_assigments_topic_2.size()); + assigments_topics_offsets.push_back(2); + + std::string fake_partition_id_1 = "fake_partition_id_1"; + std::string fake_partition_id_2 = "fake_partition_id_2"; + assigments_partition_id.insertData(fake_partition_id_1.data(), fake_partition_id_1.size()); + assigments_partition_id.insertData(fake_partition_id_2.data(), fake_partition_id_2.size()); + assigments_partition_id_offsets.push_back(2); + + std::string fake_current_offset_1 = "fake_current_offset_1"; + std::string fake_current_offset_2 = "fake_current_offset_2"; + assigments_current_offset.insertData(fake_current_offset_1.data(), fake_current_offset_1.size()); + assigments_current_offset.insertData(fake_current_offset_2.data(), fake_current_offset_2.size()); + assigments_current_offset_offsets.push_back(2); + + +#if 0 + + + consumer_id.insertValue(assert_cast &>(*res_columns[index++]); //(number? or string? - single clickhouse table can have many consumers) + assignments.topic.insertValue( = assert_cast(std::make_shared())}, + assignments.partition_id.insertValue( = assert_cast(std::make_shared())}, + assignments.current_offset.insertValue( = assert_cast(std::make_shared())}, + last_exception_time.insertValue( = assert_cast &>(*res_columns[index++]); + + last_exception.insertValue( = assert_cast &>(*res_columns[index++]); + last_poll_time.insertValue( = assert_cast &>(*res_columns[index++]); + num_messages_read.insertValue( = assert_cast &>(*res_columns[index++]); + last_commit_time.insertValue( = assert_cast &>(*res_columns[index++]); + num_commits.insertValue( = assert_cast &>(*res_columns[index++]); +last_rebalance_time = assert_cast &>(*res_columns[index++]); +num_rebalance_revocations = assert_cast &>(*res_columns[index++]); +num_rebalance_assignments = assert_cast &>(*res_columns[index++]); +is_currently_used = assert_cast &>(*res_columns[index++]); +#endif + + }; + + // UInt32 idx = 0; + // for (const auto & table_mark_dropped : tables_mark_dropped) + // add_row(idx++, table_mark_dropped); + add_row(); +#pragma GCC diagnostic pop +} + +} diff --git a/src/Storages/System/StorageSystemKafkaConsumers.h b/src/Storages/System/StorageSystemKafkaConsumers.h new file mode 100644 index 00000000000..5c9d81f3675 --- /dev/null +++ b/src/Storages/System/StorageSystemKafkaConsumers.h @@ -0,0 +1,20 @@ +#pragma once + +#include + + +namespace DB +{ + +class StorageSystemKafkaConsumers final : public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemKafkaConsumers"; } + static NamesAndTypesList getNamesAndTypes(); + +protected: + using IStorageSystemOneBlock::IStorageSystemOneBlock; + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index f0c67e0f787..4c03f01141a 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -83,6 +83,7 @@ #include #include #include +#include #ifdef OS_LINUX #include @@ -144,6 +145,7 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database) attach(context, system_database, "backups"); attach(context, system_database, "schema_inference_cache"); attach(context, system_database, "dropped_tables"); + attach(context, system_database, "kafka_consumers"); #ifdef OS_LINUX attach(context, system_database, "stack_trace"); #endif From 2304f342c90756c90a9c16f580eaa53ac08ed316 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 19 May 2023 10:01:21 +0000 Subject: [PATCH 350/777] system_kafka_consumers: table populated by fake data --- .../System/StorageSystemKafkaConsumers.cpp | 130 ++++++------------ 1 file changed, 44 insertions(+), 86 deletions(-) diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index c6f3fc3a4d9..21bd5919802 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -25,16 +25,16 @@ NamesAndTypesList StorageSystemKafkaConsumers::getNamesAndTypes() {"assignments.topic", std::make_shared(std::make_shared())}, {"assignments.partition_id", std::make_shared(std::make_shared())}, {"assignments.current_offset", std::make_shared(std::make_shared())}, - // {"last_exception_time", std::make_shared()}, - // {"last_exception", std::make_shared()}, - // {"last_poll_time", std::make_shared()}, - // {"num_messages_read", std::make_shared()}, - // {"last_commit_time", std::make_shared()}, - // {"num_commits", std::make_shared()}, - // {"last_rebalance_time", std::make_shared()}, - // {"num_rebalance_revocations", std::make_shared()}, - // {"num_rebalance_assignments", std::make_shared()}, - // {"is_currently_used", std::make_shared()}, + {"last_exception_time", std::make_shared()}, + {"last_exception", std::make_shared()}, + {"last_poll_time", std::make_shared()}, + {"num_messages_read", std::make_shared()}, + {"last_commit_time", std::make_shared()}, + {"num_commits", std::make_shared()}, + {"last_rebalance_time", std::make_shared()}, + {"num_rebalance_revocations", std::make_shared()}, + {"num_rebalance_assignments", std::make_shared()}, + {"is_currently_used", std::make_shared()}, }; return names_and_types; } @@ -43,69 +43,39 @@ NamesAndTypesList StorageSystemKafkaConsumers::getNamesAndTypes() void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const { - -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-variable" - auto tables_mark_dropped = DatabaseCatalog::instance().getTablesMarkedDropped(); size_t index = 0; - // auto & column_index = assert_cast(*res_columns[index++]); - // auto & column_database = assert_cast(*res_columns[index++]); - // auto & column_table = assert_cast(*res_columns[index++]); - // auto & column_uuid = assert_cast(*res_columns[index++]).getData(); - // auto & column_engine = assert_cast(*res_columns[index++]); - // auto & column_metadata_dropped_path = assert_cast(*res_columns[index++]); - // auto & column_table_dropped_time = assert_cast(*res_columns[index++]); + + auto & database = assert_cast(*res_columns[index++]); + auto & table = assert_cast(*res_columns[index++]); + auto & consumer_id = assert_cast(*res_columns[index++]); //(number? or string? - single clickhouse table can have many consumers) + + auto & assigments_topics = assert_cast(assert_cast(*res_columns[index]).getData()); + auto & assigments_topics_offsets = assert_cast(*res_columns[index++]).getOffsets(); + + auto & assigments_partition_id = assert_cast(assert_cast(*res_columns[index]).getData()); + auto & assigments_partition_id_offsets = assert_cast(*res_columns[index++]).getOffsets(); + + auto & assigments_current_offset = assert_cast(assert_cast(*res_columns[index]).getData()); + auto & assigments_current_offset_offsets = assert_cast(*res_columns[index++]).getOffsets(); - auto & database = assert_cast(*res_columns[index++]); - auto & table = assert_cast(*res_columns[index++]); - auto & consumer_id = assert_cast(*res_columns[index++]); //(number? or string? - single clickhouse table can have many consumers) - - auto & assigments_topics = assert_cast(assert_cast(*res_columns[index]).getData()); - auto & assigments_topics_offsets = assert_cast(*res_columns[index++]).getOffsets(); - - auto & assigments_partition_id = assert_cast(assert_cast(*res_columns[index]).getData()); - auto & assigments_partition_id_offsets = assert_cast(*res_columns[index++]).getOffsets(); - - auto & assigments_current_offset = assert_cast(assert_cast(*res_columns[index]).getData()); - auto & assigments_current_offset_offsets = assert_cast(*res_columns[index++]).getOffsets(); - - // auto & assignments.topic = assert_cast(std::make_shared())}, - // auto & assignments.partition_id = assert_cast(std::make_shared())}, - // auto & assignments.current_offset = assert_cast(std::make_shared())}, - - - - // auto & last_exception_time = assert_cast(*res_columns[index++]); - // auto & last_exception = assert_cast(*res_columns[index++]); - // auto & last_poll_time = assert_cast(*res_columns[index++]); - // auto & num_messages_read = assert_cast(*res_columns[index++]); - // auto & last_commit_time = assert_cast(*res_columns[index++]); - // auto & num_commits = assert_cast(*res_columns[index++]); - // auto & last_rebalance_time = assert_cast(*res_columns[index++]); - // auto & num_rebalance_revocations = assert_cast(*res_columns[index++]); - // auto & num_rebalance_assignments = assert_cast(*res_columns[index++]); - // auto & is_currently_used = assert_cast(*res_columns[index++]); + auto & last_exception_time = assert_cast(*res_columns[index++]); + auto & last_exception = assert_cast(*res_columns[index++]); + auto & last_poll_time = assert_cast(*res_columns[index++]); + auto & num_messages_read = assert_cast(*res_columns[index++]); + auto & last_commit_time = assert_cast(*res_columns[index++]); + auto & num_commits = assert_cast(*res_columns[index++]); + auto & last_rebalance_time = assert_cast(*res_columns[index++]); + auto & num_rebalance_revocations = assert_cast(*res_columns[index++]); + auto & num_rebalance_assigments = assert_cast(*res_columns[index++]); + auto & is_currently_used = assert_cast(*res_columns[index++]); auto add_row = [&]() { - // column_index.insertValue(idx); - // column_database.insertData(table_mark_dropped.table_id.getDatabaseName().data(), table_mark_dropped.table_id.getDatabaseName().size()); - // column_table.insertData(table_mark_dropped.table_id.getTableName().data(), table_mark_dropped.table_id.getTableName().size()); - // column_uuid.push_back(table_mark_dropped.table_id.uuid.toUnderType()); - // if (table_mark_dropped.table) - // column_engine.insertData(table_mark_dropped.table->getName().data(), table_mark_dropped.table->getName().size()); - // else - // column_engine.insertData({}, 0); - // column_metadata_dropped_path.insertData(table_mark_dropped.metadata_path.data(), table_mark_dropped.metadata_path.size()); - // column_table_dropped_time.insertValue(static_cast(table_mark_dropped.drop_time)); - - - std::string fake_database = "fake_kafka_database"; database.insertData(fake_database.data(), fake_database.size()); @@ -133,34 +103,22 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context assigments_current_offset.insertData(fake_current_offset_2.data(), fake_current_offset_2.size()); assigments_current_offset_offsets.push_back(2); + last_exception_time.insert(0); -#if 0 - - - consumer_id.insertValue(assert_cast &>(*res_columns[index++]); //(number? or string? - single clickhouse table can have many consumers) - assignments.topic.insertValue( = assert_cast(std::make_shared())}, - assignments.partition_id.insertValue( = assert_cast(std::make_shared())}, - assignments.current_offset.insertValue( = assert_cast(std::make_shared())}, - last_exception_time.insertValue( = assert_cast &>(*res_columns[index++]); - - last_exception.insertValue( = assert_cast &>(*res_columns[index++]); - last_poll_time.insertValue( = assert_cast &>(*res_columns[index++]); - num_messages_read.insertValue( = assert_cast &>(*res_columns[index++]); - last_commit_time.insertValue( = assert_cast &>(*res_columns[index++]); - num_commits.insertValue( = assert_cast &>(*res_columns[index++]); -last_rebalance_time = assert_cast &>(*res_columns[index++]); -num_rebalance_revocations = assert_cast &>(*res_columns[index++]); -num_rebalance_assignments = assert_cast &>(*res_columns[index++]); -is_currently_used = assert_cast &>(*res_columns[index++]); -#endif + std::string fake_last_exception = "fake_last_exception"; + last_exception.insertData(fake_last_exception.data(), fake_last_exception.size()); + last_poll_time.insert(0); + num_messages_read.insert(0); + last_commit_time.insert(0); + num_commits.insert(0); + last_rebalance_time.insert(0); + num_rebalance_revocations.insert(0); + num_rebalance_assigments.insert(0); + is_currently_used.insert(0); }; - // UInt32 idx = 0; - // for (const auto & table_mark_dropped : tables_mark_dropped) - // add_row(idx++, table_mark_dropped); add_row(); -#pragma GCC diagnostic pop } } From a947c7a91e61d7f7bc03822d511964da29b369dd Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 8 Jun 2023 09:42:42 +0000 Subject: [PATCH 351/777] system_kafka_consumers: start integrating with real kafka --- src/CMakeLists.txt | 2 +- .../System/StorageSystemKafkaConsumers.cpp | 51 +++++++++++++++---- 2 files changed, 43 insertions(+), 10 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index ca38a447c70..14b8030c76f 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -383,7 +383,7 @@ target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::abseil_swiss_tabl dbms_target_link_libraries(PUBLIC ch_contrib::roaring) if (TARGET ch_contrib::rdkafka) - dbms_target_link_libraries(PRIVATE ch_contrib::rdkafka ch_contrib::cppkafka) + dbms_target_link_libraries(PUBLIC ch_contrib::rdkafka ch_contrib::cppkafka) endif() if (TARGET ch_contrib::nats_io) diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 21bd5919802..dc5679c83f9 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -7,6 +6,9 @@ #include #include #include +#include +#include +#include #include #include @@ -41,7 +43,7 @@ NamesAndTypesList StorageSystemKafkaConsumers::getNamesAndTypes() -void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const +void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { auto tables_mark_dropped = DatabaseCatalog::instance().getTablesMarkedDropped(); @@ -54,12 +56,15 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context auto & assigments_topics = assert_cast(assert_cast(*res_columns[index]).getData()); auto & assigments_topics_offsets = assert_cast(*res_columns[index++]).getOffsets(); + size_t assigments_topics_last_offset = 0; auto & assigments_partition_id = assert_cast(assert_cast(*res_columns[index]).getData()); auto & assigments_partition_id_offsets = assert_cast(*res_columns[index++]).getOffsets(); + size_t assigments_partition_id_last_offset = 0; auto & assigments_current_offset = assert_cast(assert_cast(*res_columns[index]).getData()); auto & assigments_current_offset_offsets = assert_cast(*res_columns[index++]).getOffsets(); + size_t assigments_current_offset_last_offset = 0; auto & last_exception_time = assert_cast(*res_columns[index++]); @@ -73,14 +78,22 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context auto & num_rebalance_assigments = assert_cast(*res_columns[index++]); auto & is_currently_used = assert_cast(*res_columns[index++]); + const auto access = context->getAccess(); - auto add_row = [&]() + auto add_row = [&](const DatabaseTablesIteratorPtr & it) { + if (!access->isGranted(AccessType::SHOW_TABLES, it->databaseName(), it->name())) + { + return; + } + std::string fake_database = "fake_kafka_database"; - database.insertData(fake_database.data(), fake_database.size()); + // database.insertData(fake_database.data(), fake_database.size()); + database.insertData(it->databaseName().data(), it->databaseName().size()); std::string fake_table = "fake_kafka_table"; - table.insertData(fake_table.data(), fake_table.size()); + // table.insertData(fake_table.data(), fake_table.size()); + table.insertData(it->name().data(), it->name().size()); std::string fake_consumer_id = "fake_consumer_id"; consumer_id.insertData(fake_consumer_id.data(), fake_consumer_id.size()); @@ -89,19 +102,22 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context std::string fake_assigments_topic_2 = "fake_assigments_topic_2"; assigments_topics.insertData(fake_assigments_topic_1.data(), fake_assigments_topic_1.size()); assigments_topics.insertData(fake_assigments_topic_2.data(), fake_assigments_topic_2.size()); - assigments_topics_offsets.push_back(2); + assigments_topics_last_offset += 2; + assigments_topics_offsets.push_back(assigments_topics_last_offset); std::string fake_partition_id_1 = "fake_partition_id_1"; std::string fake_partition_id_2 = "fake_partition_id_2"; assigments_partition_id.insertData(fake_partition_id_1.data(), fake_partition_id_1.size()); assigments_partition_id.insertData(fake_partition_id_2.data(), fake_partition_id_2.size()); - assigments_partition_id_offsets.push_back(2); + assigments_partition_id_last_offset += 2; + assigments_partition_id_offsets.push_back(assigments_partition_id_last_offset); std::string fake_current_offset_1 = "fake_current_offset_1"; std::string fake_current_offset_2 = "fake_current_offset_2"; assigments_current_offset.insertData(fake_current_offset_1.data(), fake_current_offset_1.size()); assigments_current_offset.insertData(fake_current_offset_2.data(), fake_current_offset_2.size()); - assigments_current_offset_offsets.push_back(2); + assigments_current_offset_last_offset += 2; + assigments_current_offset_offsets.push_back(assigments_current_offset_last_offset); last_exception_time.insert(0); @@ -118,7 +134,24 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context is_currently_used.insert(0); }; - add_row(); + const bool show_tables_granted = access->isGranted(AccessType::SHOW_TABLES); + + if (show_tables_granted) + { + auto databases = DatabaseCatalog::instance().getDatabases(); + for (const auto & db : databases) + { + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) + { + StoragePtr storage = iterator->table(); + if (auto * kafka_table = dynamic_cast(storage.get())) + { + add_row(iterator); + } + } + } + + } } } From 36fb330811b26b4abb1ba53df49156a1246b4c1a Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 8 Jun 2023 20:42:13 +0000 Subject: [PATCH 352/777] system_kafka_consumers: assignments --- src/Storages/Kafka/KafkaConsumer.h | 5 + src/Storages/Kafka/StorageKafka.h | 4 + .../System/StorageSystemKafkaConsumers.cpp | 123 +++++++++++------- 3 files changed, 86 insertions(+), 46 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index feda51a682e..3154e0b8160 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -20,6 +20,8 @@ namespace Poco namespace DB { +class StorageSystemKafkaConsumers; + using ConsumerPtr = std::shared_ptr; class KafkaConsumer @@ -111,6 +113,9 @@ private: /// Return number of messages with an error. size_t filterMessageErrors(); ReadBufferPtr getNextMessage(); + + friend class DB::StorageSystemKafkaConsumers; + }; } diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 72875e77b12..e5b3aedf684 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -23,6 +23,8 @@ class Configuration; namespace DB { +class StorageSystemKafkaConsumers; + struct StorageKafkaInterceptors; using KafkaConsumerPtr = std::shared_ptr; @@ -142,6 +144,8 @@ private: bool streamToViews(); bool checkDependencies(const StorageID & table_id); + + friend class DB::StorageSystemKafkaConsumers; }; } diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index dc5679c83f9..a0e6a5adf94 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -25,8 +25,8 @@ NamesAndTypesList StorageSystemKafkaConsumers::getNamesAndTypes() {"table", std::make_shared()}, {"consumer_id", std::make_shared()}, //(number? or string? - single clickhouse table can have many consumers) {"assignments.topic", std::make_shared(std::make_shared())}, - {"assignments.partition_id", std::make_shared(std::make_shared())}, - {"assignments.current_offset", std::make_shared(std::make_shared())}, + {"assignments.partition_id", std::make_shared(std::make_shared())}, + {"assignments.current_offset", std::make_shared(std::make_shared())}, {"last_exception_time", std::make_shared()}, {"last_exception", std::make_shared()}, {"last_poll_time", std::make_shared()}, @@ -56,15 +56,12 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context auto & assigments_topics = assert_cast(assert_cast(*res_columns[index]).getData()); auto & assigments_topics_offsets = assert_cast(*res_columns[index++]).getOffsets(); - size_t assigments_topics_last_offset = 0; - auto & assigments_partition_id = assert_cast(assert_cast(*res_columns[index]).getData()); + auto & assigments_partition_id = assert_cast(assert_cast(*res_columns[index]).getData()); auto & assigments_partition_id_offsets = assert_cast(*res_columns[index++]).getOffsets(); - size_t assigments_partition_id_last_offset = 0; - auto & assigments_current_offset = assert_cast(assert_cast(*res_columns[index]).getData()); + auto & assigments_current_offset = assert_cast(assert_cast(*res_columns[index]).getData()); auto & assigments_current_offset_offsets = assert_cast(*res_columns[index++]).getOffsets(); - size_t assigments_current_offset_last_offset = 0; auto & last_exception_time = assert_cast(*res_columns[index++]); @@ -79,59 +76,93 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context auto & is_currently_used = assert_cast(*res_columns[index++]); const auto access = context->getAccess(); + size_t last_assignment_num = 0; - auto add_row = [&](const DatabaseTablesIteratorPtr & it) + auto add_row = [&](const DatabaseTablesIteratorPtr & it, StorageKafka * storage_kafka_ptr) { if (!access->isGranted(AccessType::SHOW_TABLES, it->databaseName(), it->name())) { return; } - std::string fake_database = "fake_kafka_database"; - // database.insertData(fake_database.data(), fake_database.size()); - database.insertData(it->databaseName().data(), it->databaseName().size()); + std::string database_str = it->databaseName(); + std::string table_str = it->name(); - std::string fake_table = "fake_kafka_table"; - // table.insertData(fake_table.data(), fake_table.size()); - table.insertData(it->name().data(), it->name().size()); + for (auto consumer : storage_kafka_ptr->consumers) + { + auto & cpp_consumer = consumer->consumer; - std::string fake_consumer_id = "fake_consumer_id"; - consumer_id.insertData(fake_consumer_id.data(), fake_consumer_id.size()); + database.insertData(database_str.data(), database_str.size()); + table.insertData(table_str.data(), table_str.size()); - std::string fake_assigments_topic_1 = "fake_assigments_topic_1"; - std::string fake_assigments_topic_2 = "fake_assigments_topic_2"; - assigments_topics.insertData(fake_assigments_topic_1.data(), fake_assigments_topic_1.size()); - assigments_topics.insertData(fake_assigments_topic_2.data(), fake_assigments_topic_2.size()); - assigments_topics_last_offset += 2; - assigments_topics_offsets.push_back(assigments_topics_last_offset); + std::string consumer_id_str = cpp_consumer->get_member_id(); + consumer_id.insertData(consumer_id_str.data(), consumer_id_str.size()); - std::string fake_partition_id_1 = "fake_partition_id_1"; - std::string fake_partition_id_2 = "fake_partition_id_2"; - assigments_partition_id.insertData(fake_partition_id_1.data(), fake_partition_id_1.size()); - assigments_partition_id.insertData(fake_partition_id_2.data(), fake_partition_id_2.size()); - assigments_partition_id_last_offset += 2; - assigments_partition_id_offsets.push_back(assigments_partition_id_last_offset); + if (consumer->assignment.has_value() && consumer->assignment.value().size() > 0) + { + for (const auto & assignment : consumer->assignment.value()) + { + const auto & topic_str = assignment.get_topic(); + assigments_topics.insertData(topic_str.data(), topic_str.size()); - std::string fake_current_offset_1 = "fake_current_offset_1"; - std::string fake_current_offset_2 = "fake_current_offset_2"; - assigments_current_offset.insertData(fake_current_offset_1.data(), fake_current_offset_1.size()); - assigments_current_offset.insertData(fake_current_offset_2.data(), fake_current_offset_2.size()); - assigments_current_offset_last_offset += 2; - assigments_current_offset_offsets.push_back(assigments_current_offset_last_offset); + assigments_partition_id.insert(assignment.get_partition()); + assigments_current_offset.insert(assignment.get_offset()); + } + last_assignment_num += consumer->assignment.value().size(); - last_exception_time.insert(0); + } + else + { + std::string fake_assigments_topic = "no assigned topic"; + assigments_topics.insertData(fake_assigments_topic.data(), fake_assigments_topic.size()); - std::string fake_last_exception = "fake_last_exception"; - last_exception.insertData(fake_last_exception.data(), fake_last_exception.size()); + assigments_partition_id.insert(0); + assigments_current_offset.insert(0); + + last_assignment_num += 1; + } + assigments_topics_offsets.push_back(last_assignment_num); + assigments_partition_id_offsets.push_back(last_assignment_num); + assigments_current_offset_offsets.push_back(last_assignment_num); + + + + // std::string fake_assigments_topic_1 = "fake_assigments_topic_1"; + // std::string fake_assigments_topic_2 = "fake_assigments_topic_2"; + // assigments_topics.insertData(fake_assigments_topic_1.data(), fake_assigments_topic_1.size()); + // assigments_topics.insertData(fake_assigments_topic_2.data(), fake_assigments_topic_2.size()); + // assigments_topics_last_offset += 2; + // assigments_topics_offsets.push_back(assigments_topics_last_offset); + + // std::string fake_partition_id_1 = "fake_partition_id_1"; + // std::string fake_partition_id_2 = "fake_partition_id_2"; + // assigments_partition_id.insertData(fake_partition_id_1.data(), fake_partition_id_1.size()); + // assigments_partition_id.insertData(fake_partition_id_2.data(), fake_partition_id_2.size()); + // assigments_partition_id_last_offset += 2; + // assigments_partition_id_offsets.push_back(assigments_partition_id_last_offset); + + // std::string fake_current_offset_1 = "fake_current_offset_1"; + // std::string fake_current_offset_2 = "fake_current_offset_2"; + // assigments_current_offset.insertData(fake_current_offset_1.data(), fake_current_offset_1.size()); + // assigments_current_offset.insertData(fake_current_offset_2.data(), fake_current_offset_2.size()); + // assigments_current_offset_last_offset += 2; + // assigments_current_offset_offsets.push_back(assigments_current_offset_last_offset); + + last_exception_time.insert(0); + + std::string fake_last_exception = "fake_last_exception"; + last_exception.insertData(fake_last_exception.data(), fake_last_exception.size()); + + last_poll_time.insert(0); + num_messages_read.insert(0); + last_commit_time.insert(0); + num_commits.insert(0); + last_rebalance_time.insert(0); + num_rebalance_revocations.insert(0); + num_rebalance_assigments.insert(0); + is_currently_used.insert(0); + } - last_poll_time.insert(0); - num_messages_read.insert(0); - last_commit_time.insert(0); - num_commits.insert(0); - last_rebalance_time.insert(0); - num_rebalance_revocations.insert(0); - num_rebalance_assigments.insert(0); - is_currently_used.insert(0); }; const bool show_tables_granted = access->isGranted(AccessType::SHOW_TABLES); @@ -146,7 +177,7 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context StoragePtr storage = iterator->table(); if (auto * kafka_table = dynamic_cast(storage.get())) { - add_row(iterator); + add_row(iterator, kafka_table); } } } From 957787d96a7486c0424aa96f90e51eb100b29153 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 12 Jun 2023 14:41:17 +0000 Subject: [PATCH 353/777] system_kafka_consumers: kind of works, test added --- src/Storages/Kafka/KafkaConsumer.cpp | 27 ++++++++++ src/Storages/Kafka/KafkaConsumer.h | 15 ++++++ .../System/StorageSystemKafkaConsumers.cpp | 30 ++++++----- tests/integration/test_storage_kafka/test.py | 53 +++++++++++++++++++ 4 files changed, 112 insertions(+), 13 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index b2e6129c61c..ba6948f80ee 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -76,9 +76,11 @@ KafkaConsumer::KafkaConsumer( { LOG_TRACE(log, "Topics/partitions assigned: {}", topic_partitions); CurrentMetrics::add(CurrentMetrics::KafkaConsumersWithAssignment, 1); + } assignment = topic_partitions; + num_rebalance_assignments++; }); // called (synchronously, during poll) when we leave the consumer group @@ -106,6 +108,8 @@ KafkaConsumer::KafkaConsumer( cleanUnprocessed(); stalled_status = REBALANCE_HAPPENED; + last_rebalance_timestamp_usec = static_cast(Poco::Timestamp().epochTime()); + assignment.reset(); waited_for_assignment = 0; @@ -118,6 +122,7 @@ KafkaConsumer::KafkaConsumer( // { // LOG_WARNING(log, "Commit error: {}", e.what()); // } + num_rebalance_revocations++; }); consumer->set_rebalance_error_callback([this](cppkafka::Error err) @@ -251,6 +256,8 @@ void KafkaConsumer::commit() consumer->commit(); committed = true; print_offsets("Committed offset", consumer->get_offsets_committed(consumer->get_assignment())); + last_commit_timestamp_usec = static_cast(Poco::Timestamp().epochTime()); + num_commits += 1; } catch (const cppkafka::HandleException & e) { @@ -383,6 +390,7 @@ ReadBufferPtr KafkaConsumer::consume() if (intermediate_commit) commit(); + while (true) { stalled_status = NO_MESSAGES_RETURNED; @@ -399,6 +407,8 @@ ReadBufferPtr KafkaConsumer::consume() /// Don't drop old messages immediately, since we may need them for virtual columns. auto new_messages = consumer->poll_batch(batch_size, std::chrono::milliseconds(actual_poll_timeout_ms)); + last_poll_timestamp_usec = static_cast(Poco::Timestamp().epochTime()); + num_messages_read += new_messages.size(); resetIfStopped(); if (stalled_status == CONSUMER_STOPPED) @@ -527,4 +537,21 @@ void KafkaConsumer::storeLastReadMessageOffset() } } +void KafkaConsumer::setExceptionInfo(const String & text) +{ + std::lock_guard lock(exception_mutex); + + last_exception_text = text; + last_exception_timestamp_usec = static_cast(Poco::Timestamp().epochTime()); +} + +std::pair KafkaConsumer::getExceptionInfo() const +{ + std::lock_guard lock(exception_mutex); + return std::make_pair(last_exception_text, last_exception_timestamp_usec); +} + + + + } diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index 3154e0b8160..e4cd332a1f5 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -71,6 +71,9 @@ public: auto currentTimestamp() const { return current[-1].get_timestamp(); } const auto & currentHeaderList() const { return current[-1].get_header_list(); } String currentPayload() const { return current[-1].get_payload(); } + void setExceptionInfo(const String & text); + std::pair getExceptionInfo() const; + private: using Messages = std::vector; @@ -107,6 +110,18 @@ private: std::optional assignment; const Names topics; + String last_exception_text; + Int64 last_exception_timestamp_usec = 0; + mutable std::mutex exception_mutex; + + std::atomic last_poll_timestamp_usec = 0; + std::atomic num_messages_read = 0; + std::atomic last_commit_timestamp_usec = 0; + std::atomic num_commits = 0; + std::atomic last_rebalance_timestamp_usec = 0; + std::atomic num_rebalance_assignments = 0; + std::atomic num_rebalance_revocations = 0; + void drain(); void cleanUnprocessed(); void resetIfStopped(); diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index a0e6a5adf94..0b80d17d05a 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -26,7 +27,7 @@ NamesAndTypesList StorageSystemKafkaConsumers::getNamesAndTypes() {"consumer_id", std::make_shared()}, //(number? or string? - single clickhouse table can have many consumers) {"assignments.topic", std::make_shared(std::make_shared())}, {"assignments.partition_id", std::make_shared(std::make_shared())}, - {"assignments.current_offset", std::make_shared(std::make_shared())}, + {"assignments.current_offset", std::make_shared(std::make_shared())}, {"last_exception_time", std::make_shared()}, {"last_exception", std::make_shared()}, {"last_poll_time", std::make_shared()}, @@ -60,7 +61,7 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context auto & assigments_partition_id = assert_cast(assert_cast(*res_columns[index]).getData()); auto & assigments_partition_id_offsets = assert_cast(*res_columns[index++]).getOffsets(); - auto & assigments_current_offset = assert_cast(assert_cast(*res_columns[index]).getData()); + auto & assigments_current_offset = assert_cast(assert_cast(*res_columns[index]).getData()); auto & assigments_current_offset_offsets = assert_cast(*res_columns[index++]).getOffsets(); @@ -148,19 +149,22 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context // assigments_current_offset_last_offset += 2; // assigments_current_offset_offsets.push_back(assigments_current_offset_last_offset); - last_exception_time.insert(0); + auto exception_info = consumer->getExceptionInfo(); - std::string fake_last_exception = "fake_last_exception"; - last_exception.insertData(fake_last_exception.data(), fake_last_exception.size()); - last_poll_time.insert(0); - num_messages_read.insert(0); - last_commit_time.insert(0); - num_commits.insert(0); - last_rebalance_time.insert(0); - num_rebalance_revocations.insert(0); - num_rebalance_assigments.insert(0); - is_currently_used.insert(0); + last_exception.insertData(exception_info.first.data(), exception_info.first.size()); + last_exception_time.insert(exception_info.second); + + last_poll_time.insert(consumer->last_poll_timestamp_usec.load()); + num_messages_read.insert(consumer->num_messages_read.load()); + last_commit_time.insert(consumer->last_commit_timestamp_usec.load()); + num_commits.insert(consumer->num_commits.load()); + last_rebalance_time.insert(consumer->last_rebalance_timestamp_usec.load()); + + num_rebalance_revocations.insert(consumer->num_rebalance_revocations.load()); + num_rebalance_assigments.insert(consumer->num_rebalance_assignments.load()); + + is_currently_used.insert(consumer->stalled_status != KafkaConsumer::CONSUMER_STOPPED && consumer->assignment.has_value() && consumer->assignment.value().size() > 0); } }; diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 28919cc685e..d54a88bafb6 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4545,6 +4545,59 @@ def test_block_based_formats_2(kafka_cluster): kafka_delete_topic(admin_client, format_name) +def test_system_kafka_consumers(kafka_cluster): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + topic = "system_kafka_cons" + kafka_create_topic(admin_client, topic) + + # Check that format_csv_delimiter parameter works now - as part of all available format settings. + kafka_produce( + kafka_cluster, + topic, + ["1|foo", "2|bar", "42|answer", "100|multi\n101|row\n103|message"], + ) + + instance.query( + f""" + DROP TABLE IF EXISTS test.kafka; + + CREATE TABLE test.kafka (a UInt64, b String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic}', + kafka_group_name = '{topic}', + kafka_commit_on_select = 1, + kafka_format = 'CSV', + kafka_row_delimiter = '\\n', + format_csv_delimiter = '|'; + """ + ) + + # result_system_kafka_consumers = instance.query("SELECT * FROM system.kafka_consumers format Vertical") + + result = instance.query("SELECT * FROM test.kafka ORDER BY a;") + + expected = """\ +1 foo +2 bar +42 answer +100 multi +101 row +103 message +""" + assert TSV(result) == TSV(expected) + + result_system_kafka_consumers = instance.query("SELECT * FROM system.kafka_consumers format Vertical") + + logging.debug(f"result_system_kafka_consumers: {result_system_kafka_consumers}") + assert result_system_kafka_consumers == 'empty' + + + kafka_delete_topic(admin_client, topic) + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 5e1c72a0d00313dd3fbc60d3683f0c32ef5db5e6 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 12 Jun 2023 21:29:23 +0000 Subject: [PATCH 354/777] system_kafka_consumers: test with rebalance --- tests/integration/test_storage_kafka/test.py | 88 +++++++++++++++++--- 1 file changed, 78 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index d54a88bafb6..2652c7d068f 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1186,6 +1186,7 @@ def test_kafka_consumer_hang2(kafka_cluster): instance.query( """ DROP TABLE IF EXISTS test.kafka; + DROP TABLE IF EXISTS test.kafka2; CREATE TABLE test.kafka (key UInt64, value UInt64) ENGINE = Kafka @@ -4580,22 +4581,89 @@ def test_system_kafka_consumers(kafka_cluster): result = instance.query("SELECT * FROM test.kafka ORDER BY a;") - expected = """\ -1 foo -2 bar -42 answer -100 multi -101 row -103 message -""" - assert TSV(result) == TSV(expected) - result_system_kafka_consumers = instance.query("SELECT * FROM system.kafka_consumers format Vertical") + result_system_kafka_consumers = instance.query("SELECT * FROM system.kafka_consumers WHERE database='test' and table='kafka' format Vertical") logging.debug(f"result_system_kafka_consumers: {result_system_kafka_consumers}") assert result_system_kafka_consumers == 'empty' + kafka_delete_topic(admin_client, topic) + +def test_system_kafka_consumers_rebalance(kafka_cluster): + # based on test_kafka_consumer_hang2 + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + topic = "system_kafka_cons2" + kafka_create_topic(admin_client, topic) + + # # Check that format_csv_delimiter parameter works now - as part of all available format settings. + # kafka_produce( + # kafka_cluster, + # topic, + # ["1|foo", "2|bar", "42|answer", "100|multi\n101|row\n103|message"], + # ) + + instance.query( + f""" + DROP TABLE IF EXISTS test.kafka; + DROP TABLE IF EXISTS test.kafka2; + + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic}', + kafka_group_name = '{topic}', + kafka_commit_on_select = 1, + kafka_format = 'JSONEachRow'; + + CREATE TABLE test.kafka2 (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic}', + kafka_commit_on_select = 1, + kafka_group_name = '{topic}', + kafka_format = 'JSONEachRow'; + """ + ) + + # first consumer subscribe the topic, try to poll some data, and go to rest + instance.query("SELECT * FROM test.kafka") + + # second consumer do the same leading to rebalance in the first + # consumer, try to poll some data + instance.query("SELECT * FROM test.kafka2") + + + instance.query("SELECT * FROM test.kafka") + + # second consumer do the same leading to rebalance in the first + # consumer, try to poll some data + instance.query("SELECT * FROM test.kafka2") + + instance.query("SELECT * FROM test.kafka") + + # second consumer do the same leading to rebalance in the first + # consumer, try to poll some data + instance.query("SELECT * FROM test.kafka2") + + + result_system_kafka_consumers = instance.query("SELECT * FROM system.kafka_consumers WHERE database='test' and table IN ('kafka', 'kafka2') format Vertical") + + logging.debug(f"result_system_kafka_consumers (1): {result_system_kafka_consumers}") + # assert result_system_kafka_consumers == 'empty' + + + + + instance.query("DROP TABLE test.kafka") + instance.query("DROP TABLE test.kafka2") + + + + kafka_delete_topic(admin_client, topic) if __name__ == "__main__": From 88f5dd5db8923c4c2cd54b22e1860ca905a4a2a5 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 14 Jun 2023 15:44:36 +0000 Subject: [PATCH 355/777] system_kafka_consumers: integration tests passed --- src/Storages/Kafka/StorageKafka.cpp | 6 + .../System/StorageSystemKafkaConsumers.cpp | 25 +---- tests/integration/test_storage_kafka/test.py | 106 +++++++++++++----- 3 files changed, 89 insertions(+), 48 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index a7315eb51ea..f4ffa30cad1 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -642,6 +642,12 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & kafka_config) LOG_IMPL(log, client_logs_level, poco_level, "[rdk:{}] {}", facility, message); }); + // kafka_config.set_stats_callback([this](cppkafka::KafkaHandleBase &, const std::string & json_string) + // { + // LOG_DEBUG(log, "kafka statistics {}", json_string); + // }); + + // Configure interceptor to change thread name // // TODO: add interceptors support into the cppkafka. diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 0b80d17d05a..292e12dae51 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -89,6 +89,8 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context std::string database_str = it->databaseName(); std::string table_str = it->name(); + std::lock_guard lock(storage_kafka_ptr->mutex); + for (auto consumer : storage_kafka_ptr->consumers) { auto & cpp_consumer = consumer->consumer; @@ -126,29 +128,6 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context assigments_partition_id_offsets.push_back(last_assignment_num); assigments_current_offset_offsets.push_back(last_assignment_num); - - - // std::string fake_assigments_topic_1 = "fake_assigments_topic_1"; - // std::string fake_assigments_topic_2 = "fake_assigments_topic_2"; - // assigments_topics.insertData(fake_assigments_topic_1.data(), fake_assigments_topic_1.size()); - // assigments_topics.insertData(fake_assigments_topic_2.data(), fake_assigments_topic_2.size()); - // assigments_topics_last_offset += 2; - // assigments_topics_offsets.push_back(assigments_topics_last_offset); - - // std::string fake_partition_id_1 = "fake_partition_id_1"; - // std::string fake_partition_id_2 = "fake_partition_id_2"; - // assigments_partition_id.insertData(fake_partition_id_1.data(), fake_partition_id_1.size()); - // assigments_partition_id.insertData(fake_partition_id_2.data(), fake_partition_id_2.size()); - // assigments_partition_id_last_offset += 2; - // assigments_partition_id_offsets.push_back(assigments_partition_id_last_offset); - - // std::string fake_current_offset_1 = "fake_current_offset_1"; - // std::string fake_current_offset_2 = "fake_current_offset_2"; - // assigments_current_offset.insertData(fake_current_offset_1.data(), fake_current_offset_1.size()); - // assigments_current_offset.insertData(fake_current_offset_2.data(), fake_current_offset_2.size()); - // assigments_current_offset_last_offset += 2; - // assigments_current_offset_offsets.push_back(assigments_current_offset_last_offset); - auto exception_info = consumer->getExceptionInfo(); diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 2652c7d068f..61e6a12f8d5 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4582,11 +4582,39 @@ def test_system_kafka_consumers(kafka_cluster): result = instance.query("SELECT * FROM test.kafka ORDER BY a;") - result_system_kafka_consumers = instance.query("SELECT * FROM system.kafka_consumers WHERE database='test' and table='kafka' format Vertical") + result_system_kafka_consumers = instance.query(""" + create or replace function stable_timestamp as + (d)->multiIf(d==toDateTime('1970-01-01 00:00:00'), 'never', abs(dateDiff('second', d, now())) < 20, 'now', toString(d)); + SELECT database, table, length(consumer_id), assignments.topic, assignments.partition_id, + assignments.current_offset, stable_timestamp(last_exception_time) as last_exception_time_, + if(length(last_exception)>0, last_exception, 'no exception') as last_exception_, + stable_timestamp(last_poll_time) as last_poll_time_, num_messages_read, stable_timestamp(last_commit_time) as last_commit_time_, + num_commits, stable_timestamp(last_rebalance_time) as last_rebalance_time_, + num_rebalance_revocations, num_rebalance_assignments, is_currently_used + FROM system.kafka_consumers WHERE database='test' and table='kafka' format Vertical; + """ + ) logging.debug(f"result_system_kafka_consumers: {result_system_kafka_consumers}") - assert result_system_kafka_consumers == 'empty' - + assert result_system_kafka_consumers == """Row 1: +────── +database: test +table: kafka +length(consumer_id): 67 +assignments.topic: ['system_kafka_cons'] +assignments.partition_id: [0] +assignments.current_offset: [-1001] +last_exception_time_: never +last_exception_: no exception +last_poll_time_: now +num_messages_read: 4 +last_commit_time_: now +num_commits: 1 +last_rebalance_time_: never +num_rebalance_revocations: 0 +num_rebalance_assignments: 1 +is_currently_used: 1 +""" kafka_delete_topic(admin_client, topic) @@ -4599,13 +4627,6 @@ def test_system_kafka_consumers_rebalance(kafka_cluster): topic = "system_kafka_cons2" kafka_create_topic(admin_client, topic) - # # Check that format_csv_delimiter parameter works now - as part of all available format settings. - # kafka_produce( - # kafka_cluster, - # topic, - # ["1|foo", "2|bar", "42|answer", "100|multi\n101|row\n103|message"], - # ) - instance.query( f""" DROP TABLE IF EXISTS test.kafka; @@ -4638,32 +4659,67 @@ def test_system_kafka_consumers_rebalance(kafka_cluster): instance.query("SELECT * FROM test.kafka") - - # second consumer do the same leading to rebalance in the first - # consumer, try to poll some data instance.query("SELECT * FROM test.kafka2") - instance.query("SELECT * FROM test.kafka") - - # second consumer do the same leading to rebalance in the first - # consumer, try to poll some data instance.query("SELECT * FROM test.kafka2") - result_system_kafka_consumers = instance.query("SELECT * FROM system.kafka_consumers WHERE database='test' and table IN ('kafka', 'kafka2') format Vertical") - + result_system_kafka_consumers = instance.query( + """ + create or replace function stable_timestamp as + (d)->multiIf(d==toDateTime('1970-01-01 00:00:00'), 'never', abs(dateDiff('second', d, now())) < 20, 'now', toString(d)); + SELECT database, table, length(consumer_id), assignments.topic, assignments.partition_id, + assignments.current_offset, stable_timestamp(last_exception_time) as last_exception_time_, + if(length(last_exception)>0, last_exception, 'no exception') as last_exception_, + stable_timestamp(last_poll_time) as last_poll_time_, num_messages_read, stable_timestamp(last_commit_time) as last_commit_time_, + num_commits, stable_timestamp(last_rebalance_time) as last_rebalance_time_, + num_rebalance_revocations, num_rebalance_assignments, is_currently_used + FROM system.kafka_consumers WHERE database='test' and table IN ('kafka', 'kafka2') format Vertical; + """ + ) logging.debug(f"result_system_kafka_consumers (1): {result_system_kafka_consumers}") - # assert result_system_kafka_consumers == 'empty' - - + assert result_system_kafka_consumers == """Row 1: +────── +database: test +table: kafka +length(consumer_id): 67 +assignments.topic: ['system_kafka_cons2'] +assignments.partition_id: [0] +assignments.current_offset: [-1001] +last_exception_time_: never +last_exception_: no exception +last_poll_time_: now +num_messages_read: 0 +last_commit_time_: never +num_commits: 0 +last_rebalance_time_: now +num_rebalance_revocations: 1 +num_rebalance_assignments: 2 +is_currently_used: 1 +Row 2: +────── +database: test +table: kafka2 +length(consumer_id): 68 +assignments.topic: ['no assigned topic'] +assignments.partition_id: [0] +assignments.current_offset: [0] +last_exception_time_: never +last_exception_: no exception +last_poll_time_: now +num_messages_read: 0 +last_commit_time_: never +num_commits: 0 +last_rebalance_time_: never +num_rebalance_revocations: 0 +num_rebalance_assignments: 1 +is_currently_used: 0 +""" instance.query("DROP TABLE test.kafka") instance.query("DROP TABLE test.kafka2") - - - kafka_delete_topic(admin_client, topic) if __name__ == "__main__": From 82e8b40bf5177f7e93846bf73220ca638f2ce83e Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 14 Jun 2023 15:46:56 +0000 Subject: [PATCH 356/777] system_kafka_consumers: doc --- .../system-tables/kafka_consumers.md | 56 +++++++++++++++++++ 1 file changed, 56 insertions(+) create mode 100644 docs/en/operations/system-tables/kafka_consumers.md diff --git a/docs/en/operations/system-tables/kafka_consumers.md b/docs/en/operations/system-tables/kafka_consumers.md new file mode 100644 index 00000000000..ab8e7fe2501 --- /dev/null +++ b/docs/en/operations/system-tables/kafka_consumers.md @@ -0,0 +1,56 @@ +--- +slug: /en/operations/system-tables/kafka_consumers +--- +# kafka_consumers + +Contains information about Kafka consumers. +Applicable for [Kafka table engine](../../engines/table-engines/integration/kafka) (native ClickHouse integration) + +Columns: + +- `database` (String) - database of the table with Kafka Engine. +- `table` (String) - name of the table with Kafka Engine. +- `consumer_id` (String) - Kafka consumer identificator. Note, that a table can have many consumers. Specified by `kafka_num_consumers` parameter. +- `assignments.topic` (Array(String)) - Kafka topic. +- `assignments.partition_id` (Array(Int32)) - Kafka partiton id. Note, that only one consumer can be assigned to a partition. +- `assignments.current_offset` (Array(Int64)) - current offset. +- `last_exception_time`, (DateTime) - timestamp when the most recent exception was generated. +- `last_exception`, (String) - text of the most recent exception. +- `last_poll_time`, (DateTime) - timestamp of the most recent poll. +- `num_messages_read`, (UInt64) - number of messages read by the consumer. +- `last_commit_time`, (DateTime) - timestamp of the most recent poll. +- `num_commits`, (UInt64) - total number of commits for the consumer. +- `last_rebalance_time`, (DateTime) - timestamp of the most recent Kafka rebalance +- `num_rebalance_revocations`, (UInt64) - number of times the consumer was revoked its partitions +- `num_rebalance_assignments`, (UInt64) - number of times the consumer was assigned to Kafka cluster +- `is_currently_used`, (UInt8) - consumer is in use + +Example: + +``` sql +SELECT * +FROM system.kafka_consumers +FORMAT Vertical +``` + +``` text +Row 1: +────── +database: test +table: kafka +consumer_id: ClickHouse-instance-test-kafka-1caddc7f-f917-4bb1-ac55-e28bd103a4a0 +assignments.topic: ['system_kafka_cons'] +assignments.partition_id: [0] +assignments.current_offset: [18446744073709550615] +last_exception_time: 1970-01-01 00:00:00 +last_exception: +last_poll_time: 2006-11-09 18:47:47 +num_messages_read: 4 +last_commit_time: 2006-11-10 04:39:40 +num_commits: 1 +last_rebalance_time: 1970-01-01 00:00:00 +num_rebalance_revocations: 0 +num_rebalance_assignments: 1 +is_currently_used: 1 + +``` From 40fd53c3e366c1f677852013771706a897458271 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 14 Jun 2023 16:19:43 +0000 Subject: [PATCH 357/777] system_kafka_consumers: style check --- .../system-tables/kafka_consumers.md | 4 ++-- src/Storages/Kafka/KafkaConsumer.cpp | 3 --- .../System/StorageSystemKafkaConsumers.cpp | 2 -- tests/integration/test_storage_kafka/test.py | 18 ++++++++++++------ 4 files changed, 14 insertions(+), 13 deletions(-) diff --git a/docs/en/operations/system-tables/kafka_consumers.md b/docs/en/operations/system-tables/kafka_consumers.md index ab8e7fe2501..b60c4f982c1 100644 --- a/docs/en/operations/system-tables/kafka_consumers.md +++ b/docs/en/operations/system-tables/kafka_consumers.md @@ -10,9 +10,9 @@ Columns: - `database` (String) - database of the table with Kafka Engine. - `table` (String) - name of the table with Kafka Engine. -- `consumer_id` (String) - Kafka consumer identificator. Note, that a table can have many consumers. Specified by `kafka_num_consumers` parameter. +- `consumer_id` (String) - Kafka consumer identifier. Note, that a table can have many consumers. Specified by `kafka_num_consumers` parameter. - `assignments.topic` (Array(String)) - Kafka topic. -- `assignments.partition_id` (Array(Int32)) - Kafka partiton id. Note, that only one consumer can be assigned to a partition. +- `assignments.partition_id` (Array(Int32)) - Kafka partition id. Note, that only one consumer can be assigned to a partition. - `assignments.current_offset` (Array(Int64)) - current offset. - `last_exception_time`, (DateTime) - timestamp when the most recent exception was generated. - `last_exception`, (String) - text of the most recent exception. diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index ba6948f80ee..12feab92bde 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -551,7 +551,4 @@ std::pair KafkaConsumer::getExceptionInfo() const return std::make_pair(last_exception_text, last_exception_timestamp_usec); } - - - } diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 292e12dae51..e00e8a450a1 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -42,8 +42,6 @@ NamesAndTypesList StorageSystemKafkaConsumers::getNamesAndTypes() return names_and_types; } - - void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { auto tables_mark_dropped = DatabaseCatalog::instance().getTablesMarkedDropped(); diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 61e6a12f8d5..8184e492c81 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4581,8 +4581,8 @@ def test_system_kafka_consumers(kafka_cluster): result = instance.query("SELECT * FROM test.kafka ORDER BY a;") - - result_system_kafka_consumers = instance.query(""" + result_system_kafka_consumers = instance.query( + """ create or replace function stable_timestamp as (d)->multiIf(d==toDateTime('1970-01-01 00:00:00'), 'never', abs(dateDiff('second', d, now())) < 20, 'now', toString(d)); @@ -4596,7 +4596,9 @@ def test_system_kafka_consumers(kafka_cluster): """ ) logging.debug(f"result_system_kafka_consumers: {result_system_kafka_consumers}") - assert result_system_kafka_consumers == """Row 1: + assert ( + result_system_kafka_consumers + == """Row 1: ────── database: test table: kafka @@ -4615,9 +4617,11 @@ num_rebalance_revocations: 0 num_rebalance_assignments: 1 is_currently_used: 1 """ + ) kafka_delete_topic(admin_client, topic) + def test_system_kafka_consumers_rebalance(kafka_cluster): # based on test_kafka_consumer_hang2 admin_client = KafkaAdminClient( @@ -4657,13 +4661,11 @@ def test_system_kafka_consumers_rebalance(kafka_cluster): # consumer, try to poll some data instance.query("SELECT * FROM test.kafka2") - instance.query("SELECT * FROM test.kafka") instance.query("SELECT * FROM test.kafka2") instance.query("SELECT * FROM test.kafka") instance.query("SELECT * FROM test.kafka2") - result_system_kafka_consumers = instance.query( """ create or replace function stable_timestamp as @@ -4678,7 +4680,9 @@ def test_system_kafka_consumers_rebalance(kafka_cluster): """ ) logging.debug(f"result_system_kafka_consumers (1): {result_system_kafka_consumers}") - assert result_system_kafka_consumers == """Row 1: + assert ( + result_system_kafka_consumers + == """Row 1: ────── database: test table: kafka @@ -4716,12 +4720,14 @@ num_rebalance_revocations: 0 num_rebalance_assignments: 1 is_currently_used: 0 """ + ) instance.query("DROP TABLE test.kafka") instance.query("DROP TABLE test.kafka2") kafka_delete_topic(admin_client, topic) + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From 6258ab6cb433a867b5d89c40a87f44ecb096f9b1 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 14 Jun 2023 19:00:57 +0000 Subject: [PATCH 358/777] system_kafka_consumers: doc link fix, USE_RDKAFKA --- docs/en/operations/system-tables/kafka_consumers.md | 2 +- src/Storages/System/StorageSystemKafkaConsumers.cpp | 6 ++++++ src/Storages/System/StorageSystemKafkaConsumers.h | 7 +++++++ src/Storages/System/attachSystemTables.cpp | 5 +++++ 4 files changed, 19 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/kafka_consumers.md b/docs/en/operations/system-tables/kafka_consumers.md index b60c4f982c1..2baca8b6db4 100644 --- a/docs/en/operations/system-tables/kafka_consumers.md +++ b/docs/en/operations/system-tables/kafka_consumers.md @@ -4,7 +4,7 @@ slug: /en/operations/system-tables/kafka_consumers # kafka_consumers Contains information about Kafka consumers. -Applicable for [Kafka table engine](../../engines/table-engines/integration/kafka) (native ClickHouse integration) +Applicable for [Kafka table engine](../../engines/table-engines/integrations/kafka) (native ClickHouse integration) Columns: diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index e00e8a450a1..c7563a79691 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -1,3 +1,7 @@ +#include "config.h" + +#if USE_RDKAFKA + #include #include #include @@ -167,3 +171,5 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context } } + +#endif diff --git a/src/Storages/System/StorageSystemKafkaConsumers.h b/src/Storages/System/StorageSystemKafkaConsumers.h index 5c9d81f3675..eda3a39bc7e 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.h +++ b/src/Storages/System/StorageSystemKafkaConsumers.h @@ -1,5 +1,10 @@ #pragma once +#include "config.h" + +#if USE_RDKAFKA + + #include @@ -18,3 +23,5 @@ protected: }; } + +#endif diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 4c03f01141a..b770f784c01 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -83,7 +83,10 @@ #include #include #include + +#if USE_RDKAFKA #include +#endif #ifdef OS_LINUX #include @@ -145,7 +148,9 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database) attach(context, system_database, "backups"); attach(context, system_database, "schema_inference_cache"); attach(context, system_database, "dropped_tables"); +#if USE_RDKAFKA attach(context, system_database, "kafka_consumers"); +#endif #ifdef OS_LINUX attach(context, system_database, "stack_trace"); #endif From ae6e0469ad4a3154d1fa911d474f2acd6e537bf7 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 20 Jun 2023 08:08:20 +0000 Subject: [PATCH 359/777] system_kafka_consumers: rdfakfa stat with attempt to use json --- src/Storages/Kafka/StorageKafka.cpp | 10 ++++++---- src/Storages/Kafka/StorageKafka.h | 3 +++ .../System/StorageSystemKafkaConsumers.cpp | 14 ++++++++++++++ tests/integration/test_storage_kafka/test.py | 8 ++++++++ 4 files changed, 31 insertions(+), 4 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index f4ffa30cad1..83fdd58ab42 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -642,10 +642,12 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & kafka_config) LOG_IMPL(log, client_logs_level, poco_level, "[rdk:{}] {}", facility, message); }); - // kafka_config.set_stats_callback([this](cppkafka::KafkaHandleBase &, const std::string & json_string) - // { - // LOG_DEBUG(log, "kafka statistics {}", json_string); - // }); + kafka_config.set("statistics.interval.ms","1000000"); // 10 times per a second + kafka_config.set_stats_callback([this](cppkafka::KafkaHandleBase &, const std::string & stat_json_string) + { + LOG_DEBUG(log, "kafka statistics {}", stat_json_string); + rdkafka_stat = std::make_shared(stat_json_string); + }); // Configure interceptor to change thread name diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index e5b3aedf684..2cf1f582bd5 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -78,6 +78,7 @@ public: NamesAndTypesList getVirtuals() const override; Names getVirtualColumnNames() const; HandleKafkaErrorMode getHandleKafkaErrorMode() const { return kafka_settings->kafka_handle_error_mode; } + std::shared_ptr getRdkafkaStat() const { return rdkafka_stat; } private: // Configuration and state @@ -145,6 +146,8 @@ private: bool streamToViews(); bool checkDependencies(const StorageID & table_id); + std::shared_ptr rdkafka_stat; + friend class DB::StorageSystemKafkaConsumers; }; diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index c7563a79691..428d649047c 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -42,6 +42,9 @@ NamesAndTypesList StorageSystemKafkaConsumers::getNamesAndTypes() {"num_rebalance_revocations", std::make_shared()}, {"num_rebalance_assignments", std::make_shared()}, {"is_currently_used", std::make_shared()}, + {"rdkafka_stat", std::make_shared()}, + {"json", std::make_shared("json", false)}}; + }; return names_and_types; } @@ -77,6 +80,7 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context auto & num_rebalance_revocations = assert_cast(*res_columns[index++]); auto & num_rebalance_assigments = assert_cast(*res_columns[index++]); auto & is_currently_used = assert_cast(*res_columns[index++]); + auto & rdkafka_stat = assert_cast(*res_columns[index++]); const auto access = context->getAccess(); size_t last_assignment_num = 0; @@ -146,6 +150,16 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context num_rebalance_assigments.insert(consumer->num_rebalance_assignments.load()); is_currently_used.insert(consumer->stalled_status != KafkaConsumer::CONSUMER_STOPPED && consumer->assignment.has_value() && consumer->assignment.value().size() > 0); + + auto stat_string_ptr = storage_kafka_ptr->getRdkafkaStat(); + if (stat_string_ptr) + { + rdkafka_stat.insertData(stat_string_ptr->data(), stat_string_ptr->size()); + } + else + { + rdkafka_stat.insertData(nullptr, 0); + } } }; diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 8184e492c81..e22ac169c73 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4722,6 +4722,14 @@ is_currently_used: 0 """ ) + result_rdkafka_stat = instance.query( + """ + SELECT table, rdkafka_stat + FROM system.kafka_consumers WHERE database='test' and table IN ('kafka', 'kafka2') format Vertical; + """ + ) + logging.debug(f"result_rdkafka_stat: {result_rdkafka_stat}") + instance.query("DROP TABLE test.kafka") instance.query("DROP TABLE test.kafka2") From c821c7788a019d08626b73815147a2b2497d18ab Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 27 Jun 2023 19:36:50 +0000 Subject: [PATCH 360/777] system_kafka_consumers: json commented out --- src/Storages/System/StorageSystemKafkaConsumers.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 428d649047c..6c92b5b7d55 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -43,8 +43,7 @@ NamesAndTypesList StorageSystemKafkaConsumers::getNamesAndTypes() {"num_rebalance_assignments", std::make_shared()}, {"is_currently_used", std::make_shared()}, {"rdkafka_stat", std::make_shared()}, - {"json", std::make_shared("json", false)}}; - + // {"json", std::make_shared("json", false)}} }; return names_and_types; } From de49d7e5a089c084812729289c7ce2f07a3424ed Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 28 Jun 2023 11:57:37 +0000 Subject: [PATCH 361/777] system_kafka_consumers: minor change --- src/Storages/Kafka/StorageKafka.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 83fdd58ab42..14bbcf98b8e 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -960,7 +960,7 @@ void registerStorageKafka(StorageFactory & factory) "of getting data from Kafka, consider using a setting kafka_thread_per_consumer=1, " "and ensure you have enough threads " "in MessageBrokerSchedulePool (background_message_broker_schedule_pool_size). " - "See also https://clickhouse.com/docs/integrations/kafka/kafka-table-engine#tuning-performance", max_consumers); + "See also https://clickhouse.com/docs/en/integrations/kafka#tuning-performance", max_consumers); } else if (num_consumers < 1) { From 83566e132a69a0af070d13a1c645bfae900918e4 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 29 Jun 2023 16:03:04 +0000 Subject: [PATCH 362/777] system_kafka_consumers: all_consumers introduced, offsets work --- src/Storages/Kafka/KafkaSource.cpp | 4 + src/Storages/Kafka/StorageKafka.cpp | 8 +- src/Storages/Kafka/StorageKafka.h | 2 + .../System/StorageSystemKafkaConsumers.cpp | 123 +++++++++++------- 4 files changed, 84 insertions(+), 53 deletions(-) diff --git a/src/Storages/Kafka/KafkaSource.cpp b/src/Storages/Kafka/KafkaSource.cpp index ba242417058..9be37605258 100644 --- a/src/Storages/Kafka/KafkaSource.cpp +++ b/src/Storages/Kafka/KafkaSource.cpp @@ -52,6 +52,8 @@ KafkaSource::KafkaSource( KafkaSource::~KafkaSource() { + LOG_TRACE(&Poco::Logger::get("KafkaSource"), "dtor, pushConsumer"); + if (!consumer) return; @@ -78,6 +80,8 @@ Chunk KafkaSource::generateImpl() { if (!consumer) { + LOG_TRACE(&Poco::Logger::get("KafkaSource"), "calling popConsumer"); + auto timeout = std::chrono::milliseconds(context->getSettingsRef().kafka_max_wait_ms.totalMilliseconds()); consumer = storage.popConsumer(timeout); diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 14bbcf98b8e..10a3c822894 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -416,7 +416,9 @@ void StorageKafka::startup() { try { - pushConsumer(createConsumer(i)); + auto consumer = createConsumer(i); + pushConsumer(consumer); + all_consumers.push_back(consumer); ++num_created_consumers; } catch (const cppkafka::Exception &) @@ -642,10 +644,10 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & kafka_config) LOG_IMPL(log, client_logs_level, poco_level, "[rdk:{}] {}", facility, message); }); - kafka_config.set("statistics.interval.ms","1000000"); // 10 times per a second + kafka_config.set("statistics.interval.ms","10" /*"1000000"*/); // 10 times per a second kafka_config.set_stats_callback([this](cppkafka::KafkaHandleBase &, const std::string & stat_json_string) { - LOG_DEBUG(log, "kafka statistics {}", stat_json_string); + // LOG_DEBUG(log, "kafka statistics {}", stat_json_string); rdkafka_stat = std::make_shared(stat_json_string); }); diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 2cf1f582bd5..f379079bafe 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -28,6 +28,7 @@ class StorageSystemKafkaConsumers; struct StorageKafkaInterceptors; using KafkaConsumerPtr = std::shared_ptr; +using KafkaConsumerWeakPtr = std::weak_ptr; /** Implements a Kafka queue table engine that can be used as a persistent queue / buffer, * or as a basic building block for creating pipelines with a continuous insertion / ETL. @@ -104,6 +105,7 @@ private: size_t num_created_consumers = 0; /// number of actually created consumers. std::vector consumers; /// available consumers + std::vector all_consumers; /// busy (belong to a KafkaSource) and vacant consumers std::mutex mutex; diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 6c92b5b7d55..952ebd7d797 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -19,6 +19,8 @@ #include #include "base/types.h" +#include + namespace DB { @@ -32,6 +34,7 @@ NamesAndTypesList StorageSystemKafkaConsumers::getNamesAndTypes() {"assignments.topic", std::make_shared(std::make_shared())}, {"assignments.partition_id", std::make_shared(std::make_shared())}, {"assignments.current_offset", std::make_shared(std::make_shared())}, + {"assignments.offset_committed", std::make_shared(std::make_shared())}, {"last_exception_time", std::make_shared()}, {"last_exception", std::make_shared()}, {"last_poll_time", std::make_shared()}, @@ -68,6 +71,9 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context auto & assigments_current_offset = assert_cast(assert_cast(*res_columns[index]).getData()); auto & assigments_current_offset_offsets = assert_cast(*res_columns[index++]).getOffsets(); + auto & assigments_offset_committed = assert_cast(assert_cast(*res_columns[index]).getData()); + auto & assigments_offset_committed_offsets = assert_cast(*res_columns[index++]).getOffsets(); + auto & last_exception_time = assert_cast(*res_columns[index++]); auto & last_exception = assert_cast(*res_columns[index++]); @@ -88,78 +94,95 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context { if (!access->isGranted(AccessType::SHOW_TABLES, it->databaseName(), it->name())) { + LOG_TRACE(&Poco::Logger::get("StorageSystemKafkaConsumers"), "Not granted {}", it->name()); return; } + LOG_TRACE(&Poco::Logger::get("StorageSystemKafkaConsumers"), "granted {}", it->name()); std::string database_str = it->databaseName(); std::string table_str = it->name(); std::lock_guard lock(storage_kafka_ptr->mutex); - for (auto consumer : storage_kafka_ptr->consumers) + LOG_TRACE(&Poco::Logger::get("StorageSystemKafkaConsumers"), "{} KafkaConsumers", storage_kafka_ptr->consumers.size()); + for (auto weak_consumer : storage_kafka_ptr->all_consumers) { - auto & cpp_consumer = consumer->consumer; - - database.insertData(database_str.data(), database_str.size()); - table.insertData(table_str.data(), table_str.size()); - - std::string consumer_id_str = cpp_consumer->get_member_id(); - consumer_id.insertData(consumer_id_str.data(), consumer_id_str.size()); - - if (consumer->assignment.has_value() && consumer->assignment.value().size() > 0) + if (auto consumer = weak_consumer.lock()) { - for (const auto & assignment : consumer->assignment.value()) + LOG_TRACE(&Poco::Logger::get("StorageSystemKafkaConsumers"), "consumer got"); + auto & cpp_consumer = consumer->consumer; + + database.insertData(database_str.data(), database_str.size()); + table.insertData(table_str.data(), table_str.size()); + + std::string consumer_id_str = cpp_consumer->get_member_id(); + consumer_id.insertData(consumer_id_str.data(), consumer_id_str.size()); + + bool assignment_has_value = consumer->assignment.has_value() && consumer->assignment.value().size() > 0; + + auto cpp_assignments = cpp_consumer->get_assignment(); + auto cpp_offsets = cpp_consumer->get_offsets_position(cpp_assignments); + auto cpp_offsets_committed = cpp_consumer->get_offsets_committed(cpp_assignments); + + for (size_t num = 0; num < cpp_assignments.size(); ++num) { - const auto & topic_str = assignment.get_topic(); + const auto & topic_str = cpp_assignments[num].get_topic(); assigments_topics.insertData(topic_str.data(), topic_str.size()); - assigments_partition_id.insert(assignment.get_partition()); - assigments_current_offset.insert(assignment.get_offset()); + assigments_partition_id.insert(cpp_assignments[num].get_partition()); + assigments_current_offset.insert(cpp_offsets[num].get_offset()); + assigments_offset_committed.insert(cpp_offsets_committed[num].get_offset()); } - last_assignment_num += consumer->assignment.value().size(); - - } - else - { - std::string fake_assigments_topic = "no assigned topic"; - assigments_topics.insertData(fake_assigments_topic.data(), fake_assigments_topic.size()); - - assigments_partition_id.insert(0); - assigments_current_offset.insert(0); - - last_assignment_num += 1; - } - assigments_topics_offsets.push_back(last_assignment_num); - assigments_partition_id_offsets.push_back(last_assignment_num); - assigments_current_offset_offsets.push_back(last_assignment_num); - - auto exception_info = consumer->getExceptionInfo(); - last_exception.insertData(exception_info.first.data(), exception_info.first.size()); - last_exception_time.insert(exception_info.second); + last_assignment_num += cpp_assignments.size(); - last_poll_time.insert(consumer->last_poll_timestamp_usec.load()); - num_messages_read.insert(consumer->num_messages_read.load()); - last_commit_time.insert(consumer->last_commit_timestamp_usec.load()); - num_commits.insert(consumer->num_commits.load()); - last_rebalance_time.insert(consumer->last_rebalance_timestamp_usec.load()); + // } + // else + // { + // std::string fake_assigments_topic = "no assigned topic"; + // assigments_topics.insertData(fake_assigments_topic.data(), fake_assigments_topic.size()); - num_rebalance_revocations.insert(consumer->num_rebalance_revocations.load()); - num_rebalance_assigments.insert(consumer->num_rebalance_assignments.load()); + // assigments_partition_id.insert(0); + // assigments_current_offset.insert(0); - is_currently_used.insert(consumer->stalled_status != KafkaConsumer::CONSUMER_STOPPED && consumer->assignment.has_value() && consumer->assignment.value().size() > 0); + // last_assignment_num += 1; + // } + assigments_topics_offsets.push_back(last_assignment_num); + assigments_partition_id_offsets.push_back(last_assignment_num); + assigments_current_offset_offsets.push_back(last_assignment_num); + assigments_offset_committed_offsets.push_back(last_assignment_num); - auto stat_string_ptr = storage_kafka_ptr->getRdkafkaStat(); - if (stat_string_ptr) - { - rdkafka_stat.insertData(stat_string_ptr->data(), stat_string_ptr->size()); - } - else - { - rdkafka_stat.insertData(nullptr, 0); + auto exception_info = consumer->getExceptionInfo(); + + + last_exception.insertData(exception_info.first.data(), exception_info.first.size()); + last_exception_time.insert(exception_info.second); + + last_poll_time.insert(consumer->last_poll_timestamp_usec.load()); + num_messages_read.insert(consumer->num_messages_read.load()); + last_commit_time.insert(consumer->last_commit_timestamp_usec.load()); + num_commits.insert(consumer->num_commits.load()); + last_rebalance_time.insert(consumer->last_rebalance_timestamp_usec.load()); + + num_rebalance_revocations.insert(consumer->num_rebalance_revocations.load()); + num_rebalance_assigments.insert(consumer->num_rebalance_assignments.load()); + + is_currently_used.insert(consumer->stalled_status != KafkaConsumer::CONSUMER_STOPPED && assignment_has_value); + + auto stat_string_ptr = storage_kafka_ptr->getRdkafkaStat(); + if (stat_string_ptr) + { + rdkafka_stat.insertData(stat_string_ptr->data(), stat_string_ptr->size()); + } + else + { + std::string no_rdkafka_stat = "librdkafka stat is not available"; + rdkafka_stat.insertData(no_rdkafka_stat.data(), no_rdkafka_stat.size()); + } } } + LOG_TRACE(&Poco::Logger::get("StorageSystemKafkaConsumers"), "bottom of add_row"); }; From b71c2bedfc168df2b3f21b29d3e7e206ca012a7c Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sat, 1 Jul 2023 20:55:24 +0000 Subject: [PATCH 363/777] system_kafka_consumers: rebalance test is ok, statistics_interval_ms --- src/Storages/Kafka/KafkaConsumer.h | 5 ++ src/Storages/Kafka/StorageKafka.cpp | 19 +++-- .../System/StorageSystemKafkaConsumers.cpp | 19 +---- tests/integration/test_storage_kafka/test.py | 75 ++++++++++++++++--- 4 files changed, 85 insertions(+), 33 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index e4cd332a1f5..4da0784539f 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -73,6 +73,10 @@ public: String currentPayload() const { return current[-1].get_payload(); } void setExceptionInfo(const String & text); std::pair getExceptionInfo() const; + void inUse() { in_use = true; } + void notInUse() { in_use = false; } + + private: @@ -121,6 +125,7 @@ private: std::atomic last_rebalance_timestamp_usec = 0; std::atomic num_rebalance_assignments = 0; std::atomic num_rebalance_revocations = 0; + std::atomic in_use = 0; void drain(); void cleanUnprocessed(); diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 10a3c822894..b14e5f4f4ad 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -458,6 +458,7 @@ void StorageKafka::shutdown() void StorageKafka::pushConsumer(KafkaConsumerPtr consumer) { std::lock_guard lock(mutex); + consumer->notInUse(); consumers.push_back(consumer); semaphore.set(); CurrentMetrics::sub(CurrentMetrics::KafkaConsumersInUse, 1); @@ -486,6 +487,7 @@ KafkaConsumerPtr StorageKafka::popConsumer(std::chrono::milliseconds timeout) auto consumer = consumers.back(); consumers.pop_back(); CurrentMetrics::add(CurrentMetrics::KafkaConsumersInUse, 1); + consumer->inUse(); return consumer; } @@ -644,13 +646,18 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & kafka_config) LOG_IMPL(log, client_logs_level, poco_level, "[rdk:{}] {}", facility, message); }); - kafka_config.set("statistics.interval.ms","10" /*"1000000"*/); // 10 times per a second - kafka_config.set_stats_callback([this](cppkafka::KafkaHandleBase &, const std::string & stat_json_string) + if (!kafka_config.has_property("statistics.interval.ms")) { - // LOG_DEBUG(log, "kafka statistics {}", stat_json_string); - rdkafka_stat = std::make_shared(stat_json_string); - }); - + kafka_config.set("statistics.interval.ms", "10"); // every 10 milliseconds + } + if (kafka_config.get("statistics.interval.ms") != "0") + { + kafka_config.set_stats_callback([this](cppkafka::KafkaHandleBase &, const std::string & stat_json_string) + { + // LOG_DEBUG(log, "kafka statistics {}", stat_json_string); + rdkafka_stat = std::make_shared(stat_json_string); + }); + } // Configure interceptor to change thread name // diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 952ebd7d797..b7f3faf9e5b 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -118,7 +118,7 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context std::string consumer_id_str = cpp_consumer->get_member_id(); consumer_id.insertData(consumer_id_str.data(), consumer_id_str.size()); - bool assignment_has_value = consumer->assignment.has_value() && consumer->assignment.value().size() > 0; + // bool assignment_has_value = consumer->assignment.has_value() && consumer->assignment.value().size() > 0; auto cpp_assignments = cpp_consumer->get_assignment(); auto cpp_offsets = cpp_consumer->get_offsets_position(cpp_assignments); @@ -137,17 +137,6 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context last_assignment_num += cpp_assignments.size(); - // } - // else - // { - // std::string fake_assigments_topic = "no assigned topic"; - // assigments_topics.insertData(fake_assigments_topic.data(), fake_assigments_topic.size()); - - // assigments_partition_id.insert(0); - // assigments_current_offset.insert(0); - - // last_assignment_num += 1; - // } assigments_topics_offsets.push_back(last_assignment_num); assigments_partition_id_offsets.push_back(last_assignment_num); assigments_current_offset_offsets.push_back(last_assignment_num); @@ -168,7 +157,7 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context num_rebalance_revocations.insert(consumer->num_rebalance_revocations.load()); num_rebalance_assigments.insert(consumer->num_rebalance_assignments.load()); - is_currently_used.insert(consumer->stalled_status != KafkaConsumer::CONSUMER_STOPPED && assignment_has_value); + is_currently_used.insert(consumer->in_use.load()); auto stat_string_ptr = storage_kafka_ptr->getRdkafkaStat(); if (stat_string_ptr) @@ -177,8 +166,8 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context } else { - std::string no_rdkafka_stat = "librdkafka stat is not available"; - rdkafka_stat.insertData(no_rdkafka_stat.data(), no_rdkafka_stat.size()); + const std::string empty_stat = "{}"; + rdkafka_stat.insertData(empty_stat.data(), empty_stat.size()); } } } diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index e22ac169c73..8105ccb2be7 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4628,8 +4628,16 @@ def test_system_kafka_consumers_rebalance(kafka_cluster): bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) + producer = KafkaProducer( + bootstrap_servers="localhost:{}".format(cluster.kafka_port), + value_serializer=producer_serializer, + key_serializer=producer_serializer, + ) + topic = "system_kafka_cons2" - kafka_create_topic(admin_client, topic) + kafka_create_topic(admin_client, topic, num_partitions=2) + + instance.query( f""" @@ -4651,9 +4659,39 @@ def test_system_kafka_consumers_rebalance(kafka_cluster): kafka_commit_on_select = 1, kafka_group_name = '{topic}', kafka_format = 'JSONEachRow'; + DROP TABLE IF EXISTS test.kafka_persistent; """ + # CREATE TABLE test.kafka_persistent + # ( + # `key` UInt64, + # `value` UInt64 + # ) + # ENGINE = MergeTree + # ORDER BY key; + + # DROP VIEW IF EXISTS test.kafka_mv1; + # CREATE MATERIALIZED VIEW test.kafka_mv1 TO test.kafka_persistent AS + # SELECT * + # FROM test.kafka; + # DROP VIEW IF EXISTS test.kafka_mv2; + # CREATE MATERIALIZED VIEW test.kafka_mv2 TO test.kafka_persistent AS + # SELECT * + # FROM test.kafka2; + # """ ) + producer.send( + topic=topic, + value=json.dumps({"key": 1, "value": 1}), + partition=0 + ) + producer.send( + topic=topic, + value=json.dumps({"key": 11, "value": 11}), + partition=1 + ) + time.sleep(3) + # first consumer subscribe the topic, try to poll some data, and go to rest instance.query("SELECT * FROM test.kafka") @@ -4661,11 +4699,24 @@ def test_system_kafka_consumers_rebalance(kafka_cluster): # consumer, try to poll some data instance.query("SELECT * FROM test.kafka2") + producer.send( + topic=topic, + value=json.dumps({"key": 1, "value": 1}), + partition=0 + ) + producer.send( + topic=topic, + value=json.dumps({"key": 10, "value": 10}), + partition=1 + ) + time.sleep(3) + instance.query("SELECT * FROM test.kafka") instance.query("SELECT * FROM test.kafka2") instance.query("SELECT * FROM test.kafka") instance.query("SELECT * FROM test.kafka2") + result_system_kafka_consumers = instance.query( """ create or replace function stable_timestamp as @@ -4689,32 +4740,32 @@ table: kafka length(consumer_id): 67 assignments.topic: ['system_kafka_cons2'] assignments.partition_id: [0] -assignments.current_offset: [-1001] +assignments.current_offset: [2] last_exception_time_: never last_exception_: no exception last_poll_time_: now -num_messages_read: 0 -last_commit_time_: never -num_commits: 0 +num_messages_read: 4 +last_commit_time_: now +num_commits: 2 last_rebalance_time_: now num_rebalance_revocations: 1 num_rebalance_assignments: 2 -is_currently_used: 1 +is_currently_used: 0 Row 2: ────── database: test table: kafka2 length(consumer_id): 68 -assignments.topic: ['no assigned topic'] -assignments.partition_id: [0] -assignments.current_offset: [0] +assignments.topic: ['system_kafka_cons2'] +assignments.partition_id: [1] +assignments.current_offset: [2] last_exception_time_: never last_exception_: no exception last_poll_time_: now -num_messages_read: 0 -last_commit_time_: never -num_commits: 0 +num_messages_read: 1 +last_commit_time_: now +num_commits: 1 last_rebalance_time_: never num_rebalance_revocations: 0 num_rebalance_assignments: 1 From 1f03e320eeb7fc3e6d29278f45658dec38bb5291 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sat, 1 Jul 2023 21:34:14 +0000 Subject: [PATCH 364/777] system_kafka_consumers: test fix + black formatter --- tests/integration/test_storage_kafka/test.py | 66 ++++++-------------- 1 file changed, 20 insertions(+), 46 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 8105ccb2be7..fcc4315ead4 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4577,8 +4577,6 @@ def test_system_kafka_consumers(kafka_cluster): """ ) - # result_system_kafka_consumers = instance.query("SELECT * FROM system.kafka_consumers format Vertical") - result = instance.query("SELECT * FROM test.kafka ORDER BY a;") result_system_kafka_consumers = instance.query( @@ -4605,7 +4603,7 @@ table: kafka length(consumer_id): 67 assignments.topic: ['system_kafka_cons'] assignments.partition_id: [0] -assignments.current_offset: [-1001] +assignments.current_offset: [4] last_exception_time_: never last_exception_: no exception last_poll_time_: now @@ -4615,7 +4613,7 @@ num_commits: 1 last_rebalance_time_: never num_rebalance_revocations: 0 num_rebalance_assignments: 1 -is_currently_used: 1 +is_currently_used: 0 """ ) @@ -4637,8 +4635,6 @@ def test_system_kafka_consumers_rebalance(kafka_cluster): topic = "system_kafka_cons2" kafka_create_topic(admin_client, topic, num_partitions=2) - - instance.query( f""" DROP TABLE IF EXISTS test.kafka; @@ -4661,35 +4657,10 @@ def test_system_kafka_consumers_rebalance(kafka_cluster): kafka_format = 'JSONEachRow'; DROP TABLE IF EXISTS test.kafka_persistent; """ - # CREATE TABLE test.kafka_persistent - # ( - # `key` UInt64, - # `value` UInt64 - # ) - # ENGINE = MergeTree - # ORDER BY key; - - # DROP VIEW IF EXISTS test.kafka_mv1; - # CREATE MATERIALIZED VIEW test.kafka_mv1 TO test.kafka_persistent AS - # SELECT * - # FROM test.kafka; - # DROP VIEW IF EXISTS test.kafka_mv2; - # CREATE MATERIALIZED VIEW test.kafka_mv2 TO test.kafka_persistent AS - # SELECT * - # FROM test.kafka2; - # """ ) - producer.send( - topic=topic, - value=json.dumps({"key": 1, "value": 1}), - partition=0 - ) - producer.send( - topic=topic, - value=json.dumps({"key": 11, "value": 11}), - partition=1 - ) + producer.send(topic=topic, value=json.dumps({"key": 1, "value": 1}), partition=0) + producer.send(topic=topic, value=json.dumps({"key": 11, "value": 11}), partition=1) time.sleep(3) # first consumer subscribe the topic, try to poll some data, and go to rest @@ -4699,16 +4670,8 @@ def test_system_kafka_consumers_rebalance(kafka_cluster): # consumer, try to poll some data instance.query("SELECT * FROM test.kafka2") - producer.send( - topic=topic, - value=json.dumps({"key": 1, "value": 1}), - partition=0 - ) - producer.send( - topic=topic, - value=json.dumps({"key": 10, "value": 10}), - partition=1 - ) + producer.send(topic=topic, value=json.dumps({"key": 1, "value": 1}), partition=0) + producer.send(topic=topic, value=json.dumps({"key": 10, "value": 10}), partition=1) time.sleep(3) instance.query("SELECT * FROM test.kafka") @@ -4716,7 +4679,6 @@ def test_system_kafka_consumers_rebalance(kafka_cluster): instance.query("SELECT * FROM test.kafka") instance.query("SELECT * FROM test.kafka2") - result_system_kafka_consumers = instance.query( """ create or replace function stable_timestamp as @@ -4775,11 +4737,23 @@ is_currently_used: 0 result_rdkafka_stat = instance.query( """ - SELECT table, rdkafka_stat + SELECT table, JSONExtractString(rdkafka_stat, 'type') FROM system.kafka_consumers WHERE database='test' and table IN ('kafka', 'kafka2') format Vertical; """ ) - logging.debug(f"result_rdkafka_stat: {result_rdkafka_stat}") + assert ( + result_rdkafka_stat + == """Row 1: +────── +table: kafka +JSONExtractString(rdkafka_stat, 'type'): consumer + +Row 2: +────── +table: kafka2 +JSONExtractString(rdkafka_stat, 'type'): consumer +""" + ) instance.query("DROP TABLE test.kafka") instance.query("DROP TABLE test.kafka2") From 3f205f4997b0cddfc6ae870033f51420981c0f65 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sat, 1 Jul 2023 21:51:28 +0000 Subject: [PATCH 365/777] system_kafka_consumers: tiny doc changes --- docs/en/engines/table-engines/integrations/kafka.md | 1 + docs/en/operations/system-tables/kafka_consumers.md | 1 + src/Storages/System/StorageSystemKafkaConsumers.cpp | 1 - 3 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index b81d5624c1a..cb46b449136 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -260,3 +260,4 @@ The number of rows in one Kafka message depends on whether the format is row-bas - [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns) - [background_message_broker_schedule_pool_size](../../../operations/server-configuration-parameters/settings.md#background_message_broker_schedule_pool_size) +- [system.kafka_consumers](../../../operations/system-tables/kafka-consumers.md) diff --git a/docs/en/operations/system-tables/kafka_consumers.md b/docs/en/operations/system-tables/kafka_consumers.md index 2baca8b6db4..1a004ee30c4 100644 --- a/docs/en/operations/system-tables/kafka_consumers.md +++ b/docs/en/operations/system-tables/kafka_consumers.md @@ -24,6 +24,7 @@ Columns: - `num_rebalance_revocations`, (UInt64) - number of times the consumer was revoked its partitions - `num_rebalance_assignments`, (UInt64) - number of times the consumer was assigned to Kafka cluster - `is_currently_used`, (UInt8) - consumer is in use +- `rdkafka_stat` (String) - library internal statistic. See https://github.com/ClickHouse/librdkafka/blob/master/STATISTICS.md . Set `statistics_interval_ms` to disable. Example: diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index b7f3faf9e5b..a69b535d3ab 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -46,7 +46,6 @@ NamesAndTypesList StorageSystemKafkaConsumers::getNamesAndTypes() {"num_rebalance_assignments", std::make_shared()}, {"is_currently_used", std::make_shared()}, {"rdkafka_stat", std::make_shared()}, - // {"json", std::make_shared("json", false)}} }; return names_and_types; } From bba4dcacc73b66d42bf2430cf0fa2fe2f277f270 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sat, 1 Jul 2023 21:58:54 +0000 Subject: [PATCH 366/777] system_kafka_consumers: small code cleanup --- src/Storages/Kafka/KafkaConsumer.cpp | 2 -- src/Storages/System/StorageSystemKafkaConsumers.cpp | 12 ------------ 2 files changed, 14 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index 12feab92bde..dc4cb326306 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -76,7 +76,6 @@ KafkaConsumer::KafkaConsumer( { LOG_TRACE(log, "Topics/partitions assigned: {}", topic_partitions); CurrentMetrics::add(CurrentMetrics::KafkaConsumersWithAssignment, 1); - } assignment = topic_partitions; @@ -390,7 +389,6 @@ ReadBufferPtr KafkaConsumer::consume() if (intermediate_commit) commit(); - while (true) { stalled_status = NO_MESSAGES_RETURNED; diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index a69b535d3ab..57ad45ebf39 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -19,9 +19,6 @@ #include #include "base/types.h" -#include - - namespace DB { @@ -73,7 +70,6 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context auto & assigments_offset_committed = assert_cast(assert_cast(*res_columns[index]).getData()); auto & assigments_offset_committed_offsets = assert_cast(*res_columns[index++]).getOffsets(); - auto & last_exception_time = assert_cast(*res_columns[index++]); auto & last_exception = assert_cast(*res_columns[index++]); auto & last_poll_time = assert_cast(*res_columns[index++]); @@ -93,22 +89,18 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context { if (!access->isGranted(AccessType::SHOW_TABLES, it->databaseName(), it->name())) { - LOG_TRACE(&Poco::Logger::get("StorageSystemKafkaConsumers"), "Not granted {}", it->name()); return; } - LOG_TRACE(&Poco::Logger::get("StorageSystemKafkaConsumers"), "granted {}", it->name()); std::string database_str = it->databaseName(); std::string table_str = it->name(); std::lock_guard lock(storage_kafka_ptr->mutex); - LOG_TRACE(&Poco::Logger::get("StorageSystemKafkaConsumers"), "{} KafkaConsumers", storage_kafka_ptr->consumers.size()); for (auto weak_consumer : storage_kafka_ptr->all_consumers) { if (auto consumer = weak_consumer.lock()) { - LOG_TRACE(&Poco::Logger::get("StorageSystemKafkaConsumers"), "consumer got"); auto & cpp_consumer = consumer->consumer; database.insertData(database_str.data(), database_str.size()); @@ -117,8 +109,6 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context std::string consumer_id_str = cpp_consumer->get_member_id(); consumer_id.insertData(consumer_id_str.data(), consumer_id_str.size()); - // bool assignment_has_value = consumer->assignment.has_value() && consumer->assignment.value().size() > 0; - auto cpp_assignments = cpp_consumer->get_assignment(); auto cpp_offsets = cpp_consumer->get_offsets_position(cpp_assignments); auto cpp_offsets_committed = cpp_consumer->get_offsets_committed(cpp_assignments); @@ -170,8 +160,6 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context } } } - LOG_TRACE(&Poco::Logger::get("StorageSystemKafkaConsumers"), "bottom of add_row"); - }; const bool show_tables_granted = access->isGranted(AccessType::SHOW_TABLES); From dd6727be3e7e8b13601ebd008447c1c1af1b7c32 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sun, 2 Jul 2023 21:20:20 +0000 Subject: [PATCH 367/777] system_kafka_consumers: style check fixes --- docs/en/engines/table-engines/integrations/kafka.md | 3 ++- src/Storages/Kafka/KafkaConsumer.h | 4 ---- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index cb46b449136..3ced66cefdf 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -173,6 +173,7 @@ Similar to GraphiteMergeTree, the Kafka engine supports extended configuration u cgrp smallest + 10 @@ -260,4 +261,4 @@ The number of rows in one Kafka message depends on whether the format is row-bas - [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns) - [background_message_broker_schedule_pool_size](../../../operations/server-configuration-parameters/settings.md#background_message_broker_schedule_pool_size) -- [system.kafka_consumers](../../../operations/system-tables/kafka-consumers.md) +- [system.kafka_consumers](../../../operations/system-tables/kafka_consumers.md) diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index 4da0784539f..808007f43ea 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -76,9 +76,6 @@ public: void inUse() { in_use = true; } void notInUse() { in_use = false; } - - - private: using Messages = std::vector; CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaConsumers}; @@ -135,7 +132,6 @@ private: ReadBufferPtr getNextMessage(); friend class DB::StorageSystemKafkaConsumers; - }; } From a86548a733e084d2ae0fba093959a25949b938bd Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 3 Jul 2023 21:33:52 +0000 Subject: [PATCH 368/777] system_kafka_consumers: statistics.interval.ms default, tiny cleanup --- src/Storages/Kafka/KafkaSource.cpp | 4 ---- src/Storages/Kafka/StorageKafka.cpp | 6 +++--- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Storages/Kafka/KafkaSource.cpp b/src/Storages/Kafka/KafkaSource.cpp index 9be37605258..ba242417058 100644 --- a/src/Storages/Kafka/KafkaSource.cpp +++ b/src/Storages/Kafka/KafkaSource.cpp @@ -52,8 +52,6 @@ KafkaSource::KafkaSource( KafkaSource::~KafkaSource() { - LOG_TRACE(&Poco::Logger::get("KafkaSource"), "dtor, pushConsumer"); - if (!consumer) return; @@ -80,8 +78,6 @@ Chunk KafkaSource::generateImpl() { if (!consumer) { - LOG_TRACE(&Poco::Logger::get("KafkaSource"), "calling popConsumer"); - auto timeout = std::chrono::milliseconds(context->getSettingsRef().kafka_max_wait_ms.totalMilliseconds()); consumer = storage.popConsumer(timeout); diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index b14e5f4f4ad..59abbfbbcd1 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -646,15 +646,15 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & kafka_config) LOG_IMPL(log, client_logs_level, poco_level, "[rdk:{}] {}", facility, message); }); - if (!kafka_config.has_property("statistics.interval.ms")) + if (!config.has(config_prefix + "." + "statistics_interval_ms")) { - kafka_config.set("statistics.interval.ms", "10"); // every 10 milliseconds + kafka_config.set("statistics.interval.ms", "600"); // every 600 milliseconds } + if (kafka_config.get("statistics.interval.ms") != "0") { kafka_config.set_stats_callback([this](cppkafka::KafkaHandleBase &, const std::string & stat_json_string) { - // LOG_DEBUG(log, "kafka statistics {}", stat_json_string); rdkafka_stat = std::make_shared(stat_json_string); }); } From 803a80b09852754158f541c4d88e9f3e9759b17a Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 10 Jul 2023 08:16:39 +0000 Subject: [PATCH 369/777] system_kafka_consumers: refactored with getStat, test passed --- src/Storages/Kafka/KafkaConsumer.cpp | 44 ++++++++++++++--- src/Storages/Kafka/KafkaConsumer.h | 35 ++++++++++++-- .../System/StorageSystemKafkaConsumers.cpp | 48 ++++++++----------- 3 files changed, 88 insertions(+), 39 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index dc4cb326306..12f36f4eb76 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -537,16 +537,46 @@ void KafkaConsumer::storeLastReadMessageOffset() void KafkaConsumer::setExceptionInfo(const String & text) { - std::lock_guard lock(exception_mutex); - - last_exception_text = text; last_exception_timestamp_usec = static_cast(Poco::Timestamp().epochTime()); -} -std::pair KafkaConsumer::getExceptionInfo() const -{ std::lock_guard lock(exception_mutex); - return std::make_pair(last_exception_text, last_exception_timestamp_usec); + last_exception_text = text; +} + + +KafkaConsumer::Stat KafkaConsumer::getStat() +{ + KafkaConsumer::Stat::Assignments assignments; + auto cpp_assignments = consumer->get_assignment(); + auto cpp_offsets = consumer->get_offsets_position(cpp_assignments); + auto cpp_offsets_committed = consumer->get_offsets_committed(cpp_assignments); + + for (size_t num = 0; num < cpp_assignments.size(); ++num) + { + assignments.push_back({ + cpp_assignments[num].get_topic(), + cpp_assignments[num].get_partition(), + cpp_offsets[num].get_offset(), + cpp_offsets_committed[num].get_offset() + }); + } + + return { + .consumer_id = consumer->get_member_id(), + .assignments = std::move(assignments), + .last_exception = [&](){std::lock_guard lock(exception_mutex); + return last_exception_text;}(), + .last_exception_time = last_exception_timestamp_usec.load(), + .last_poll_time = last_poll_timestamp_usec.load(), + .num_messages_read = num_messages_read.load(), + + .last_commit_timestamp_usec = last_commit_timestamp_usec.load(), + .last_rebalance_timestamp_usec = last_rebalance_timestamp_usec.load(), + .num_commits = num_commits.load(), + .num_rebalance_assignments = num_rebalance_assignments.load(), + .num_rebalance_revocations = num_rebalance_revocations.load(), + .in_use = in_use.load() + }; } } diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index 808007f43ea..c0ce59dc343 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -26,6 +26,31 @@ using ConsumerPtr = std::shared_ptr; class KafkaConsumer { +public: + struct Stat // for system.kafka_consumers + { + struct Assignment + { + String topic_str; + Int32 partition_id; + Int64 current_offset; + Int64 offset_committed; + }; + using Assignments = std::vector; + String consumer_id; // cpp_consumer->get_member_id(); + Assignments assignments; + String last_exception; + Int64 last_exception_time; + UInt64 last_poll_time; + UInt64 num_messages_read; + UInt64 last_commit_timestamp_usec; + UInt64 last_rebalance_timestamp_usec; + UInt64 num_commits; + UInt64 num_rebalance_assignments; + UInt64 num_rebalance_revocations; + bool in_use; + }; + public: KafkaConsumer( ConsumerPtr consumer_, @@ -72,10 +97,11 @@ public: const auto & currentHeaderList() const { return current[-1].get_header_list(); } String currentPayload() const { return current[-1].get_payload(); } void setExceptionInfo(const String & text); - std::pair getExceptionInfo() const; void inUse() { in_use = true; } void notInUse() { in_use = false; } + Stat getStat(); + private: using Messages = std::vector; CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaConsumers}; @@ -111,9 +137,10 @@ private: std::optional assignment; const Names topics; - String last_exception_text; - Int64 last_exception_timestamp_usec = 0; mutable std::mutex exception_mutex; + String last_exception_text; + + std::atomic last_exception_timestamp_usec = 0; std::atomic last_poll_timestamp_usec = 0; std::atomic num_messages_read = 0; @@ -130,8 +157,6 @@ private: /// Return number of messages with an error. size_t filterMessageErrors(); ReadBufferPtr getNextMessage(); - - friend class DB::StorageSystemKafkaConsumers; }; } diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 57ad45ebf39..a80f1280b1d 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -101,52 +101,46 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context { if (auto consumer = weak_consumer.lock()) { - auto & cpp_consumer = consumer->consumer; + auto consumer_stat = consumer->getStat(); database.insertData(database_str.data(), database_str.size()); table.insertData(table_str.data(), table_str.size()); - std::string consumer_id_str = cpp_consumer->get_member_id(); - consumer_id.insertData(consumer_id_str.data(), consumer_id_str.size()); + consumer_id.insertData(consumer_stat.consumer_id.data(), consumer_stat.consumer_id.size()); - auto cpp_assignments = cpp_consumer->get_assignment(); - auto cpp_offsets = cpp_consumer->get_offsets_position(cpp_assignments); - auto cpp_offsets_committed = cpp_consumer->get_offsets_committed(cpp_assignments); + const auto num_assignnemts = consumer_stat.assignments.size(); - for (size_t num = 0; num < cpp_assignments.size(); ++num) + for (size_t num = 0; num < num_assignnemts; ++num) { - const auto & topic_str = cpp_assignments[num].get_topic(); - assigments_topics.insertData(topic_str.data(), topic_str.size()); + const auto & assign = consumer_stat.assignments[num]; - assigments_partition_id.insert(cpp_assignments[num].get_partition()); - assigments_current_offset.insert(cpp_offsets[num].get_offset()); - assigments_offset_committed.insert(cpp_offsets_committed[num].get_offset()); + assigments_topics.insertData(assign.topic_str.data(), assign.topic_str.size()); + + assigments_partition_id.insert(assign.partition_id); + assigments_current_offset.insert(assign.current_offset); + assigments_offset_committed.insert(assign.offset_committed); } - - - last_assignment_num += cpp_assignments.size(); + last_assignment_num += num_assignnemts; assigments_topics_offsets.push_back(last_assignment_num); assigments_partition_id_offsets.push_back(last_assignment_num); assigments_current_offset_offsets.push_back(last_assignment_num); assigments_offset_committed_offsets.push_back(last_assignment_num); - auto exception_info = consumer->getExceptionInfo(); + last_exception.insertData(consumer_stat.last_exception.data(), consumer_stat.last_exception.size()); + last_exception_time.insert(consumer_stat.last_exception_time); - last_exception.insertData(exception_info.first.data(), exception_info.first.size()); - last_exception_time.insert(exception_info.second); + last_poll_time.insert(consumer_stat.last_poll_time); + num_messages_read.insert(consumer_stat.num_messages_read); + last_commit_time.insert(consumer_stat.last_commit_timestamp_usec); + num_commits.insert(consumer_stat.num_commits); + last_rebalance_time.insert(consumer_stat.last_rebalance_timestamp_usec); - last_poll_time.insert(consumer->last_poll_timestamp_usec.load()); - num_messages_read.insert(consumer->num_messages_read.load()); - last_commit_time.insert(consumer->last_commit_timestamp_usec.load()); - num_commits.insert(consumer->num_commits.load()); - last_rebalance_time.insert(consumer->last_rebalance_timestamp_usec.load()); + num_rebalance_revocations.insert(consumer_stat.num_rebalance_revocations); + num_rebalance_assigments.insert(consumer_stat.num_rebalance_assignments); - num_rebalance_revocations.insert(consumer->num_rebalance_revocations.load()); - num_rebalance_assigments.insert(consumer->num_rebalance_assignments.load()); - - is_currently_used.insert(consumer->in_use.load()); + is_currently_used.insert(consumer_stat.in_use); auto stat_string_ptr = storage_kafka_ptr->getRdkafkaStat(); if (stat_string_ptr) From 46be5e578201ee28463732f25b5a26432728626a Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 10 Jul 2023 14:03:49 +0000 Subject: [PATCH 370/777] system_kafka_consumers: cleanup, test should be more stable --- .../table-engines/integrations/kafka.md | 2 +- .../system-tables/kafka_consumers.md | 3 +- src/Storages/Kafka/KafkaConsumer.h | 12 ++++--- src/Storages/Kafka/StorageKafka.cpp | 2 +- .../test_storage_kafka/configs/kafka.xml | 4 +++ tests/integration/test_storage_kafka/test.py | 31 ++++++++++--------- 6 files changed, 32 insertions(+), 22 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index 3ced66cefdf..e4d3ac762ed 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -173,7 +173,7 @@ Similar to GraphiteMergeTree, the Kafka engine supports extended configuration u cgrp smallest - 10 + 600 diff --git a/docs/en/operations/system-tables/kafka_consumers.md b/docs/en/operations/system-tables/kafka_consumers.md index 1a004ee30c4..1a64fdd5353 100644 --- a/docs/en/operations/system-tables/kafka_consumers.md +++ b/docs/en/operations/system-tables/kafka_consumers.md @@ -14,6 +14,7 @@ Columns: - `assignments.topic` (Array(String)) - Kafka topic. - `assignments.partition_id` (Array(Int32)) - Kafka partition id. Note, that only one consumer can be assigned to a partition. - `assignments.current_offset` (Array(Int64)) - current offset. +- `assignments.offset_committed` (Array(Int64)) - committed offset. - `last_exception_time`, (DateTime) - timestamp when the most recent exception was generated. - `last_exception`, (String) - text of the most recent exception. - `last_poll_time`, (DateTime) - timestamp of the most recent poll. @@ -24,7 +25,7 @@ Columns: - `num_rebalance_revocations`, (UInt64) - number of times the consumer was revoked its partitions - `num_rebalance_assignments`, (UInt64) - number of times the consumer was assigned to Kafka cluster - `is_currently_used`, (UInt8) - consumer is in use -- `rdkafka_stat` (String) - library internal statistic. See https://github.com/ClickHouse/librdkafka/blob/master/STATISTICS.md . Set `statistics_interval_ms` to disable. +- `rdkafka_stat` (String) - library internal statistic. See https://github.com/ClickHouse/librdkafka/blob/master/STATISTICS.md . Set `statistics_interval_ms` to 0 disable, default is 3000 (once in three seconds). Example: diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index c0ce59dc343..2f860d4769b 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -27,7 +27,7 @@ using ConsumerPtr = std::shared_ptr; class KafkaConsumer { public: - struct Stat // for system.kafka_consumers + struct Stat // system.kafka_consumers data { struct Assignment { @@ -37,10 +37,11 @@ public: Int64 offset_committed; }; using Assignments = std::vector; - String consumer_id; // cpp_consumer->get_member_id(); + + String consumer_id; Assignments assignments; String last_exception; - Int64 last_exception_time; + UInt64 last_exception_time; UInt64 last_poll_time; UInt64 num_messages_read; UInt64 last_commit_timestamp_usec; @@ -100,6 +101,7 @@ public: void inUse() { in_use = true; } void notInUse() { in_use = false; } + // For system.kafka_consumers Stat getStat(); private: @@ -137,11 +139,11 @@ private: std::optional assignment; const Names topics; + /// system.kafka_consumers data is retrieved asynchronously, mutable std::mutex exception_mutex; String last_exception_text; - std::atomic last_exception_timestamp_usec = 0; - + std::atomic last_exception_timestamp_usec = 0; std::atomic last_poll_timestamp_usec = 0; std::atomic num_messages_read = 0; std::atomic last_commit_timestamp_usec = 0; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 59abbfbbcd1..05548dc8696 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -648,7 +648,7 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & kafka_config) if (!config.has(config_prefix + "." + "statistics_interval_ms")) { - kafka_config.set("statistics.interval.ms", "600"); // every 600 milliseconds + kafka_config.set("statistics.interval.ms", "3000"); // every 3 seconds by default. set to 0 to disable. } if (kafka_config.get("statistics.interval.ms") != "0") diff --git a/tests/integration/test_storage_kafka/configs/kafka.xml b/tests/integration/test_storage_kafka/configs/kafka.xml index 062c98a2ac7..c6075aff715 100644 --- a/tests/integration/test_storage_kafka/configs/kafka.xml +++ b/tests/integration/test_storage_kafka/configs/kafka.xml @@ -10,6 +10,10 @@ --> cgrp,consumer,topic,protocol + + + 600 + consumer_hang diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index fcc4315ead4..a88cc490a1a 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4582,7 +4582,7 @@ def test_system_kafka_consumers(kafka_cluster): result_system_kafka_consumers = instance.query( """ create or replace function stable_timestamp as - (d)->multiIf(d==toDateTime('1970-01-01 00:00:00'), 'never', abs(dateDiff('second', d, now())) < 20, 'now', toString(d)); + (d)->multiIf(d==toDateTime('1970-01-01 00:00:00'), 'never', abs(dateDiff('second', d, now())) < 30, 'now', toString(d)); SELECT database, table, length(consumer_id), assignments.topic, assignments.partition_id, assignments.current_offset, stable_timestamp(last_exception_time) as last_exception_time_, @@ -4620,7 +4620,7 @@ is_currently_used: 0 kafka_delete_topic(admin_client, topic) -def test_system_kafka_consumers_rebalance(kafka_cluster): +def test_system_kafka_consumers_rebalance(kafka_cluster, max_retries = 15): # based on test_kafka_consumer_hang2 admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) @@ -4682,7 +4682,7 @@ def test_system_kafka_consumers_rebalance(kafka_cluster): result_system_kafka_consumers = instance.query( """ create or replace function stable_timestamp as - (d)->multiIf(d==toDateTime('1970-01-01 00:00:00'), 'never', abs(dateDiff('second', d, now())) < 20, 'now', toString(d)); + (d)->multiIf(d==toDateTime('1970-01-01 00:00:00'), 'never', abs(dateDiff('second', d, now())) < 30, 'now', toString(d)); SELECT database, table, length(consumer_id), assignments.topic, assignments.partition_id, assignments.current_offset, stable_timestamp(last_exception_time) as last_exception_time_, if(length(last_exception)>0, last_exception, 'no exception') as last_exception_, @@ -4735,23 +4735,26 @@ is_currently_used: 0 """ ) - result_rdkafka_stat = instance.query( - """ - SELECT table, JSONExtractString(rdkafka_stat, 'type') - FROM system.kafka_consumers WHERE database='test' and table IN ('kafka', 'kafka2') format Vertical; - """ - ) + retries = 0 + result_rdkafka_stat = "" + while True: + result_rdkafka_stat = instance.query( + """ + SELECT table, JSONExtractString(rdkafka_stat, 'type') + FROM system.kafka_consumers WHERE database='test' and table = 'kafka' format Vertical; + """ + ) + if result_rdkafka_stat.find('consumer') or retries > max_retries: + break + retries += 1 + time.sleep(1) + assert ( result_rdkafka_stat == """Row 1: ────── table: kafka JSONExtractString(rdkafka_stat, 'type'): consumer - -Row 2: -────── -table: kafka2 -JSONExtractString(rdkafka_stat, 'type'): consumer """ ) From 2cc4bb355ca6fe73ddec1340a85481916c921711 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 10 Jul 2023 15:35:23 +0000 Subject: [PATCH 371/777] system_kafka_consumers: make black formatter happy --- tests/integration/test_storage_kafka/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index a88cc490a1a..75d2b4a95b5 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4620,7 +4620,7 @@ is_currently_used: 0 kafka_delete_topic(admin_client, topic) -def test_system_kafka_consumers_rebalance(kafka_cluster, max_retries = 15): +def test_system_kafka_consumers_rebalance(kafka_cluster, max_retries=15): # based on test_kafka_consumer_hang2 admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) @@ -4744,7 +4744,7 @@ is_currently_used: 0 FROM system.kafka_consumers WHERE database='test' and table = 'kafka' format Vertical; """ ) - if result_rdkafka_stat.find('consumer') or retries > max_retries: + if result_rdkafka_stat.find("consumer") or retries > max_retries: break retries += 1 time.sleep(1) From b325ddceb43e8f79c1c9d9d67d8967f245dc11ae Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 11 Jul 2023 16:27:52 +0000 Subject: [PATCH 372/777] system_kafka_consumers: small thing to make clang tidy happy --- src/Storages/System/StorageSystemKafkaConsumers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index a80f1280b1d..f571af0b095 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -97,7 +97,7 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context std::lock_guard lock(storage_kafka_ptr->mutex); - for (auto weak_consumer : storage_kafka_ptr->all_consumers) + for (const auto & weak_consumer : storage_kafka_ptr->all_consumers) { if (auto consumer = weak_consumer.lock()) { From ae3e6f0fac506cb50344d9d91fd7f3e37eb37d06 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 13 Jul 2023 21:47:11 +0000 Subject: [PATCH 373/777] system_kafka_consumers: getMemberId introduced to avoid leak --- src/Storages/Kafka/KafkaConsumer.cpp | 16 +++++++++++++++- src/Storages/Kafka/KafkaConsumer.h | 2 ++ tests/integration/test_storage_kafka/test.py | 1 + 3 files changed, 18 insertions(+), 1 deletion(-) diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index 12f36f4eb76..3bc124fcc77 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -543,6 +543,20 @@ void KafkaConsumer::setExceptionInfo(const String & text) last_exception_text = text; } +/* + * Needed until + * https://github.com/mfontanini/cppkafka/pull/309 + * is merged, + * because consumer->get_member_id() contains a leak + */ +std::string KafkaConsumer::getMemberId() +{ + char * memberid_ptr = rd_kafka_memberid(consumer->get_handle()); + std::string memberid_string = memberid_ptr; + rd_kafka_mem_free(nullptr, memberid_ptr); + return memberid_string; +} + KafkaConsumer::Stat KafkaConsumer::getStat() { @@ -562,7 +576,7 @@ KafkaConsumer::Stat KafkaConsumer::getStat() } return { - .consumer_id = consumer->get_member_id(), + .consumer_id = getMemberId() /* consumer->get_member_id() */ , .assignments = std::move(assignments), .last_exception = [&](){std::lock_guard lock(exception_mutex); return last_exception_text;}(), diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index 2f860d4769b..4e8c25b8abb 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -159,6 +159,8 @@ private: /// Return number of messages with an error. size_t filterMessageErrors(); ReadBufferPtr getNextMessage(); + + std::string getMemberId(); }; } diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 75d2b4a95b5..b43ecc022d5 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4617,6 +4617,7 @@ is_currently_used: 0 """ ) + instance.query("DROP TABLE test.kafka") kafka_delete_topic(admin_client, topic) From 6d6aae24f83f696c5e950acca8af25b45d394b2e Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 14 Jul 2023 20:51:05 +0000 Subject: [PATCH 374/777] system_kafka_consumers: no friendship with StorageKafka --- src/Storages/Kafka/StorageKafka.h | 11 +++++++++-- src/Storages/System/StorageSystemKafkaConsumers.cpp | 4 ++-- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index f379079bafe..81c8f72f117 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -81,6 +81,15 @@ public: HandleKafkaErrorMode getHandleKafkaErrorMode() const { return kafka_settings->kafka_handle_error_mode; } std::shared_ptr getRdkafkaStat() const { return rdkafka_stat; } + struct SafeConsumers + { + std::unique_lock lock; + std::vector & consumers; + std::shared_ptr storage_ptr; + }; + + SafeConsumers getSafeConsumers() { return {std::unique_lock(mutex), all_consumers, shared_from_this()}; } + private: // Configuration and state std::unique_ptr kafka_settings; @@ -149,8 +158,6 @@ private: bool checkDependencies(const StorageID & table_id); std::shared_ptr rdkafka_stat; - - friend class DB::StorageSystemKafkaConsumers; }; } diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index f571af0b095..a752b853f7c 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -95,9 +95,9 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context std::string database_str = it->databaseName(); std::string table_str = it->name(); - std::lock_guard lock(storage_kafka_ptr->mutex); + auto safe_consumers = storage_kafka_ptr->getSafeConsumers(); - for (const auto & weak_consumer : storage_kafka_ptr->all_consumers) + for (const auto & weak_consumer : safe_consumers.consumers) { if (auto consumer = weak_consumer.lock()) { From 80829a6bf7260c49e109220fb9e9f92c9bc647f0 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sun, 16 Jul 2023 20:37:46 +0000 Subject: [PATCH 375/777] system_kafka_consumers: no offset_committed, new test with MV --- .../system-tables/kafka_consumers.md | 1 - src/Storages/Kafka/KafkaConsumer.cpp | 2 - src/Storages/Kafka/KafkaConsumer.h | 1 - src/Storages/Kafka/StorageKafka.h | 4 +- .../System/StorageSystemKafkaConsumers.cpp | 7 -- tests/integration/test_storage_kafka/test.py | 70 ++++++++++++++++++- 6 files changed, 71 insertions(+), 14 deletions(-) diff --git a/docs/en/operations/system-tables/kafka_consumers.md b/docs/en/operations/system-tables/kafka_consumers.md index 1a64fdd5353..a45d5dbdf0b 100644 --- a/docs/en/operations/system-tables/kafka_consumers.md +++ b/docs/en/operations/system-tables/kafka_consumers.md @@ -14,7 +14,6 @@ Columns: - `assignments.topic` (Array(String)) - Kafka topic. - `assignments.partition_id` (Array(Int32)) - Kafka partition id. Note, that only one consumer can be assigned to a partition. - `assignments.current_offset` (Array(Int64)) - current offset. -- `assignments.offset_committed` (Array(Int64)) - committed offset. - `last_exception_time`, (DateTime) - timestamp when the most recent exception was generated. - `last_exception`, (String) - text of the most recent exception. - `last_poll_time`, (DateTime) - timestamp of the most recent poll. diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index 3bc124fcc77..14eb7fd2e0b 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -563,7 +563,6 @@ KafkaConsumer::Stat KafkaConsumer::getStat() KafkaConsumer::Stat::Assignments assignments; auto cpp_assignments = consumer->get_assignment(); auto cpp_offsets = consumer->get_offsets_position(cpp_assignments); - auto cpp_offsets_committed = consumer->get_offsets_committed(cpp_assignments); for (size_t num = 0; num < cpp_assignments.size(); ++num) { @@ -571,7 +570,6 @@ KafkaConsumer::Stat KafkaConsumer::getStat() cpp_assignments[num].get_topic(), cpp_assignments[num].get_partition(), cpp_offsets[num].get_offset(), - cpp_offsets_committed[num].get_offset() }); } diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index 4e8c25b8abb..65106148c59 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -34,7 +34,6 @@ public: String topic_str; Int32 partition_id; Int64 current_offset; - Int64 offset_committed; }; using Assignments = std::vector; diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 81c8f72f117..415e03d11b3 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -83,12 +83,12 @@ public: struct SafeConsumers { + std::shared_ptr storage_ptr; std::unique_lock lock; std::vector & consumers; - std::shared_ptr storage_ptr; }; - SafeConsumers getSafeConsumers() { return {std::unique_lock(mutex), all_consumers, shared_from_this()}; } + SafeConsumers getSafeConsumers() { return {shared_from_this(), std::unique_lock(mutex), all_consumers}; } private: // Configuration and state diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index a752b853f7c..30e07a9549e 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -31,7 +31,6 @@ NamesAndTypesList StorageSystemKafkaConsumers::getNamesAndTypes() {"assignments.topic", std::make_shared(std::make_shared())}, {"assignments.partition_id", std::make_shared(std::make_shared())}, {"assignments.current_offset", std::make_shared(std::make_shared())}, - {"assignments.offset_committed", std::make_shared(std::make_shared())}, {"last_exception_time", std::make_shared()}, {"last_exception", std::make_shared()}, {"last_poll_time", std::make_shared()}, @@ -67,9 +66,6 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context auto & assigments_current_offset = assert_cast(assert_cast(*res_columns[index]).getData()); auto & assigments_current_offset_offsets = assert_cast(*res_columns[index++]).getOffsets(); - auto & assigments_offset_committed = assert_cast(assert_cast(*res_columns[index]).getData()); - auto & assigments_offset_committed_offsets = assert_cast(*res_columns[index++]).getOffsets(); - auto & last_exception_time = assert_cast(*res_columns[index++]); auto & last_exception = assert_cast(*res_columns[index++]); auto & last_poll_time = assert_cast(*res_columns[index++]); @@ -118,15 +114,12 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context assigments_partition_id.insert(assign.partition_id); assigments_current_offset.insert(assign.current_offset); - assigments_offset_committed.insert(assign.offset_committed); } last_assignment_num += num_assignnemts; assigments_topics_offsets.push_back(last_assignment_num); assigments_partition_id_offsets.push_back(last_assignment_num); assigments_current_offset_offsets.push_back(last_assignment_num); - assigments_offset_committed_offsets.push_back(last_assignment_num); - last_exception.insertData(consumer_stat.last_exception.data(), consumer_stat.last_exception.size()); last_exception_time.insert(consumer_stat.last_exception_time); diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index b43ecc022d5..f935b1ca582 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4656,7 +4656,6 @@ def test_system_kafka_consumers_rebalance(kafka_cluster, max_retries=15): kafka_commit_on_select = 1, kafka_group_name = '{topic}', kafka_format = 'JSONEachRow'; - DROP TABLE IF EXISTS test.kafka_persistent; """ ) @@ -4736,6 +4735,71 @@ is_currently_used: 0 """ ) + instance.query("DROP TABLE test.kafka") + instance.query("DROP TABLE test.kafka2") + + kafka_delete_topic(admin_client, topic) + + +def test_system_kafka_consumers_rebalance_mv(kafka_cluster, max_retries=15): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + producer = KafkaProducer( + bootstrap_servers="localhost:{}".format(cluster.kafka_port), + value_serializer=producer_serializer, + key_serializer=producer_serializer, + ) + + topic = "system_kafka_cons_mv" + kafka_create_topic(admin_client, topic, num_partitions=2) + + instance.query( + f""" + DROP TABLE IF EXISTS test.kafka; + DROP TABLE IF EXISTS test.kafka2; + DROP TABLE IF EXISTS test.kafka_persistent; + DROP TABLE IF EXISTS test.kafka_persistent2; + + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic}', + kafka_group_name = '{topic}', + kafka_commit_on_select = 1, + kafka_format = 'JSONEachRow'; + + CREATE TABLE test.kafka2 (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic}', + kafka_commit_on_select = 1, + kafka_group_name = '{topic}', + kafka_format = 'JSONEachRow'; + + CREATE TABLE test.kafka_persistent (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE TABLE test.kafka_persistent2 (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + + CREATE MATERIALIZED VIEW test.persistent_kafka_mv TO test.kafka_persistent AS + SELECT key, value + FROM test.kafka; + + CREATE MATERIALIZED VIEW test.persistent_kafka_mv2 TO test.kafka_persistent2 AS + SELECT key, value + FROM test.kafka2; + """ + ) + + producer.send(topic=topic, value=json.dumps({"key": 1, "value": 1}), partition=0) + producer.send(topic=topic, value=json.dumps({"key": 11, "value": 11}), partition=1) + time.sleep(3) + + retries = 0 result_rdkafka_stat = "" while True: @@ -4761,6 +4825,10 @@ JSONExtractString(rdkafka_stat, 'type'): consumer instance.query("DROP TABLE test.kafka") instance.query("DROP TABLE test.kafka2") + instance.query("DROP TABLE test.kafka_persistent") + instance.query("DROP TABLE test.kafka_persistent2") + instance.query("DROP TABLE test.persistent_kafka_mv") + instance.query("DROP TABLE test.persistent_kafka_mv2") kafka_delete_topic(admin_client, topic) From e8b10f012649d7d2a99882e0d1082a30207fcdb0 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sun, 16 Jul 2023 20:59:26 +0000 Subject: [PATCH 376/777] system_kafka_consumers: black formatter thing --- tests/integration/test_storage_kafka/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index f935b1ca582..3f5f497897e 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -4799,7 +4799,6 @@ def test_system_kafka_consumers_rebalance_mv(kafka_cluster, max_retries=15): producer.send(topic=topic, value=json.dumps({"key": 11, "value": 11}), partition=1) time.sleep(3) - retries = 0 result_rdkafka_stat = "" while True: From 7dab2dc0416b90e41d44cb03d5a5f1e9fa3893eb Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 1 Aug 2023 12:01:13 +0000 Subject: [PATCH 377/777] system_kafka_consumers: exceptions seem to work --- src/Storages/Kafka/KafkaConsumer.cpp | 21 ++- src/Storages/Kafka/KafkaConsumer.h | 21 ++- src/Storages/Kafka/KafkaSource.cpp | 1 + .../System/StorageSystemKafkaConsumers.cpp | 26 +++- .../test_kafka_bad_messages/test.py | 126 +++++++++++++++++- tests/integration/test_storage_kafka/test.py | 20 ++- 6 files changed, 193 insertions(+), 22 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index 14eb7fd2e0b..5b0cfe06f50 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -61,6 +61,7 @@ KafkaConsumer::KafkaConsumer( , stopped(stopped_) , current(messages.begin()) , topics(_topics) + , exceptions_buffer(EXCEPTIONS_DEPTH) { // called (synchronously, during poll) when we enter the consumer group consumer->set_assignment_callback([this](const cppkafka::TopicPartitionList & topic_partitions) @@ -128,6 +129,7 @@ KafkaConsumer::KafkaConsumer( { LOG_ERROR(log, "Rebalance error: {}", err); ProfileEvents::increment(ProfileEvents::KafkaRebalanceErrors); + setExceptionInfo(err); }); } @@ -181,6 +183,7 @@ void KafkaConsumer::drain() else { LOG_ERROR(log, "Error during draining: {}", error); + setExceptionInfo(error); } } @@ -265,7 +268,10 @@ void KafkaConsumer::commit() if (e.get_error() == RD_KAFKA_RESP_ERR__NO_OFFSET) committed = true; else + { LOG_ERROR(log, "Exception during commit attempt: {}", e.what()); + setExceptionInfo(e.what()); + } } --max_retries; } @@ -503,6 +509,7 @@ size_t KafkaConsumer::filterMessageErrors() { ProfileEvents::increment(ProfileEvents::KafkaConsumerErrors); LOG_ERROR(log, "Consumer error: {}", error); + setExceptionInfo(error); return true; } return false; @@ -535,12 +542,15 @@ void KafkaConsumer::storeLastReadMessageOffset() } } +void KafkaConsumer::setExceptionInfo(const cppkafka::Error & err) +{ + setExceptionInfo(err.to_string()); +} + void KafkaConsumer::setExceptionInfo(const String & text) { - last_exception_timestamp_usec = static_cast(Poco::Timestamp().epochTime()); - std::lock_guard lock(exception_mutex); - last_exception_text = text; + exceptions_buffer.push_back({text, static_cast(Poco::Timestamp().epochTime())}); } /* @@ -576,9 +586,6 @@ KafkaConsumer::Stat KafkaConsumer::getStat() return { .consumer_id = getMemberId() /* consumer->get_member_id() */ , .assignments = std::move(assignments), - .last_exception = [&](){std::lock_guard lock(exception_mutex); - return last_exception_text;}(), - .last_exception_time = last_exception_timestamp_usec.load(), .last_poll_time = last_poll_timestamp_usec.load(), .num_messages_read = num_messages_read.load(), @@ -587,6 +594,8 @@ KafkaConsumer::Stat KafkaConsumer::getStat() .num_commits = num_commits.load(), .num_rebalance_assignments = num_rebalance_assignments.load(), .num_rebalance_revocations = num_rebalance_revocations.load(), + .exceptions_buffer = [&](){std::lock_guard lock(exception_mutex); + return exceptions_buffer;}(), .in_use = in_use.load() }; } diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index 65106148c59..cf81b8133d4 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include #include @@ -27,6 +29,13 @@ using ConsumerPtr = std::shared_ptr; class KafkaConsumer { public: + struct ExceptionInfo + { + String text; + UInt64 timestamp_usec; + }; + using ExceptionsBuffer = boost::circular_buffer; + struct Stat // system.kafka_consumers data { struct Assignment @@ -39,8 +48,8 @@ public: String consumer_id; Assignments assignments; - String last_exception; - UInt64 last_exception_time; + // String last_exception; + // UInt64 last_exception_time; UInt64 last_poll_time; UInt64 num_messages_read; UInt64 last_commit_timestamp_usec; @@ -48,6 +57,7 @@ public: UInt64 num_commits; UInt64 num_rebalance_assignments; UInt64 num_rebalance_revocations; + KafkaConsumer::ExceptionsBuffer exceptions_buffer; bool in_use; }; @@ -96,6 +106,7 @@ public: auto currentTimestamp() const { return current[-1].get_timestamp(); } const auto & currentHeaderList() const { return current[-1].get_header_list(); } String currentPayload() const { return current[-1].get_payload(); } + void setExceptionInfo(const cppkafka::Error & err); void setExceptionInfo(const String & text); void inUse() { in_use = true; } void notInUse() { in_use = false; } @@ -103,6 +114,7 @@ public: // For system.kafka_consumers Stat getStat(); + private: using Messages = std::vector; CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaConsumers}; @@ -140,7 +152,10 @@ private: /// system.kafka_consumers data is retrieved asynchronously, mutable std::mutex exception_mutex; - String last_exception_text; + + const size_t EXCEPTIONS_DEPTH = 10; + + ExceptionsBuffer exceptions_buffer; std::atomic last_exception_timestamp_usec = 0; std::atomic last_poll_timestamp_usec = 0; diff --git a/src/Storages/Kafka/KafkaSource.cpp b/src/Storages/Kafka/KafkaSource.cpp index ba242417058..cd83a6a1422 100644 --- a/src/Storages/Kafka/KafkaSource.cpp +++ b/src/Storages/Kafka/KafkaSource.cpp @@ -133,6 +133,7 @@ Chunk KafkaSource::generateImpl() { e.addMessage("while parsing Kafka message (topic: {}, partition: {}, offset: {})'", consumer->currentTopic(), consumer->currentPartition(), consumer->currentOffset()); + consumer->setExceptionInfo(e.message()); throw std::move(e); } }; diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 30e07a9549e..0e13d095967 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -31,8 +31,8 @@ NamesAndTypesList StorageSystemKafkaConsumers::getNamesAndTypes() {"assignments.topic", std::make_shared(std::make_shared())}, {"assignments.partition_id", std::make_shared(std::make_shared())}, {"assignments.current_offset", std::make_shared(std::make_shared())}, - {"last_exception_time", std::make_shared()}, - {"last_exception", std::make_shared()}, + {"exceptions.time", std::make_shared(std::make_shared())}, + {"exceptions.text", std::make_shared(std::make_shared())}, {"last_poll_time", std::make_shared()}, {"num_messages_read", std::make_shared()}, {"last_commit_time", std::make_shared()}, @@ -66,8 +66,10 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context auto & assigments_current_offset = assert_cast(assert_cast(*res_columns[index]).getData()); auto & assigments_current_offset_offsets = assert_cast(*res_columns[index++]).getOffsets(); - auto & last_exception_time = assert_cast(*res_columns[index++]); - auto & last_exception = assert_cast(*res_columns[index++]); + auto & exceptions_time = assert_cast(assert_cast(*res_columns[index]).getData()); + auto & exceptions_time_offset = assert_cast(*res_columns[index++]).getOffsets(); + auto & exceptions_text = assert_cast(assert_cast(*res_columns[index]).getData()); + auto & exceptions_text_offset = assert_cast(*res_columns[index++]).getOffsets(); auto & last_poll_time = assert_cast(*res_columns[index++]); auto & num_messages_read = assert_cast(*res_columns[index++]); auto & last_commit_time = assert_cast(*res_columns[index++]); @@ -80,6 +82,7 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context const auto access = context->getAccess(); size_t last_assignment_num = 0; + size_t exceptions_num = 0; auto add_row = [&](const DatabaseTablesIteratorPtr & it, StorageKafka * storage_kafka_ptr) { @@ -121,8 +124,19 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context assigments_partition_id_offsets.push_back(last_assignment_num); assigments_current_offset_offsets.push_back(last_assignment_num); - last_exception.insertData(consumer_stat.last_exception.data(), consumer_stat.last_exception.size()); - last_exception_time.insert(consumer_stat.last_exception_time); + // last_exception.insertData(consumer_stat.last_exception.data(), consumer_stat.last_exception.size()); + // last_exception_time.insert(consumer_stat.last_exception_time); + for (auto excit = consumer_stat.exceptions_buffer.begin(); + excit != consumer_stat.exceptions_buffer.end(); + ++excit) + { + exceptions_text.insertData(excit->text.data(), excit->text.size()); + exceptions_time.insert(excit->timestamp_usec); + } + exceptions_num += consumer_stat.exceptions_buffer.size(); + exceptions_text_offset.push_back(exceptions_num); + exceptions_time_offset.push_back(exceptions_num); + last_poll_time.insert(consumer_stat.last_poll_time); num_messages_read.insert(consumer_stat.num_messages_read); diff --git a/tests/integration/test_kafka_bad_messages/test.py b/tests/integration/test_kafka_bad_messages/test.py index da3cf36c853..7cf8c593ec0 100644 --- a/tests/integration/test_kafka_bad_messages/test.py +++ b/tests/integration/test_kafka_bad_messages/test.py @@ -90,7 +90,7 @@ def producer_serializer(x): return x.encode() if isinstance(x, str) else x -def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=15): +def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=15, partition=None): logging.debug( "kafka_produce server:{}:{} topic:{}".format( "localhost", kafka_cluster.kafka_port, topic @@ -100,7 +100,7 @@ def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=15): kafka_cluster.kafka_port, producer_serializer, retries ) for message in messages: - producer.send(topic=topic, value=message, timestamp_ms=timestamp) + producer.send(topic=topic, value=message, timestamp_ms=timestamp, partition=partition) producer.flush() @@ -115,7 +115,7 @@ def kafka_cluster(): cluster.shutdown() -def test_bad_messages_parsing(kafka_cluster): +def test_bad_messages_parsing_stream(kafka_cluster): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) ) @@ -227,6 +227,13 @@ message Message { assert rows == len(messages) + result_system_kafka_consumers = instance.query( + """ + SELECT last_exception, last_exception_time, database, table FROM system.kafka_consumers + """ + ) + logging.debug(f"result_system_kafka_consumers (test_bad_messages_parsing 2): {result_system_kafka_consumers}") + kafka_delete_topic(admin_client, f"{format_name}_err") capn_proto_schema = """ @@ -244,7 +251,7 @@ struct Message f""" DROP TABLE IF EXISTS view; DROP TABLE IF EXISTS kafka; - + CREATE TABLE kafka (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', @@ -253,9 +260,9 @@ struct Message kafka_format = 'CapnProto', kafka_handle_error_mode='stream', kafka_schema='schema_test_errors:Message'; - + CREATE MATERIALIZED VIEW view Engine=Log AS - SELECT _error FROM kafka WHERE length(_error) != 0 ; + SELECT _error FROM kafka WHERE length(_error) != 0 ; """ ) @@ -278,6 +285,113 @@ struct Message kafka_delete_topic(admin_client, "CapnProto_err") +def test_bad_messages_parsing_exception(kafka_cluster, max_retries=20): + admin_client = KafkaAdminClient( + bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) + ) + + for format_name in [ + "Avro", + "JSONEachRow", + ]: + print(format_name) + + kafka_create_topic(admin_client, f"{format_name}_err", num_partitions=2) + + instance.query( + f""" + DROP TABLE IF EXISTS view_{format_name}; + DROP TABLE IF EXISTS kafka_{format_name}; + + CREATE TABLE kafka_{format_name} (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{format_name}_err', + kafka_group_name = '{format_name}', + kafka_format = '{format_name}', + kafka_num_consumers = 2; + + CREATE MATERIALIZED VIEW view_{format_name} Engine=Log AS + SELECT * FROM kafka_{format_name}; + """ + ) + + kafka_produce(kafka_cluster, f"{format_name}_err", ["qwertyuiop"], partition=0) + kafka_produce(kafka_cluster, f"{format_name}_err", ["asdfghjkl"], partition=1) + kafka_produce(kafka_cluster, f"{format_name}_err", ["zxcvbnm"], partition=0) + + time.sleep(6) + + result_system_kafka_consumers = instance.query( + # """ + # SELECT exceptions.text[1], length(exceptions.text) > 3 AND length(exceptions.text) < 10, length(exceptions.time) > 3 AND length(exceptions.time) < 10, abs(dateDiff('second', exceptions.time[1], now())) < 30, database, table FROM system.kafka_consumers ORDER BY table, assignments.partition_id[1] + # """ + """ + SELECT length(exceptions.text), length(exceptions.time), exceptions.time[1], database, table FROM system.kafka_consumers ORDER BY table, assignments.partition_id[1] + """ + ) + logging.debug(f"result_system_kafka_consumers (test_bad_messages_parsing_exception 1): {result_system_kafka_consumers}") + + + logging.debug(f"result_system_kafka_consumers (test_bad_messages_parsing_exception 2): {result_system_kafka_consumers}") + expected_result = """1|1|1|default|kafka_Avro +1|1|1|default|kafka_Avro +1|1|1|default|kafka_JSONEachRow +1|1|1|default|kafka_JSONEachRow +""" + retries = 0 + result_system_kafka_consumers = "" + while True: + result_system_kafka_consumers = instance.query( + # """ + # SELECT exceptions.text[1], length(exceptions.text) > 3 AND length(exceptions.text) < 10, length(exceptions.time) > 3 AND length(exceptions.time) < 10, abs(dateDiff('second', exceptions.time[1], now())) < 30, database, table FROM system.kafka_consumers ORDER BY table, assignments.partition_id[1] + # """ + # """ + # SELECT CONCAT((length(exceptions.text) > 2 AND length(exceptions.text) < 15)::String, '_', (length(exceptions.time) > 2 AND length(exceptions.time) < 15)::String, '_', (abs(dateDiff('second', exceptions.time[1], now())) < 40)::String, '_', database, '_', table) FROM system.kafka_consumers ORDER BY table, assignments.partition_id[1] format PrettySpaceNoEscapesMonoBlock + # """ + """ + SELECT length(exceptions.text) > 1 AND length(exceptions.text) < 15, length(exceptions.time) > 1 AND length(exceptions.time) < 15, abs(dateDiff('second', exceptions.time[1], now())) < 40, database, table FROM system.kafka_consumers ORDER BY table, assignments.partition_id[1] + """ + ) + result_system_kafka_consumers = result_system_kafka_consumers.replace('\t', '|') + if result_system_kafka_consumers == expected_result or retries > max_retries: + break + retries += 1 + time.sleep(1) + +# assert result_system_kafka_consumers == """ avro::Exception: Invalid data file. Magic does not match: : while parsing Kafka message (topic: Avro_err, partition: 0, offset: 0)' 1 1 1 default kafka_Avro +# avro::Exception: Invalid data file. Magic does not match: : while parsing Kafka message (topic: Avro_err, partition: 1, offset: 0)' 1 1 1 default kafka_Avro +# Cannot parse input: expected '{' before: 'qwertyuiop': while parsing Kafka message (topic: JSONEachRow_err, partition: 0, offset: 0)' 1 1 1 default kafka_JSONEachRow +# Cannot parse input: expected '{' before: 'asdfghjkl': while parsing Kafka message (topic: JSONEachRow_err, partition: 1, offset: 0)' 1 1 1 default kafka_JSONEachRow""" + + + assert result_system_kafka_consumers == expected_result + + # for format_name in [ + # "TSV", + # "TSKV", + # "JSONEachRow", + # "CSV", + # "Values", + # "JSON", + # "JSONCompactEachRow", + # "JSONObjectEachRow", + # "Avro", + # "RowBinary", + # "JSONColumns", + # "JSONColumnsWithMetadata", + # "Native", + # "Arrow", + # "ArrowStream", + # "Parquet", + # "ORC", + # "JSONCompactColumns", + # "BSONEachRow", + # "MySQLDump", + # ]: + # kafka_delete_topic(admin_client, f"{format_name}_err") + + if __name__ == "__main__": cluster.start() diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 3f5f497897e..6805bfbbbd4 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1929,6 +1929,15 @@ def test_kafka_flush_on_big_message(kafka_cluster): if int(result) == kafka_messages * batch_messages: break + result_system_kafka_consumers = instance.query( + """ + SELECT last_exception, last_exception_time, database, table FROM system.kafka_consumers + """ + ) + logging.debug(f"result_system_kafka_consumers (test_kafka_flush_on_big_message): {result_system_kafka_consumers}") + assert (result_system_kafka_consumers == "fake string") + + instance.query( """ DROP TABLE test.consumer; @@ -3213,6 +3222,15 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): result = instance.query("SELECT count(), uniqExact(key), max(key) FROM test.view") logging.debug(result) + + result_system_kafka_consumers = instance.query( + """ + SELECT last_exception, last_exception_time, database, table FROM system.kafka_consumers + """ + ) + logging.debug(f"result_system_kafka_consumers (test_kafka_duplicates_when_commit_failed): {result_system_kafka_consumers}") + + instance.query( """ DROP TABLE test.consumer SYNC; @@ -4692,7 +4710,7 @@ def test_system_kafka_consumers_rebalance(kafka_cluster, max_retries=15): FROM system.kafka_consumers WHERE database='test' and table IN ('kafka', 'kafka2') format Vertical; """ ) - logging.debug(f"result_system_kafka_consumers (1): {result_system_kafka_consumers}") + logging.debug(f"result_system_kafka_consumers: {result_system_kafka_consumers}") assert ( result_system_kafka_consumers == """Row 1: From 978d8bf9a621913a284e96edbb3556bf1de59a76 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 1 Aug 2023 14:33:23 +0000 Subject: [PATCH 378/777] system_kafka_consumers: cleanup with exceptions --- .../system-tables/kafka_consumers.md | 9 +-- src/Storages/Kafka/KafkaConsumer.h | 8 +-- .../System/StorageSystemKafkaConsumers.cpp | 6 +- .../test_kafka_bad_messages/test.py | 68 +++---------------- tests/integration/test_storage_kafka/test.py | 28 ++------ 5 files changed, 26 insertions(+), 93 deletions(-) diff --git a/docs/en/operations/system-tables/kafka_consumers.md b/docs/en/operations/system-tables/kafka_consumers.md index a45d5dbdf0b..7e28a251e26 100644 --- a/docs/en/operations/system-tables/kafka_consumers.md +++ b/docs/en/operations/system-tables/kafka_consumers.md @@ -14,8 +14,8 @@ Columns: - `assignments.topic` (Array(String)) - Kafka topic. - `assignments.partition_id` (Array(Int32)) - Kafka partition id. Note, that only one consumer can be assigned to a partition. - `assignments.current_offset` (Array(Int64)) - current offset. -- `last_exception_time`, (DateTime) - timestamp when the most recent exception was generated. -- `last_exception`, (String) - text of the most recent exception. +- `exceptions.time`, (Array(DateTime)) - timestamp when the 10 most recent exceptions were generated. +- `exceptions.text`, (Array(String)) - text of 10 most recent exceptions. - `last_poll_time`, (DateTime) - timestamp of the most recent poll. - `num_messages_read`, (UInt64) - number of messages read by the consumer. - `last_commit_time`, (DateTime) - timestamp of the most recent poll. @@ -43,8 +43,8 @@ consumer_id: ClickHouse-instance-test-kafka-1caddc7f-f917-4bb1-ac assignments.topic: ['system_kafka_cons'] assignments.partition_id: [0] assignments.current_offset: [18446744073709550615] -last_exception_time: 1970-01-01 00:00:00 -last_exception: +exceptions.time: [] +exceptions.text: [] last_poll_time: 2006-11-09 18:47:47 num_messages_read: 4 last_commit_time: 2006-11-10 04:39:40 @@ -53,5 +53,6 @@ last_rebalance_time: 1970-01-01 00:00:00 num_rebalance_revocations: 0 num_rebalance_assignments: 1 is_currently_used: 1 +rdkafka_stat: {...} ``` diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index cf81b8133d4..64c76e7b837 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -48,8 +48,6 @@ public: String consumer_id; Assignments assignments; - // String last_exception; - // UInt64 last_exception_time; UInt64 last_poll_time; UInt64 num_messages_read; UInt64 last_commit_timestamp_usec; @@ -114,7 +112,6 @@ public: // For system.kafka_consumers Stat getStat(); - private: using Messages = std::vector; CurrentMetrics::Increment metric_increment{CurrentMetrics::KafkaConsumers}; @@ -150,11 +147,10 @@ private: std::optional assignment; const Names topics; - /// system.kafka_consumers data is retrieved asynchronously, + /// system.kafka_consumers data is retrieved asynchronously + /// so we have to protect exceptions_buffer mutable std::mutex exception_mutex; - const size_t EXCEPTIONS_DEPTH = 10; - ExceptionsBuffer exceptions_buffer; std::atomic last_exception_timestamp_usec = 0; diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 0e13d095967..213b7b326ac 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -124,10 +124,8 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context assigments_partition_id_offsets.push_back(last_assignment_num); assigments_current_offset_offsets.push_back(last_assignment_num); - // last_exception.insertData(consumer_stat.last_exception.data(), consumer_stat.last_exception.size()); - // last_exception_time.insert(consumer_stat.last_exception_time); - for (auto excit = consumer_stat.exceptions_buffer.begin(); - excit != consumer_stat.exceptions_buffer.end(); + for (auto excit = consumer_stat.exceptions_buffer.cbegin(); + excit != consumer_stat.exceptions_buffer.cend(); ++excit) { exceptions_text.insertData(excit->text.data(), excit->text.size()); diff --git a/tests/integration/test_kafka_bad_messages/test.py b/tests/integration/test_kafka_bad_messages/test.py index 7cf8c593ec0..92b4af94e2d 100644 --- a/tests/integration/test_kafka_bad_messages/test.py +++ b/tests/integration/test_kafka_bad_messages/test.py @@ -262,7 +262,7 @@ struct Message kafka_schema='schema_test_errors:Message'; CREATE MATERIALIZED VIEW view Engine=Log AS - SELECT _error FROM kafka WHERE length(_error) != 0 ; + SELECT _error FROM kafka WHERE length(_error) != 0; """ ) @@ -320,37 +320,17 @@ def test_bad_messages_parsing_exception(kafka_cluster, max_retries=20): kafka_produce(kafka_cluster, f"{format_name}_err", ["asdfghjkl"], partition=1) kafka_produce(kafka_cluster, f"{format_name}_err", ["zxcvbnm"], partition=0) - time.sleep(6) - - result_system_kafka_consumers = instance.query( - # """ - # SELECT exceptions.text[1], length(exceptions.text) > 3 AND length(exceptions.text) < 10, length(exceptions.time) > 3 AND length(exceptions.time) < 10, abs(dateDiff('second', exceptions.time[1], now())) < 30, database, table FROM system.kafka_consumers ORDER BY table, assignments.partition_id[1] - # """ - """ - SELECT length(exceptions.text), length(exceptions.time), exceptions.time[1], database, table FROM system.kafka_consumers ORDER BY table, assignments.partition_id[1] - """ - ) - logging.debug(f"result_system_kafka_consumers (test_bad_messages_parsing_exception 1): {result_system_kafka_consumers}") - - - logging.debug(f"result_system_kafka_consumers (test_bad_messages_parsing_exception 2): {result_system_kafka_consumers}") - expected_result = """1|1|1|default|kafka_Avro -1|1|1|default|kafka_Avro -1|1|1|default|kafka_JSONEachRow -1|1|1|default|kafka_JSONEachRow + expected_result = """avro::Exception: Invalid data file. Magic does not match: : while parsing Kafka message (topic: Avro_err, partition: 0, offset: 0)\\'|1|1|1|default|kafka_Avro +avro::Exception: Invalid data file. Magic does not match: : while parsing Kafka message (topic: Avro_err, partition: 1, offset: 0)\\'|1|1|1|default|kafka_Avro +Cannot parse input: expected \\'{\\' before: \\'qwertyuiop\\': while parsing Kafka message (topic: JSONEachRow_err, partition: 0, offset: 0)\\'|1|1|1|default|kafka_JSONEachRow +Cannot parse input: expected \\'{\\' before: \\'asdfghjkl\\': while parsing Kafka message (topic: JSONEachRow_err, partition: 1, offset: 0)\\'|1|1|1|default|kafka_JSONEachRow """ retries = 0 result_system_kafka_consumers = "" while True: result_system_kafka_consumers = instance.query( - # """ - # SELECT exceptions.text[1], length(exceptions.text) > 3 AND length(exceptions.text) < 10, length(exceptions.time) > 3 AND length(exceptions.time) < 10, abs(dateDiff('second', exceptions.time[1], now())) < 30, database, table FROM system.kafka_consumers ORDER BY table, assignments.partition_id[1] - # """ - # """ - # SELECT CONCAT((length(exceptions.text) > 2 AND length(exceptions.text) < 15)::String, '_', (length(exceptions.time) > 2 AND length(exceptions.time) < 15)::String, '_', (abs(dateDiff('second', exceptions.time[1], now())) < 40)::String, '_', database, '_', table) FROM system.kafka_consumers ORDER BY table, assignments.partition_id[1] format PrettySpaceNoEscapesMonoBlock - # """ """ - SELECT length(exceptions.text) > 1 AND length(exceptions.text) < 15, length(exceptions.time) > 1 AND length(exceptions.time) < 15, abs(dateDiff('second', exceptions.time[1], now())) < 40, database, table FROM system.kafka_consumers ORDER BY table, assignments.partition_id[1] + SELECT exceptions.text[1], length(exceptions.text) > 1 AND length(exceptions.text) < 15, length(exceptions.time) > 1 AND length(exceptions.time) < 15, abs(dateDiff('second', exceptions.time[1], now())) < 40, database, table FROM system.kafka_consumers ORDER BY table, assignments.partition_id[1] """ ) result_system_kafka_consumers = result_system_kafka_consumers.replace('\t', '|') @@ -359,39 +339,13 @@ def test_bad_messages_parsing_exception(kafka_cluster, max_retries=20): retries += 1 time.sleep(1) -# assert result_system_kafka_consumers == """ avro::Exception: Invalid data file. Magic does not match: : while parsing Kafka message (topic: Avro_err, partition: 0, offset: 0)' 1 1 1 default kafka_Avro -# avro::Exception: Invalid data file. Magic does not match: : while parsing Kafka message (topic: Avro_err, partition: 1, offset: 0)' 1 1 1 default kafka_Avro -# Cannot parse input: expected '{' before: 'qwertyuiop': while parsing Kafka message (topic: JSONEachRow_err, partition: 0, offset: 0)' 1 1 1 default kafka_JSONEachRow -# Cannot parse input: expected '{' before: 'asdfghjkl': while parsing Kafka message (topic: JSONEachRow_err, partition: 1, offset: 0)' 1 1 1 default kafka_JSONEachRow""" - - assert result_system_kafka_consumers == expected_result - # for format_name in [ - # "TSV", - # "TSKV", - # "JSONEachRow", - # "CSV", - # "Values", - # "JSON", - # "JSONCompactEachRow", - # "JSONObjectEachRow", - # "Avro", - # "RowBinary", - # "JSONColumns", - # "JSONColumnsWithMetadata", - # "Native", - # "Arrow", - # "ArrowStream", - # "Parquet", - # "ORC", - # "JSONCompactColumns", - # "BSONEachRow", - # "MySQLDump", - # ]: - # kafka_delete_topic(admin_client, f"{format_name}_err") - - + for format_name in [ + "Avro", + "JSONEachRow", + ]: + kafka_delete_topic(admin_client, f"{format_name}_err") if __name__ == "__main__": cluster.start() diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 6805bfbbbd4..36815badd53 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1929,15 +1929,6 @@ def test_kafka_flush_on_big_message(kafka_cluster): if int(result) == kafka_messages * batch_messages: break - result_system_kafka_consumers = instance.query( - """ - SELECT last_exception, last_exception_time, database, table FROM system.kafka_consumers - """ - ) - logging.debug(f"result_system_kafka_consumers (test_kafka_flush_on_big_message): {result_system_kafka_consumers}") - assert (result_system_kafka_consumers == "fake string") - - instance.query( """ DROP TABLE test.consumer; @@ -3222,15 +3213,6 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): result = instance.query("SELECT count(), uniqExact(key), max(key) FROM test.view") logging.debug(result) - - result_system_kafka_consumers = instance.query( - """ - SELECT last_exception, last_exception_time, database, table FROM system.kafka_consumers - """ - ) - logging.debug(f"result_system_kafka_consumers (test_kafka_duplicates_when_commit_failed): {result_system_kafka_consumers}") - - instance.query( """ DROP TABLE test.consumer SYNC; @@ -4603,8 +4585,9 @@ def test_system_kafka_consumers(kafka_cluster): (d)->multiIf(d==toDateTime('1970-01-01 00:00:00'), 'never', abs(dateDiff('second', d, now())) < 30, 'now', toString(d)); SELECT database, table, length(consumer_id), assignments.topic, assignments.partition_id, - assignments.current_offset, stable_timestamp(last_exception_time) as last_exception_time_, - if(length(last_exception)>0, last_exception, 'no exception') as last_exception_, + assignments.current_offset, + if(length(exceptions.time)>0, exceptions.time[1]::String, 'never') as last_exception_time_, + if(length(exceptions.text)>0, exceptions.text[1], 'no exception') as last_exception_, stable_timestamp(last_poll_time) as last_poll_time_, num_messages_read, stable_timestamp(last_commit_time) as last_commit_time_, num_commits, stable_timestamp(last_rebalance_time) as last_rebalance_time_, num_rebalance_revocations, num_rebalance_assignments, is_currently_used @@ -4702,8 +4685,9 @@ def test_system_kafka_consumers_rebalance(kafka_cluster, max_retries=15): create or replace function stable_timestamp as (d)->multiIf(d==toDateTime('1970-01-01 00:00:00'), 'never', abs(dateDiff('second', d, now())) < 30, 'now', toString(d)); SELECT database, table, length(consumer_id), assignments.topic, assignments.partition_id, - assignments.current_offset, stable_timestamp(last_exception_time) as last_exception_time_, - if(length(last_exception)>0, last_exception, 'no exception') as last_exception_, + assignments.current_offset, + if(length(exceptions.time)>0, exceptions.time[1]::String, 'never') as last_exception_time_, + if(length(exceptions.text)>0, exceptions.text[1], 'no exception') as last_exception_, stable_timestamp(last_poll_time) as last_poll_time_, num_messages_read, stable_timestamp(last_commit_time) as last_commit_time_, num_commits, stable_timestamp(last_rebalance_time) as last_rebalance_time_, num_rebalance_revocations, num_rebalance_assignments, is_currently_used From a38459b407ee1db50214f5660a83d2a5dbf93e4a Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 1 Aug 2023 15:57:06 +0000 Subject: [PATCH 379/777] system_kafka_consumers: black formatter --- .../integration/test_kafka_bad_messages/test.py | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_kafka_bad_messages/test.py b/tests/integration/test_kafka_bad_messages/test.py index 92b4af94e2d..73e31f8c8ca 100644 --- a/tests/integration/test_kafka_bad_messages/test.py +++ b/tests/integration/test_kafka_bad_messages/test.py @@ -90,7 +90,9 @@ def producer_serializer(x): return x.encode() if isinstance(x, str) else x -def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=15, partition=None): +def kafka_produce( + kafka_cluster, topic, messages, timestamp=None, retries=15, partition=None +): logging.debug( "kafka_produce server:{}:{} topic:{}".format( "localhost", kafka_cluster.kafka_port, topic @@ -100,7 +102,9 @@ def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=15, pa kafka_cluster.kafka_port, producer_serializer, retries ) for message in messages: - producer.send(topic=topic, value=message, timestamp_ms=timestamp, partition=partition) + producer.send( + topic=topic, value=message, timestamp_ms=timestamp, partition=partition + ) producer.flush() @@ -232,7 +236,9 @@ message Message { SELECT last_exception, last_exception_time, database, table FROM system.kafka_consumers """ ) - logging.debug(f"result_system_kafka_consumers (test_bad_messages_parsing 2): {result_system_kafka_consumers}") + logging.debug( + f"result_system_kafka_consumers (test_bad_messages_parsing 2): {result_system_kafka_consumers}" + ) kafka_delete_topic(admin_client, f"{format_name}_err") @@ -285,6 +291,7 @@ struct Message kafka_delete_topic(admin_client, "CapnProto_err") + def test_bad_messages_parsing_exception(kafka_cluster, max_retries=20): admin_client = KafkaAdminClient( bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port) @@ -333,7 +340,7 @@ Cannot parse input: expected \\'{\\' before: \\'asdfghjkl\\': while parsing Kafk SELECT exceptions.text[1], length(exceptions.text) > 1 AND length(exceptions.text) < 15, length(exceptions.time) > 1 AND length(exceptions.time) < 15, abs(dateDiff('second', exceptions.time[1], now())) < 40, database, table FROM system.kafka_consumers ORDER BY table, assignments.partition_id[1] """ ) - result_system_kafka_consumers = result_system_kafka_consumers.replace('\t', '|') + result_system_kafka_consumers = result_system_kafka_consumers.replace("\t", "|") if result_system_kafka_consumers == expected_result or retries > max_retries: break retries += 1 @@ -347,6 +354,7 @@ Cannot parse input: expected \\'{\\' before: \\'asdfghjkl\\': while parsing Kafk ]: kafka_delete_topic(admin_client, f"{format_name}_err") + if __name__ == "__main__": cluster.start() input("Cluster created, press any key to destroy...") From bccf42c08381eb77470969ac789c21bf2ac11ce9 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 3 Aug 2023 07:19:24 +0000 Subject: [PATCH 380/777] system_kafka_consumers: some legacy forgotten --- tests/integration/test_kafka_bad_messages/test.py | 9 --------- 1 file changed, 9 deletions(-) diff --git a/tests/integration/test_kafka_bad_messages/test.py b/tests/integration/test_kafka_bad_messages/test.py index 73e31f8c8ca..adb8efc0809 100644 --- a/tests/integration/test_kafka_bad_messages/test.py +++ b/tests/integration/test_kafka_bad_messages/test.py @@ -231,15 +231,6 @@ message Message { assert rows == len(messages) - result_system_kafka_consumers = instance.query( - """ - SELECT last_exception, last_exception_time, database, table FROM system.kafka_consumers - """ - ) - logging.debug( - f"result_system_kafka_consumers (test_bad_messages_parsing 2): {result_system_kafka_consumers}" - ) - kafka_delete_topic(admin_client, f"{format_name}_err") capn_proto_schema = """ From 3bced2f08c2973dfa413112a5767357644753b21 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sat, 5 Aug 2023 22:19:49 +0000 Subject: [PATCH 381/777] system_kafka_consumers: tidy fix, one partition for stable test --- src/Storages/System/StorageSystemKafkaConsumers.cpp | 8 +++----- tests/integration/test_kafka_bad_messages/test.py | 10 +++------- 2 files changed, 6 insertions(+), 12 deletions(-) diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 213b7b326ac..663ae670b81 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -124,12 +124,10 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context assigments_partition_id_offsets.push_back(last_assignment_num); assigments_current_offset_offsets.push_back(last_assignment_num); - for (auto excit = consumer_stat.exceptions_buffer.cbegin(); - excit != consumer_stat.exceptions_buffer.cend(); - ++excit) + for (const auto & exc : consumer_stat.exceptions_buffer) { - exceptions_text.insertData(excit->text.data(), excit->text.size()); - exceptions_time.insert(excit->timestamp_usec); + exceptions_text.insertData(exc.text.data(), exc.text.size()); + exceptions_time.insert(exc.timestamp_usec); } exceptions_num += consumer_stat.exceptions_buffer.size(); exceptions_text_offset.push_back(exceptions_num); diff --git a/tests/integration/test_kafka_bad_messages/test.py b/tests/integration/test_kafka_bad_messages/test.py index adb8efc0809..6494ff8dad4 100644 --- a/tests/integration/test_kafka_bad_messages/test.py +++ b/tests/integration/test_kafka_bad_messages/test.py @@ -294,7 +294,7 @@ def test_bad_messages_parsing_exception(kafka_cluster, max_retries=20): ]: print(format_name) - kafka_create_topic(admin_client, f"{format_name}_err", num_partitions=2) + kafka_create_topic(admin_client, f"{format_name}_err") instance.query( f""" @@ -307,21 +307,17 @@ def test_bad_messages_parsing_exception(kafka_cluster, max_retries=20): kafka_topic_list = '{format_name}_err', kafka_group_name = '{format_name}', kafka_format = '{format_name}', - kafka_num_consumers = 2; + kafka_num_consumers = 1; CREATE MATERIALIZED VIEW view_{format_name} Engine=Log AS SELECT * FROM kafka_{format_name}; """ ) - kafka_produce(kafka_cluster, f"{format_name}_err", ["qwertyuiop"], partition=0) - kafka_produce(kafka_cluster, f"{format_name}_err", ["asdfghjkl"], partition=1) - kafka_produce(kafka_cluster, f"{format_name}_err", ["zxcvbnm"], partition=0) + kafka_produce(kafka_cluster, f"{format_name}_err", ["qwertyuiop", "asdfghjkl", "zxcvbnm"]) expected_result = """avro::Exception: Invalid data file. Magic does not match: : while parsing Kafka message (topic: Avro_err, partition: 0, offset: 0)\\'|1|1|1|default|kafka_Avro -avro::Exception: Invalid data file. Magic does not match: : while parsing Kafka message (topic: Avro_err, partition: 1, offset: 0)\\'|1|1|1|default|kafka_Avro Cannot parse input: expected \\'{\\' before: \\'qwertyuiop\\': while parsing Kafka message (topic: JSONEachRow_err, partition: 0, offset: 0)\\'|1|1|1|default|kafka_JSONEachRow -Cannot parse input: expected \\'{\\' before: \\'asdfghjkl\\': while parsing Kafka message (topic: JSONEachRow_err, partition: 1, offset: 0)\\'|1|1|1|default|kafka_JSONEachRow """ retries = 0 result_system_kafka_consumers = "" From 4a8ca0ffaf5dac012896942e053bf179ec61ded8 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sat, 5 Aug 2023 22:44:26 +0000 Subject: [PATCH 382/777] system_kafka_consumers: black formatter again --- tests/integration/test_kafka_bad_messages/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_kafka_bad_messages/test.py b/tests/integration/test_kafka_bad_messages/test.py index 6494ff8dad4..b6204fe741c 100644 --- a/tests/integration/test_kafka_bad_messages/test.py +++ b/tests/integration/test_kafka_bad_messages/test.py @@ -314,7 +314,9 @@ def test_bad_messages_parsing_exception(kafka_cluster, max_retries=20): """ ) - kafka_produce(kafka_cluster, f"{format_name}_err", ["qwertyuiop", "asdfghjkl", "zxcvbnm"]) + kafka_produce( + kafka_cluster, f"{format_name}_err", ["qwertyuiop", "asdfghjkl", "zxcvbnm"] + ) expected_result = """avro::Exception: Invalid data file. Magic does not match: : while parsing Kafka message (topic: Avro_err, partition: 0, offset: 0)\\'|1|1|1|default|kafka_Avro Cannot parse input: expected \\'{\\' before: \\'qwertyuiop\\': while parsing Kafka message (topic: JSONEachRow_err, partition: 0, offset: 0)\\'|1|1|1|default|kafka_JSONEachRow From 7d7fbc90490ea4cbfbde38150b7a59e6ce94fe55 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 7 Aug 2023 08:29:10 +0000 Subject: [PATCH 383/777] system_kafka_consumers: avoid collision in test --- tests/integration/test_kafka_bad_messages/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_kafka_bad_messages/test.py b/tests/integration/test_kafka_bad_messages/test.py index b6204fe741c..a634ce36631 100644 --- a/tests/integration/test_kafka_bad_messages/test.py +++ b/tests/integration/test_kafka_bad_messages/test.py @@ -300,6 +300,7 @@ def test_bad_messages_parsing_exception(kafka_cluster, max_retries=20): f""" DROP TABLE IF EXISTS view_{format_name}; DROP TABLE IF EXISTS kafka_{format_name}; + DROP TABLE IF EXISTS kafka; CREATE TABLE kafka_{format_name} (key UInt64, value UInt64) ENGINE = Kafka From ade9c3d9707544784efc4a8cf6a0e74431d850f9 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 10 Aug 2023 22:27:01 +0000 Subject: [PATCH 384/777] system_kafka_consumers: per consumer librdkafka stat --- src/Storages/Kafka/KafkaConsumer.cpp | 8 ++-- src/Storages/Kafka/KafkaConsumer.h | 13 +++++- src/Storages/Kafka/StorageKafka.cpp | 45 +++++++++++++------ src/Storages/Kafka/StorageKafka.h | 9 ++-- .../System/StorageSystemKafkaConsumers.cpp | 11 +---- 5 files changed, 55 insertions(+), 31 deletions(-) diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index 5b0cfe06f50..9e558940012 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -559,7 +559,7 @@ void KafkaConsumer::setExceptionInfo(const String & text) * is merged, * because consumer->get_member_id() contains a leak */ -std::string KafkaConsumer::getMemberId() +std::string KafkaConsumer::getMemberId() const { char * memberid_ptr = rd_kafka_memberid(consumer->get_handle()); std::string memberid_string = memberid_ptr; @@ -568,7 +568,7 @@ std::string KafkaConsumer::getMemberId() } -KafkaConsumer::Stat KafkaConsumer::getStat() +KafkaConsumer::Stat KafkaConsumer::getStat() const { KafkaConsumer::Stat::Assignments assignments; auto cpp_assignments = consumer->get_assignment(); @@ -596,7 +596,9 @@ KafkaConsumer::Stat KafkaConsumer::getStat() .num_rebalance_revocations = num_rebalance_revocations.load(), .exceptions_buffer = [&](){std::lock_guard lock(exception_mutex); return exceptions_buffer;}(), - .in_use = in_use.load() + .in_use = in_use.load(), + .rdkafka_stat = [&](){std::lock_guard lock(rdkafka_stat_mutex); + return rdkafka_stat;}(), }; } diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index 64c76e7b837..91bb2ae8d77 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -57,6 +57,7 @@ public: UInt64 num_rebalance_revocations; KafkaConsumer::ExceptionsBuffer exceptions_buffer; bool in_use; + std::string rdkafka_stat; }; public: @@ -106,11 +107,16 @@ public: String currentPayload() const { return current[-1].get_payload(); } void setExceptionInfo(const cppkafka::Error & err); void setExceptionInfo(const String & text); + void setRDKafkaStat(const std::string & stat_json_string) + { + std::lock_guard lock(rdkafka_stat_mutex); + rdkafka_stat = stat_json_string; + } void inUse() { in_use = true; } void notInUse() { in_use = false; } // For system.kafka_consumers - Stat getStat(); + Stat getStat() const; private: using Messages = std::vector; @@ -163,6 +169,9 @@ private: std::atomic num_rebalance_revocations = 0; std::atomic in_use = 0; + mutable std::mutex rdkafka_stat_mutex; + std::string rdkafka_stat; + void drain(); void cleanUnprocessed(); void resetIfStopped(); @@ -170,7 +179,7 @@ private: size_t filterMessageErrors(); ReadBufferPtr getNextMessage(); - std::string getMemberId(); + std::string getMemberId() const; }; } diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 05548dc8696..54db0f29cb8 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -516,7 +516,11 @@ KafkaConsumerPtr StorageKafka::createConsumer(size_t consumer_number) size_t default_queued_min_messages = 100000; // we don't want to decrease the default conf.set("queued.min.messages", std::max(getMaxBlockSize(),default_queued_min_messages)); - updateConfiguration(conf); + /// a reference to the consumer is needed in statistic callback + /// although the consumer does not exist when callback is being registered + /// shared_ptr> comes to the rescue + auto consumer_weak_ptr_ptr = std::make_shared(); + updateConfiguration(conf, consumer_weak_ptr_ptr); // those settings should not be changed by users. conf.set("enable.auto.commit", "false"); // We manually commit offsets after a stream successfully finished @@ -527,13 +531,20 @@ KafkaConsumerPtr StorageKafka::createConsumer(size_t consumer_number) auto consumer_impl = std::make_shared(conf); consumer_impl->set_destroy_flags(RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE); + KafkaConsumerPtr kafka_consumer_ptr; + /// NOTE: we pass |stream_cancelled| by reference here, so the buffers should not outlive the storage. if (thread_per_consumer) { auto& stream_cancelled = tasks[consumer_number]->stream_cancelled; - return std::make_shared(consumer_impl, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), intermediate_commit, stream_cancelled, topics); + kafka_consumer_ptr = std::make_shared(consumer_impl, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), intermediate_commit, stream_cancelled, topics); } - return std::make_shared(consumer_impl, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), intermediate_commit, tasks.back()->stream_cancelled, topics); + else + { + kafka_consumer_ptr = std::make_shared(consumer_impl, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), intermediate_commit, tasks.back()->stream_cancelled, topics); + } + *consumer_weak_ptr_ptr = kafka_consumer_ptr; + return kafka_consumer_ptr; } size_t StorageKafka::getMaxBlockSize() const @@ -566,7 +577,8 @@ String StorageKafka::getConfigPrefix() const return CONFIG_KAFKA_TAG; } -void StorageKafka::updateConfiguration(cppkafka::Configuration & kafka_config) +void StorageKafka::updateConfiguration(cppkafka::Configuration & kafka_config, + std::shared_ptr kafka_consumer_weak_ptr_ptr) { // Update consumer configuration from the configuration. Example: // @@ -646,17 +658,24 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & kafka_config) LOG_IMPL(log, client_logs_level, poco_level, "[rdk:{}] {}", facility, message); }); - if (!config.has(config_prefix + "." + "statistics_interval_ms")) + if (kafka_consumer_weak_ptr_ptr) { - kafka_config.set("statistics.interval.ms", "3000"); // every 3 seconds by default. set to 0 to disable. - } - - if (kafka_config.get("statistics.interval.ms") != "0") - { - kafka_config.set_stats_callback([this](cppkafka::KafkaHandleBase &, const std::string & stat_json_string) + if (!config.has(config_prefix + "." + "statistics_interval_ms")) { - rdkafka_stat = std::make_shared(stat_json_string); - }); + kafka_config.set("statistics.interval.ms", "3000"); // every 3 seconds by default. set to 0 to disable. + } + + if (kafka_config.get("statistics.interval.ms") != "0") + { + kafka_config.set_stats_callback([kafka_consumer_weak_ptr_ptr](cppkafka::KafkaHandleBase &, const std::string & stat_json_string) + { + auto kafka_consumer_ptr = kafka_consumer_weak_ptr_ptr->lock(); + if (kafka_consumer_ptr) + { + kafka_consumer_ptr->setRDKafkaStat(stat_json_string); + } + }); + } } // Configure interceptor to change thread name diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 415e03d11b3..77e1370c2b7 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -79,7 +79,6 @@ public: NamesAndTypesList getVirtuals() const override; Names getVirtualColumnNames() const; HandleKafkaErrorMode getHandleKafkaErrorMode() const { return kafka_settings->kafka_handle_error_mode; } - std::shared_ptr getRdkafkaStat() const { return rdkafka_stat; } struct SafeConsumers { @@ -143,7 +142,12 @@ private: std::atomic shutdown_called = false; // Update Kafka configuration with values from CH user configuration. - void updateConfiguration(cppkafka::Configuration & kafka_config); + void updateConfiguration(cppkafka::Configuration & kafka_config, std::shared_ptr); + void updateConfiguration(cppkafka::Configuration & kafka_config) + { + updateConfiguration(kafka_config, std::make_shared()); + } + String getConfigPrefix() const; void threadFunc(size_t idx); @@ -157,7 +161,6 @@ private: bool streamToViews(); bool checkDependencies(const StorageID & table_id); - std::shared_ptr rdkafka_stat; }; } diff --git a/src/Storages/System/StorageSystemKafkaConsumers.cpp b/src/Storages/System/StorageSystemKafkaConsumers.cpp index 663ae670b81..eb7d84603c0 100644 --- a/src/Storages/System/StorageSystemKafkaConsumers.cpp +++ b/src/Storages/System/StorageSystemKafkaConsumers.cpp @@ -145,16 +145,7 @@ void StorageSystemKafkaConsumers::fillData(MutableColumns & res_columns, Context is_currently_used.insert(consumer_stat.in_use); - auto stat_string_ptr = storage_kafka_ptr->getRdkafkaStat(); - if (stat_string_ptr) - { - rdkafka_stat.insertData(stat_string_ptr->data(), stat_string_ptr->size()); - } - else - { - const std::string empty_stat = "{}"; - rdkafka_stat.insertData(empty_stat.data(), empty_stat.size()); - } + rdkafka_stat.insertData(consumer_stat.rdkafka_stat.data(), consumer_stat.rdkafka_stat.size()); } } }; From ea8ae28c7dfeae6c3641c8617152750aa0d94be2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 00:40:18 +0200 Subject: [PATCH 385/777] Fix build with clang-17 --- src/Core/examples/coro.cpp | 194 ------------------------------------- 1 file changed, 194 deletions(-) delete mode 100644 src/Core/examples/coro.cpp diff --git a/src/Core/examples/coro.cpp b/src/Core/examples/coro.cpp deleted file mode 100644 index fbccc261e9d..00000000000 --- a/src/Core/examples/coro.cpp +++ /dev/null @@ -1,194 +0,0 @@ -#include -#include -#include -#include - -#include -#include -#include -#include -#include - -#if defined(__clang__) -#include - -namespace std // NOLINT(cert-dcl58-cpp) -{ - using namespace experimental::coroutines_v1; // NOLINT(cert-dcl58-cpp) -} - -#if __has_warning("-Wdeprecated-experimental-coroutine") -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wdeprecated-experimental-coroutine" -#endif - -#else -#include -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wzero-as-null-pointer-constant" -#endif - - -template -struct suspend_value // NOLINT(readability-identifier-naming) -{ - constexpr bool await_ready() const noexcept { return true; } // NOLINT(readability-identifier-naming) - constexpr void await_suspend(std::coroutine_handle<>) const noexcept {} // NOLINT(readability-identifier-naming) - constexpr T await_resume() const noexcept // NOLINT(readability-identifier-naming) - { - std::cout << " ret " << val << std::endl; - return val; - } - - T val; -}; - -template -struct Task -{ - struct promise_type // NOLINT(readability-identifier-naming) - { - using coro_handle = std::coroutine_handle; - auto get_return_object() { return coro_handle::from_promise(*this); } // NOLINT(readability-identifier-naming) - auto initial_suspend() { return std::suspend_never(); } // NOLINT(readability-identifier-naming) - auto final_suspend() noexcept { return suspend_value{*r->value}; } // NOLINT(readability-identifier-naming) - //void return_void() {} - void return_value(T value_) { r->value = value_; } // NOLINT(readability-identifier-naming) - void unhandled_exception() // NOLINT(readability-identifier-naming) - { - DB::tryLogCurrentException("Logger"); - r->exception = std::current_exception(); // NOLINT(bugprone-throw-keyword-missing) - } - - explicit promise_type(std::string tag_) : tag(tag_) {} - ~promise_type() { std::cout << "~promise_type " << tag << std::endl; } - std::string tag; - coro_handle next; - Task * r = nullptr; - }; - - using coro_handle = std::coroutine_handle; - - bool await_ready() const noexcept { return false; } // NOLINT(readability-identifier-naming) - void await_suspend(coro_handle g) noexcept // NOLINT(readability-identifier-naming) - { - std::cout << " await_suspend " << my.promise().tag << std::endl; - std::cout << " g tag " << g.promise().tag << std::endl; - g.promise().next = my; - } - T await_resume() noexcept // NOLINT(readability-identifier-naming) - { - std::cout << " await_res " << my.promise().tag << std::endl; - return *value; - } - - Task(coro_handle handle) : my(handle), tag(handle.promise().tag) // NOLINT(google-explicit-constructor) - { - assert(handle); - my.promise().r = this; - std::cout << " Task " << tag << std::endl; - } - Task(Task &) = delete; - Task(Task &&rhs) noexcept : my(rhs.my), tag(rhs.tag) - { - rhs.my = {}; - std::cout << " Task&& " << tag << std::endl; - } - static bool resumeImpl(Task *r) - { - if (r->value) - return false; - - auto & next = r->my.promise().next; - - if (next) - { - if (resumeImpl(next.promise().r)) - return true; - next = {}; - } - - if (!r->value) - { - r->my.resume(); - if (r->exception) - std::rethrow_exception(r->exception); - } - return !r->value; - } - - bool resume() - { - return resumeImpl(this); - } - - T res() - { - return *value; - } - - ~Task() - { - std::cout << " ~Task " << tag << std::endl; - } - -private: - coro_handle my; - std::string tag; - std::optional value; - std::exception_ptr exception; -}; - -Task boo([[maybe_unused]] std::string tag) -{ - std::cout << "x" << std::endl; - co_await std::suspend_always(); - std::cout << StackTrace().toString(); - std::cout << "y" << std::endl; - co_return 1; -} - -Task bar([[maybe_unused]] std::string tag) -{ - std::cout << "a" << std::endl; - int res1 = co_await boo("boo1"); - std::cout << "b " << res1 << std::endl; - int res2 = co_await boo("boo2"); - if (res2 == 1) - throw DB::Exception(1, "hello"); - std::cout << "c " << res2 << std::endl; - co_return res1 + res2; // 1 + 1 = 2 -} - -Task foo([[maybe_unused]] std::string tag) -{ - std::cout << "Hello" << std::endl; - auto res1 = co_await bar("bar1"); - std::cout << "Coro " << res1 << std::endl; - auto res2 = co_await bar("bar2"); - std::cout << "World " << res2 << std::endl; - co_return res1 * res2; // 2 * 2 = 4 -} - -int main() -{ - Poco::AutoPtr app_channel(new Poco::ConsoleChannel(std::cerr)); - Poco::Logger::root().setChannel(app_channel); - Poco::Logger::root().setLevel("trace"); - - LOG_INFO(&Poco::Logger::get(""), "Starting"); - - try - { - auto t = foo("foo"); - std::cout << ".. started" << std::endl; - while (t.resume()) - std::cout << ".. yielded" << std::endl; - std::cout << ".. done: " << t.res() << std::endl; - } - catch (DB::Exception & e) - { - std::cout << "Got exception " << e.what() << std::endl; - std::cout << e.getStackTraceString() << std::endl; - } -} From ea62629d83ee487e64b1ba3c051211103f258069 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 00:40:26 +0200 Subject: [PATCH 386/777] Fix linking of examples --- src/Core/examples/CMakeLists.txt | 3 --- src/Parsers/examples/CMakeLists.txt | 4 ++-- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Core/examples/CMakeLists.txt b/src/Core/examples/CMakeLists.txt index 868173e0e31..2326eada96d 100644 --- a/src/Core/examples/CMakeLists.txt +++ b/src/Core/examples/CMakeLists.txt @@ -9,6 +9,3 @@ target_link_libraries (string_ref_hash PRIVATE clickhouse_common_io) clickhouse_add_executable (mysql_protocol mysql_protocol.cpp) target_link_libraries (mysql_protocol PRIVATE dbms) - -clickhouse_add_executable (coro coro.cpp) -target_link_libraries (coro PRIVATE clickhouse_common_io) diff --git a/src/Parsers/examples/CMakeLists.txt b/src/Parsers/examples/CMakeLists.txt index e411574bd65..261f234081c 100644 --- a/src/Parsers/examples/CMakeLists.txt +++ b/src/Parsers/examples/CMakeLists.txt @@ -4,7 +4,7 @@ clickhouse_add_executable(lexer lexer.cpp ${SRCS}) target_link_libraries(lexer PRIVATE clickhouse_parsers) clickhouse_add_executable(select_parser select_parser.cpp ${SRCS} "../../Server/ServerType.cpp") -target_link_libraries(select_parser PRIVATE clickhouse_parsers) +target_link_libraries(select_parser PRIVATE dbms) clickhouse_add_executable(create_parser create_parser.cpp ${SRCS} "../../Server/ServerType.cpp") -target_link_libraries(create_parser PRIVATE clickhouse_parsers) +target_link_libraries(create_parser PRIVATE dbms) From 3193c6d4ccc7ef02857f14825e1a781e8c2e83c2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 00:55:57 +0200 Subject: [PATCH 387/777] Fix build --- src/Functions/array/arrayAUC.cpp | 2 +- src/Server/ServerType.cpp | 2 +- src/Server/ServerType.h | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/array/arrayAUC.cpp b/src/Functions/array/arrayAUC.cpp index b7bd7dcc0ad..499fe4ce7b2 100644 --- a/src/Functions/array/arrayAUC.cpp +++ b/src/Functions/array/arrayAUC.cpp @@ -139,7 +139,7 @@ private: for (size_t i = 0; i < size; ++i) { auto next_offset = offsets[i]; - result[i] = apply(data1, data2, current_offset, next_offset); + result[i] = apply(scores, labels, current_offset, next_offset); current_offset = next_offset; } } diff --git a/src/Server/ServerType.cpp b/src/Server/ServerType.cpp index 4952cd1bd24..29ba7224c75 100644 --- a/src/Server/ServerType.cpp +++ b/src/Server/ServerType.cpp @@ -2,10 +2,10 @@ #include #include -#include #include + namespace DB { diff --git a/src/Server/ServerType.h b/src/Server/ServerType.h index 1fab492222a..eafe4f941dd 100644 --- a/src/Server/ServerType.h +++ b/src/Server/ServerType.h @@ -1,13 +1,13 @@ #pragma once #include + namespace DB { class ServerType { public: - enum Type { TCP, From 396db7ab3cee77955173de76f355e1d08c1ec755 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 01:01:43 +0200 Subject: [PATCH 388/777] Fix test `00002_log_and_exception_messages_formatting` --- .../00002_log_and_exception_messages_formatting.sql | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index 86fe01dc0e3..54daeb928a3 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -36,7 +36,9 @@ create temporary table known_short_messages (s String) as select * from (select 'Database {} doesn''t exist', 'Dictionary ({}) not found', 'Unknown table function {}', 'Unknown format {}', 'Unknown explain kind ''{}''', 'Unknown setting {}', 'Unknown input format {}', 'Unknown identifier: ''{}''', 'User name is empty', 'Expected function, got: {}', -'Attempt to read after eof', 'String size is too big ({}), maximum: {}', 'API mode: {}' +'Attempt to read after eof', 'String size is too big ({}), maximum: {}', 'API mode: {}', +'Processed: {}%', 'Creating {}: {}', 'Table {}.{} doesn''t exist', 'Invalid cache key hex: {}', +'User has been dropped', 'Illegal type {} of argument of function {}. Should be DateTime or DateTime64' ] as arr) array join arr; -- Check that we don't have too many short meaningless message patterns. From c98e38611fccca5acfaf466a7ce22e9ae2bc0a6a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Aug 2023 23:36:47 +0000 Subject: [PATCH 389/777] path to libc --- cmake/linux/toolchain-s390x.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-s390x.cmake b/cmake/linux/toolchain-s390x.cmake index 03314a9d2ba..945eb9affa4 100644 --- a/cmake/linux/toolchain-s390x.cmake +++ b/cmake/linux/toolchain-s390x.cmake @@ -20,9 +20,9 @@ set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/s390x-linux-gnu/libc") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") -set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold -Wl,-L/usr/lib/x86_64-linux-gnu") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L${CMAKE_SYSROOT}/usr/lib64") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} -fuse-ld=mold -Wl,-L${CMAKE_SYSROOT}/usr/lib64") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -fuse-ld=mold -Wl,-L${CMAKE_SYSROOT}/usr/lib64") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From 3b9d2c708096e51b106acffbfd3cc2c4ac8606e0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 11 Aug 2023 03:06:07 +0200 Subject: [PATCH 390/777] Simplification --- .../config.d/create_logs_at_startup.yaml | 1 - programs/server/config.xml | 3 - src/Common/SystemLogBase.h | 2 +- src/Interpreters/Context.cpp | 16 ++++ src/Interpreters/Context.h | 3 + src/Interpreters/InterpreterSystemQuery.cpp | 82 ++++++++----------- src/Interpreters/SystemLog.cpp | 14 +--- src/Interpreters/SystemLog.h | 5 +- .../config.d/create_logs_at_startup.yaml | 34 -------- tests/config/install.sh | 1 - 10 files changed, 57 insertions(+), 104 deletions(-) delete mode 120000 programs/server/config.d/create_logs_at_startup.yaml delete mode 100644 tests/config/config.d/create_logs_at_startup.yaml diff --git a/programs/server/config.d/create_logs_at_startup.yaml b/programs/server/config.d/create_logs_at_startup.yaml deleted file mode 120000 index a09d2783a9e..00000000000 --- a/programs/server/config.d/create_logs_at_startup.yaml +++ /dev/null @@ -1 +0,0 @@ -../../../tests/config/config.d/create_logs_at_startup.yaml \ No newline at end of file diff --git a/programs/server/config.xml b/programs/server/config.xml index 3d1c92b135c..14b8954fc39 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1037,9 +1037,6 @@ - - - false + And so, to avoid extra memory reference switch *_log to Memory engine. + --> ENGINE = Memory From e357702fd05b77ed01e43a9be38e1e6dfff393a8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Aug 2023 01:26:38 +0200 Subject: [PATCH 501/777] What will happen if I remove this? --- .../zzz-perf-comparison-tweaks-config.xml | 26 ------------------- 1 file changed, 26 deletions(-) diff --git a/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml b/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml index 10a5916264a..292665c4f68 100644 --- a/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml +++ b/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml @@ -19,32 +19,6 @@ - - - ENGINE = Memory - - - - ENGINE = Memory - - - - ENGINE = Memory - - - - ENGINE = Memory - - - - ENGINE = Memory - - - 1000000000 10 From fea74ce17b45ae38336253f06608d6412e98417c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Kriszti=C3=A1n=20Sz=C5=B1cs?= Date: Sun, 13 Aug 2023 17:02:08 +0200 Subject: [PATCH 502/777] Documentation: add Ibis project to the integrations section --- docs/en/interfaces/third-party/integrations.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/third-party/integrations.md b/docs/en/interfaces/third-party/integrations.md index 3e1b1e84f5d..a9f1af93495 100644 --- a/docs/en/interfaces/third-party/integrations.md +++ b/docs/en/interfaces/third-party/integrations.md @@ -83,8 +83,8 @@ ClickHouse, Inc. does **not** maintain the tools and libraries listed below and - Python - [SQLAlchemy](https://www.sqlalchemy.org) - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) (uses [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)) - - [pandas](https://pandas.pydata.org) - - [pandahouse](https://github.com/kszucs/pandahouse) + - [PyArrow/Pandas](https://pandas.pydata.org) + - [Ibis](https://github.com/ibis-project/ibis) - PHP - [Doctrine](https://www.doctrine-project.org/) - [dbal-clickhouse](https://packagist.org/packages/friendsofdoctrine/dbal-clickhouse) From 7135b344bfcfaf22bba21b716962de9303f6409a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Kriszti=C3=A1n=20Sz=C5=B1cs?= Date: Mon, 14 Aug 2023 08:40:49 +0200 Subject: [PATCH 503/777] Documentation: exclude PyArrow from spell checking --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 812908931ef..2e231120e41 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1892,7 +1892,6 @@ overfitting packetpool packetsize pageviews -pandahouse parallelization parallelize parallelized @@ -2001,6 +2000,7 @@ ptrs pushdown pwrite py +PyArrow qryn quantile quantileBFloat From 385332a5542997a14e71ae8e2b34cd3b4247d553 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 07:10:50 +0000 Subject: [PATCH 504/777] Docs: Update anchors in ANN indexes docs --- .../en/engines/table-engines/mergetree-family/annindexes.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 5944048f6c3..6618c6ddc06 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -1,4 +1,4 @@ -# Approximate Nearest Neighbor Search Indexes [experimental] {#table_engines-ANNIndex} +# Approximate Nearest Neighbor Search Indexes [experimental] Nearest neighborhood search is the problem of finding the M closest points for a given point in an N-dimensional vector space. The most straightforward approach to solve this problem is a brute force search where the distance between all points in the vector space and the @@ -45,7 +45,7 @@ With brute force search, both queries are expensive (linear in the number of poi `Point` must be computed. To speed this process up, Approximate Nearest Neighbor Search Indexes (ANN indexes) store a compact representation of the search space (using clustering, search trees, etc.) which allows to compute an approximate answer much quicker (in sub-linear time). -# Creating and Using ANN Indexes +# Creating and Using ANN Indexes {#creating_using_ann_indexes} Syntax to create an ANN index over an [Array](../../../sql-reference/data-types/array.md) column: @@ -138,7 +138,7 @@ back to a smaller `GRANULARITY` values only in case of problems like excessive m was specified for ANN indexes, the default value is 100 million. -# Available ANN Indexes +# Available ANN Indexes {#available_ann_indexes} - [Annoy](/docs/en/engines/table-engines/mergetree-family/annindexes.md#annoy-annoy) From 0b1f6faec231e34cb3da0bcd114c5469f22f06c4 Mon Sep 17 00:00:00 2001 From: Pradeep Chhetri Date: Mon, 14 Aug 2023 15:13:26 +0800 Subject: [PATCH 505/777] Add clickhouse-keeper-converter symlink to clickhouse-keeper package --- packages/clickhouse-keeper.yaml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/packages/clickhouse-keeper.yaml b/packages/clickhouse-keeper.yaml index 4dcdf0697cd..3b77d968763 100644 --- a/packages/clickhouse-keeper.yaml +++ b/packages/clickhouse-keeper.yaml @@ -35,6 +35,9 @@ contents: - src: clickhouse dst: /usr/bin/clickhouse-keeper-client type: symlink +- src: clickhouse + dst: /usr/bin/clickhouse-keeper-converter + type: symlink # docs - src: ../AUTHORS dst: /usr/share/doc/clickhouse-keeper/AUTHORS From f71ce2641c09cab4d70e24d867f5014b86edecef Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 07:36:27 +0000 Subject: [PATCH 506/777] Fix copyright issues in ANN docs --- .../mergetree-family/annindexes.md | 24 +++++++------------ 1 file changed, 9 insertions(+), 15 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 6618c6ddc06..156f64e94d4 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -188,23 +188,17 @@ ENGINE = MergeTree ORDER BY id; ``` -Annoy currently supports `L2Distance` and `cosineDistance` as distance function `Distance`. If no distance function was specified during -index creation, `L2Distance` is used as default. Parameter `NumTrees` is the number of trees which the algorithm creates (default if not -specified: 100). Higher values of `NumTree` mean more accurate search results but slower index creation / query times (approximately -linearly) as well as larger index sizes. +Annoy currently supports two distance functions: +- `L2Distance`, also called Euclidean distance is the length of a line segment between two points in Euclidean space + ([Wikipedia](https://en.wikipedia.org/wiki/Euclidean_distance)). +- `cosineDistance`, also called cosine similarity, is the cosine of the angle between two (non-zero) vectors + ([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)). -`L2Distance` is also called Euclidean distance, the Euclidean distance between two points in Euclidean space is the length of a line segment between the two points. -For example: If we have point P(p1,p2), Q(q1,q2), their distance will be d(p,q) -![L2Distance](https://en.wikipedia.org/wiki/Euclidean_distance#/media/File:Euclidean_distance_2d.svg) +For normalized data, `L2Distance` is usually a better choice, otherwise `cosineDistance` is recommended to compensate for scale. If no +distance function was specified during index creation, `L2Distance` is used as default. -`cosineDistance` also called cosine similarity is a measure of similarity between two non-zero vectors defined in an inner product space. Cosine similarity is the cosine of the angle between the vectors; that is, it is the dot product of the vectors divided by the product of their lengths. -![cosineDistance](https://www.tyrrell4innovation.ca/wp-content/uploads/2021/06/rsz_jenny_du_miword.png) - -The Euclidean distance corresponds to the L2-norm of a difference between vectors. The cosine similarity is proportional to the dot product of two vectors and inversely proportional to the product of their magnitudes. -![compare](https://www.researchgate.net/publication/320914786/figure/fig2/AS:558221849841664@1510101868614/The-difference-between-Euclidean-distance-and-cosine-similarity.png) -In one sentence: cosine similarity care only about the angle between them, but do not care about the "distance" we normally think. -![L2 distance](https://www.baeldung.com/wp-content/uploads/sites/4/2020/06/4-1.png) -![cosineDistance](https://www.baeldung.com/wp-content/uploads/sites/4/2020/06/5.png) +Parameter `NumTrees` is the number of trees which the algorithm creates (default if not specified: 100). Higher values of `NumTree` mean +more accurate search results but slower index creation / query times (approximately linearly) as well as larger index sizes. :::note Indexes over columns of type `Array` will generally work faster than indexes on `Tuple` columns. All arrays **must** have same length. Use From 1c3f4d3719d9171f4bbe1aee1a7c7109ddb1ce59 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 07:46:15 +0000 Subject: [PATCH 507/777] + , --- docs/en/engines/table-engines/mergetree-family/annindexes.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 156f64e94d4..9c9067099c9 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -189,7 +189,7 @@ ORDER BY id; ``` Annoy currently supports two distance functions: -- `L2Distance`, also called Euclidean distance is the length of a line segment between two points in Euclidean space +- `L2Distance`, also called Euclidean distance, is the length of a line segment between two points in Euclidean space ([Wikipedia](https://en.wikipedia.org/wiki/Euclidean_distance)). - `cosineDistance`, also called cosine similarity, is the cosine of the angle between two (non-zero) vectors ([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)). From 5a180b53ed4e1d79820cd3e4ced5d094b9a5675f Mon Sep 17 00:00:00 2001 From: Filipp Ozinov Date: Mon, 14 Aug 2023 12:30:29 +0400 Subject: [PATCH 508/777] Style fix --- src/Storages/StorageMaterializedMySQL.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMaterializedMySQL.h b/src/Storages/StorageMaterializedMySQL.h index f16fa8ffa78..2cd589bfd75 100644 --- a/src/Storages/StorageMaterializedMySQL.h +++ b/src/Storages/StorageMaterializedMySQL.h @@ -43,7 +43,8 @@ public: bool supportsTrivialCountOptimization() const override { return false; } - IndexSizeByName getSecondaryIndexSizes() const override { + IndexSizeByName getSecondaryIndexSizes() const override + { return nested_storage->getSecondaryIndexSizes(); } From 2d3bf55d454880104044804b1142ce1feeeb43ac Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 08:50:20 +0000 Subject: [PATCH 509/777] Docs: Update table name in ANN docs --- .../mergetree-family/annindexes.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 9c9067099c9..81c69215472 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -17,7 +17,7 @@ In terms of SQL, the nearest neighborhood problem can be expressed as follows: ``` sql SELECT * -FROM table +FROM table_with_ann_index ORDER BY Distance(vectors, Point) LIMIT N ``` @@ -32,7 +32,7 @@ An alternative formulation of the nearest neighborhood search problem looks as f ``` sql SELECT * -FROM table +FROM table_with_ann_index WHERE Distance(vectors, Point) < MaxDistance LIMIT N ``` @@ -50,7 +50,7 @@ of the search space (using clustering, search trees, etc.) which allows to compu Syntax to create an ANN index over an [Array](../../../sql-reference/data-types/array.md) column: ```sql -CREATE TABLE table +CREATE TABLE table_with_ann_index ( `id` Int64, `vectors` Array(Float32), @@ -63,7 +63,7 @@ ORDER BY id; Syntax to create an ANN index over a [Tuple](../../../sql-reference/data-types/tuple.md) column: ```sql -CREATE TABLE table +CREATE TABLE table_with_ann_index ( `id` Int64, `vectors` Tuple(Float32[, Float32[, ...]]), @@ -83,7 +83,7 @@ ANN indexes support two types of queries: ``` sql SELECT * - FROM table + FROM table_with_ann_index [WHERE ...] ORDER BY Distance(vectors, Point) LIMIT N @@ -93,7 +93,7 @@ ANN indexes support two types of queries: ``` sql SELECT * - FROM table + FROM table_with_ann_index WHERE Distance(vectors, Point) < MaxDistance LIMIT N ``` @@ -103,7 +103,7 @@ To avoid writing out large vectors, you can use [query parameters](/docs/en/interfaces/cli.md#queries-with-parameters-cli-queries-with-parameters), e.g. ```bash -clickhouse-client --param_vec='hello' --query="SELECT * FROM table WHERE L2Distance(vectors, {vec: Array(Float32)}) < 1.0" +clickhouse-client --param_vec='hello' --query="SELECT * FROM table_with_ann_index WHERE L2Distance(vectors, {vec: Array(Float32)}) < 1.0" ``` ::: @@ -165,7 +165,7 @@ space in random linear surfaces (lines in 2D, planes in 3D etc.). Syntax to create an Annoy index over an [Array](../../../sql-reference/data-types/array.md) column: ```sql -CREATE TABLE table +CREATE TABLE table_with_annoy_index ( id Int64, vectors Array(Float32), @@ -178,7 +178,7 @@ ORDER BY id; Syntax to create an ANN index over a [Tuple](../../../sql-reference/data-types/tuple.md) column: ```sql -CREATE TABLE table +CREATE TABLE table_with_annoy_index ( id Int64, vectors Tuple(Float32[, Float32[, ...]]), From 57bef64fbc3bb1d8bcc56a7104599ab90473723b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 14 Aug 2023 17:25:58 +0800 Subject: [PATCH 510/777] add new native orc files --- .../Impl/NativeORCBlockInputFormat.cpp | 1086 +++++++++++++++++ .../Formats/Impl/NativeORCBlockInputFormat.h | 120 ++ 2 files changed, 1206 insertions(+) create mode 100644 src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp create mode 100644 src/Processors/Formats/Impl/NativeORCBlockInputFormat.h diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp new file mode 100644 index 00000000000..54105e80ed4 --- /dev/null +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -0,0 +1,1086 @@ +#include "NativeORCBlockInputFormat.h" + +#if USE_ORC +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int UNKNOWN_TYPE; + extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; + extern const int BAD_ARGUMENTS; + extern const int THERE_IS_NO_COLUMN; +} + +ORCInputStream::ORCInputStream(SeekableReadBuffer & in_) : in(in_) +{ +} + +uint64_t ORCInputStream::getLength() const +{ + return getFileSizeFromReadBuffer(in); +} + +uint64_t ORCInputStream::getNaturalReadSize() const +{ + return 128 * 1024; +} + +void ORCInputStream::read(void * buf, uint64_t length, uint64_t offset) +{ + if (offset != static_cast(in.getPosition())) + in.seek(offset, SEEK_SET); + + in.readStrict(reinterpret_cast(buf), length); +} + +std::unique_ptr asORCInputStream(ReadBuffer & in) +{ + bool has_file_size = isBufferWithFileSize(in); + if (!has_file_size) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "ORC format supports only input with known size"); + + auto * seekable_in = dynamic_cast(&in); + if (!seekable_in) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "ORC format supports only seekable input"); + + return std::make_unique(*seekable_in); +} + +static DataTypePtr parseORCType(const orc::Type * orc_type) +{ + assert(orc_type != nullptr); + + const int subtype_count = static_cast(orc_type->getSubtypeCount()); + switch (orc_type->getKind()) + { + case orc::TypeKind::BOOLEAN: + return DataTypeFactory::instance().get("Bool"); + case orc::TypeKind::BYTE: + return std::make_shared(); + case orc::TypeKind::SHORT: + return std::make_shared(); + case orc::TypeKind::INT: + return std::make_shared(); + case orc::TypeKind::LONG: + return std::make_shared(); + case orc::TypeKind::FLOAT: + return std::make_shared(); + case orc::TypeKind::DOUBLE: + return std::make_shared(); + case orc::TypeKind::DATE: + return std::make_shared(); + case orc::TypeKind::TIMESTAMP: + return std::make_shared(9); + case orc::TypeKind::VARCHAR: + case orc::TypeKind::BINARY: + case orc::TypeKind::STRING: + return std::make_shared(); + case orc::TypeKind::CHAR: + return std::make_shared(orc_type->getMaximumLength()); + case orc::TypeKind::DECIMAL: { + UInt64 precision = orc_type->getPrecision(); + UInt64 scale = orc_type->getScale(); + if (precision == 0) + { + // In HIVE 0.11/0.12 precision is set as 0, but means max precision + return createDecimal(38, 6); + } + else + return createDecimal(precision, scale); + } + case orc::TypeKind::LIST: { + if (subtype_count != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid Orc List type {}", orc_type->toString()); + + DataTypePtr nested_type = parseORCType(orc_type->getSubtype(0)); + return std::make_shared(nested_type); + } + case orc::TypeKind::MAP: { + if (subtype_count != 2) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid Orc Map type {}", orc_type->toString()); + + DataTypePtr key_type = parseORCType(orc_type->getSubtype(0)); + DataTypePtr value_type = parseORCType(orc_type->getSubtype(1)); + return std::make_shared(key_type, value_type); + } + case orc::TypeKind::STRUCT: { + DataTypes nested_types; + Strings nested_names; + nested_types.reserve(subtype_count); + nested_names.reserve(subtype_count); + + for (size_t i = 0; i < orc_type->getSubtypeCount(); ++i) + { + nested_types.push_back(parseORCType(orc_type->getSubtype(i))); + nested_names.push_back(orc_type->getFieldName(i)); + } + return std::make_shared(nested_types, nested_names); + } + default: + throw Exception(ErrorCodes::UNKNOWN_TYPE, "ORC type {} is not supported", orc_type->toString()); + } +} + + +static void getFileReaderAndSchema( + ReadBuffer & in, + std::unique_ptr & file_reader, + Block & header, + const FormatSettings & /*format_settings*/, + std::atomic & is_stopped) +{ + if (is_stopped) + return; + + orc::ReaderOptions options; + auto input_stream = asORCInputStream(in); + file_reader = orc::createReader(std::move(input_stream), options); + const auto & schema = file_reader->getType(); + + for (size_t i = 0; i < schema.getSubtypeCount(); ++i) + { + const std::string & name = schema.getFieldName(i); + const orc::Type * orc_type = schema.getSubtype(i); + DataTypePtr type = parseORCType(orc_type); + header.insert(ColumnWithTypeAndName{type, name}); + } +} + +ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_) + : IInputFormat(std::move(header_), &in_), format_settings(format_settings_), skip_stripes(format_settings.orc.skip_stripes) +{ +} + +void ORCBlockInputFormat::prepareFileReader() +{ + Block schema; + getFileReaderAndSchema(*in, file_reader, schema, format_settings, is_stopped); + if (is_stopped) + return; + + total_stripes = static_cast(file_reader->getNumberOfStripes()); + current_stripe = -1; + + orc_column_to_ch_column = std::make_unique( + getPort().getHeader(), + format_settings.orc.import_nested, + format_settings.orc.allow_missing_columns, + format_settings.null_as_default, + format_settings.orc.case_insensitive_column_matching); + + const bool ignore_case = format_settings.orc.case_insensitive_column_matching; + std::unordered_set nested_table_names; + if (format_settings.orc.import_nested) + nested_table_names = Nested::getAllTableNames(getPort().getHeader(), ignore_case); + + for (size_t i = 0; i < schema.columns(); ++i) + { + const auto & name = schema.getByPosition(i).name; + if (getPort().getHeader().has(name, ignore_case) || nested_table_names.contains(ignore_case ? boost::to_lower_copy(name) : name)) + include_indices.push_back(static_cast(i)); + } + + // std::cout << "schema:" << schema.dumpStructure() << std::endl; + // std::cout << "header:" << getPort().getHeader().dumpStructure() << std::endl; +} + +bool ORCBlockInputFormat::prepareStripeReader() +{ + assert(file_reader); + + ++current_stripe; + for (; current_stripe < total_stripes && skip_stripes.contains(current_stripe); ++current_stripe) + ; + + /// No more stripes to read + if (current_stripe >= total_stripes) + return false; + + /// Seek to current stripe + current_stripe_info = file_reader->getStripe(current_stripe); + if (!current_stripe_info->getNumberOfRows()) + return false; + + orc::RowReaderOptions row_reader_options; + row_reader_options.include(include_indices); + row_reader_options.range(current_stripe_info->getOffset(), current_stripe_info->getLength()); + stripe_reader = file_reader->createRowReader(row_reader_options); + + if (!batch) + batch = stripe_reader->createRowBatch(format_settings.orc.row_batch_size); + + return true; +} + +Chunk ORCBlockInputFormat::generate() +{ + block_missing_values.clear(); + + if (!file_reader) + prepareFileReader(); + + if (!stripe_reader) + { + if (!prepareStripeReader()) + return {}; + } + + if (is_stopped) + return {}; + + while (true) + { + bool ok = stripe_reader->next(*batch); + if (ok) + break; + + /// No more rows to read in current stripe, continue to prepare reading next stripe + if (!prepareStripeReader()) + return {}; + } + + Chunk res; + size_t num_rows = batch->numElements; + const auto & schema = stripe_reader->getSelectedType(); + orc_column_to_ch_column->orcTableToCHChunk(res, &schema, batch.get(), num_rows, &block_missing_values); + + approx_bytes_read_for_chunk = num_rows * current_stripe_info->getLength() / current_stripe_info->getNumberOfRows(); + return res; +} + +void ORCBlockInputFormat::resetParser() +{ + IInputFormat::resetParser(); + + file_reader.reset(); + stripe_reader.reset(); + include_indices.clear(); + block_missing_values.clear(); +} + +const BlockMissingValues & ORCBlockInputFormat::getMissingValues() const +{ + return block_missing_values; +} + +ORCSchemaReader::ORCSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) + : ISchemaReader(in_), format_settings(format_settings_) +{ +} + +NamesAndTypesList ORCSchemaReader::readSchema() +{ + Block header; + std::unique_ptr file_reader; + std::atomic is_stopped = 0; + getFileReaderAndSchema(in, file_reader, header, format_settings, is_stopped); + + if (format_settings.schema_inference_make_columns_nullable) + return getNamesAndRecursivelyNullableTypes(header); + return header.getNamesAndTypesList(); +} + + +ORCColumnToCHColumn::ORCColumnToCHColumn( + const Block & header_, bool import_nested_, bool allow_missing_columns_, bool null_as_default_, bool case_insensitive_matching_) + : header(header_) + , import_nested(import_nested_) + , allow_missing_columns(allow_missing_columns_) + , null_as_default(null_as_default_) + , case_insensitive_matching(case_insensitive_matching_) +{ +} + +void ORCColumnToCHColumn::orcTableToCHChunk( + Chunk & res, const orc::Type * schema, const orc::ColumnVectorBatch * table, size_t num_rows, BlockMissingValues * block_missing_values) +{ + const auto * struct_batch = dynamic_cast(table); + if (!struct_batch) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ORC table must be StructVectorBatch but is {}", struct_batch->toString()); + + if (schema->getSubtypeCount() != struct_batch->fields.size()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "ORC table has {} fields but schema has {}", struct_batch->fields.size(), schema->getSubtypeCount()); + + size_t field_num = struct_batch->fields.size(); + NameToColumnPtr name_to_column_ptr; + for (size_t i = 0; i < field_num; ++i) + { + auto name = schema->getFieldName(i); + const auto * field = struct_batch->fields[i]; + if (!field) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ORC table field {} is null", name); + + if (case_insensitive_matching) + boost::to_lower(name); + + name_to_column_ptr[std::move(name)] = {field, schema->getSubtype(i)}; + } + + orcColumnsToCHChunk(res, name_to_column_ptr, num_rows, block_missing_values); +} + +/// Creates a null bytemap from arrow's null bitmap +static ColumnPtr readByteMapFromORCColumn(const orc::ColumnVectorBatch * orc_column) +{ + if (!orc_column->hasNulls) + return ColumnUInt8::create(orc_column->numElements, 0); + + auto nullmap_column = ColumnUInt8::create(); + PaddedPODArray & bytemap_data = assert_cast &>(*nullmap_column).getData(); + bytemap_data.resize(orc_column->numElements); + + for (size_t i = 0; i < orc_column->numElements; ++i) + bytemap_data[i] = 1 - orc_column->notNull[i]; + return nullmap_column; +} + + +static const orc::ColumnVectorBatch * getNestedORCColumn(const orc::ListVectorBatch * orc_column) +{ + return orc_column->elements.get(); +} + +template +static ColumnPtr readOffsetsFromORCListColumn(const BatchType * orc_column) +{ + auto offsets_column = ColumnUInt64::create(); + ColumnArray::Offsets & offsets_data = assert_cast &>(*offsets_column).getData(); + offsets_data.reserve(orc_column->numElements); + + for (size_t i = 0; i < orc_column->numElements; ++i) + offsets_data.push_back(orc_column->offsets[i + 1]); + + return offsets_column; +} + +/// Inserts numeric data right into internal column data to reduce an overhead +template > +static ColumnWithTypeAndName +readColumnWithNumericData(const orc::ColumnVectorBatch * orc_column, const orc::Type *, const String & column_name) +{ + auto internal_type = std::make_shared>(); + auto internal_column = internal_type->createColumn(); + auto & column_data = static_cast(*internal_column).getData(); + column_data.reserve(orc_column->numElements); + + const auto * orc_int_column = typeid_cast(orc_column); + column_data.insert_assume_reserved(orc_int_column->data.data(), orc_int_column->data.data() + orc_int_column->numElements); + + return {std::move(internal_column), std::move(internal_type), column_name}; +} + +/// Inserts numeric data right into internal column data to reduce an overhead +template > +static ColumnWithTypeAndName +readColumnWithNumericDataCast(const orc::ColumnVectorBatch * orc_column, const orc::Type *, const String & column_name) +{ + auto internal_type = std::make_shared>(); + auto internal_column = internal_type->createColumn(); + auto & column_data = static_cast(*internal_column).getData(); + column_data.reserve(orc_column->numElements); + + const auto * orc_int_column = typeid_cast(orc_column); + for (size_t i = 0; i < orc_int_column->numElements; ++i) + column_data.push_back(static_cast(orc_int_column->data[i])); + + return {std::move(internal_column), std::move(internal_type), column_name}; +} + +/// Inserts chars and offsets right into internal column data to reduce an overhead. +static ColumnWithTypeAndName +readColumnWithStringData(const orc::ColumnVectorBatch * orc_column, const orc::Type *, const String & column_name) +{ + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + PaddedPODArray & column_chars_t = assert_cast(*internal_column).getChars(); + PaddedPODArray & column_offsets = assert_cast(*internal_column).getOffsets(); + + const auto * orc_str_column = dynamic_cast(orc_column); + size_t reserver_size = 0; + for (size_t i = 0; i < orc_str_column->numElements; ++i) + reserver_size += orc_str_column->length[i] + 1; + column_chars_t.reserve(reserver_size); + column_offsets.reserve(orc_str_column->numElements); + + size_t curr_offset = 0; + for (size_t i = 0; i < orc_str_column->numElements; ++i) + { + const auto * buf = orc_str_column->data[i]; + if (buf) + { + size_t buf_size = orc_str_column->length[i]; + column_chars_t.insert_assume_reserved(buf, buf + buf_size); + curr_offset += buf_size; + } + + column_chars_t.push_back(0); + ++curr_offset; + + column_offsets.push_back(curr_offset); + } + return {std::move(internal_column), std::move(internal_type), column_name}; +} + +static ColumnWithTypeAndName +readColumnWithFixedStringData(const orc::ColumnVectorBatch * orc_column, const orc::Type * orc_type, const String & column_name) +{ + size_t fixed_len = orc_type->getMaximumLength(); + auto internal_type = std::make_shared(fixed_len); + auto internal_column = internal_type->createColumn(); + PaddedPODArray & column_chars_t = assert_cast(*internal_column).getChars(); + column_chars_t.reserve(orc_column->numElements * fixed_len); + + const auto * orc_str_column = dynamic_cast(orc_column); + for (size_t i = 0; i < orc_str_column->numElements; ++i) + { + if (orc_str_column->data[i]) + column_chars_t.insert_assume_reserved(orc_str_column->data[i], orc_str_column->data[i] + orc_str_column->length[i]); + else + column_chars_t.resize(column_chars_t.size() + fixed_len); + } + + return {std::move(internal_column), std::move(internal_type), column_name}; +} + + +/// Inserts decimal data right into internal column data to reduce an overhead +template > +static ColumnWithTypeAndName readColumnWithDecimalDataCast( + const orc::ColumnVectorBatch * orc_column, const orc::Type *, const String & column_name, DataTypePtr internal_type) +{ + using NativeType = typename DecimalType::NativeType; + static_assert(std::is_same_v || std::is_same_v); + + auto internal_column = internal_type->createColumn(); + auto & column_data = static_cast(*internal_column).getData(); + column_data.reserve(orc_column->numElements); + + const auto * orc_decimal_column = typeid_cast(orc_column); + for (size_t i = 0; i < orc_decimal_column->numElements; ++i) + { + DecimalType decimal_value; + if constexpr (std::is_same_v) + { + Int128 int128_value; + int128_value.items[0] = orc_decimal_column->values[i].getLowBits(); + int128_value.items[1] = orc_decimal_column->values[i].getHighBits(); + decimal_value.value = static_cast(int128_value); + } + else + decimal_value.value = static_cast(orc_decimal_column->values[i]); + + column_data.push_back(std::move(decimal_value)); + // std::cout << "i:" << i << "size:" << column_data.size() << "size2:" << internal_column->size() + // << ", value:" << static_cast(column_data.back().value) << std::endl; + } + // std::cout << "orc rows:" << orc_decimal_column->numElements << std::endl; + // std::cout << "ch rows:" << internal_column->size() << std::endl; + + return {std::move(internal_column), internal_type, column_name}; +} + + +static ColumnWithTypeAndName +readIPv6ColumnFromBinaryData(const orc::ColumnVectorBatch * orc_column, const orc::Type * orc_type, const String & column_name) +{ + const auto * orc_str_column = dynamic_cast(orc_column); + + for (size_t i = 0; i < orc_str_column->numElements; ++i) + { + /// If at least one value size is not 16 bytes, fallback to reading String column and further cast to IPv6. + if (orc_str_column->data[i] && orc_str_column->length[i] != sizeof(IPv6)) + return readColumnWithStringData(orc_column, orc_type, column_name); + } + + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + auto & ipv6_column = assert_cast(*internal_column); + ipv6_column.reserve(orc_str_column->numElements); + + for (size_t i = 0; i < orc_str_column->numElements; ++i) + { + if (!orc_str_column->data[i]) [[unlikely]] + ipv6_column.insertDefault(); + else + ipv6_column.insertData(orc_str_column->data[i], orc_str_column->length[i]); + } + + return {std::move(internal_column), std::move(internal_type), column_name}; +} + + +static ColumnWithTypeAndName +readIPv4ColumnWithInt32Data(const orc::ColumnVectorBatch * orc_column, const orc::Type *, const String & column_name) +{ + const auto * orc_int_column = dynamic_cast(orc_column); + + auto internal_type = std::make_shared(); + auto internal_column = internal_type->createColumn(); + auto & column_data = assert_cast(*internal_column).getData(); + column_data.reserve(orc_int_column->numElements); + + for (size_t i = 0; i < orc_int_column->numElements; ++i) + column_data.push_back(static_cast(orc_int_column->data[i])); + + return {std::move(internal_column), std::move(internal_type), column_name}; +} + +template +static ColumnWithTypeAndName readColumnWithBigNumberFromBinaryData( + const orc::ColumnVectorBatch * orc_column, const orc::Type *, const String & column_name, const DataTypePtr & column_type) +{ + const auto * orc_str_column = dynamic_cast(orc_column); + + auto internal_column = column_type->createColumn(); + auto & integer_column = assert_cast(*internal_column); + integer_column.reserve(orc_str_column->numElements); + + for (size_t i = 0; i < orc_str_column->numElements; ++i) + { + if (!orc_str_column->data[i]) [[unlikely]] + integer_column.insertDefault(); + else + integer_column.insertData(orc_str_column->data[i], orc_str_column->length[i]); + } + return {std::move(internal_column), column_type, column_name}; +} + + +static ColumnWithTypeAndName readColumnWithDateData( + const orc::ColumnVectorBatch * orc_column, const orc::Type *, const String & column_name, const DataTypePtr & type_hint) +{ + DataTypePtr internal_type; + bool check_date_range = false; + /// Make result type Date32 when requested type is actually Date32 or when we use schema inference + if (!type_hint || (type_hint && isDate32(*type_hint))) + { + internal_type = std::make_shared(); + check_date_range = true; + } + else + { + internal_type = std::make_shared(); + } + + const auto * orc_int_column = dynamic_cast(orc_column); + auto internal_column = internal_type->createColumn(); + PaddedPODArray & column_data = assert_cast &>(*internal_column).getData(); + column_data.reserve(orc_int_column->numElements); + + for (size_t i = 0; i < orc_int_column->numElements; ++i) + { + Int32 days_num = static_cast(orc_int_column->data[i]); + if (check_date_range && (days_num > DATE_LUT_MAX_EXTEND_DAY_NUM || days_num < -DAYNUM_OFFSET_EPOCH)) + throw Exception( + ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE, + "Input value {} of a column \"{}\" exceeds the range of type Date32", + days_num, + column_name); + + column_data.push_back(days_num); + } + + return {std::move(internal_column), internal_type, column_name}; +} + +static ColumnWithTypeAndName +readColumnWithTimestampData(const orc::ColumnVectorBatch * orc_column, const orc::Type *, const String & column_name) +{ + const auto * orc_ts_column = dynamic_cast(orc_column); + + auto internal_type = std::make_shared(9); + auto internal_column = internal_type->createColumn(); + auto & column_data = assert_cast(*internal_column).getData(); + column_data.reserve(orc_ts_column->numElements); + + constexpr Int64 multiplier = 1e9L; + for (size_t i = 0; i < orc_ts_column->numElements; ++i) + { + Decimal64 decimal64; + decimal64.value = orc_ts_column->data[i] * multiplier + orc_ts_column->nanoseconds[i]; + column_data.emplace_back(std::move(decimal64)); + } + return {std::move(internal_column), std::move(internal_type), column_name}; +} + + +static ColumnWithTypeAndName readColumnFromORCColumn( + const orc::ColumnVectorBatch * orc_column, + const orc::Type * orc_type, + const std::string & column_name, + bool inside_nullable, + bool allow_null_type, + bool skip_columns_with_unsupported_types, + bool & skipped, + DataTypePtr type_hint = nullptr) +{ + if (!inside_nullable && (orc_column->hasNulls || (type_hint && type_hint->isNullable()))) + { + DataTypePtr nested_type_hint; + if (type_hint) + nested_type_hint = removeNullable(type_hint); + + auto nested_column = readColumnFromORCColumn( + orc_column, orc_type, column_name, true, allow_null_type, skip_columns_with_unsupported_types, skipped, nested_type_hint); + + if (skipped) + return {}; + + auto nullmap_column = readByteMapFromORCColumn(orc_column); + auto nullable_type = std::make_shared(std::move(nested_column.type)); + auto nullable_column = ColumnNullable::create(nested_column.column, nullmap_column); + return {std::move(nullable_column), std::move(nullable_type), column_name}; + } + + switch (orc_type->getKind()) + { + case orc::STRING: + case orc::BINARY: + case orc::VARCHAR: { + if (type_hint) + { + switch (type_hint->getTypeId()) + { + case TypeIndex::IPv6: + return readIPv6ColumnFromBinaryData(orc_column, orc_type, column_name); + /// ORC format outputs big integers as binary column, because there is no fixed binary in ORC. + case TypeIndex::Int128: + return readColumnWithBigNumberFromBinaryData(orc_column, orc_type, column_name, type_hint); + case TypeIndex::UInt128: + return readColumnWithBigNumberFromBinaryData(orc_column, orc_type, column_name, type_hint); + case TypeIndex::Int256: + return readColumnWithBigNumberFromBinaryData(orc_column, orc_type, column_name, type_hint); + case TypeIndex::UInt256: + return readColumnWithBigNumberFromBinaryData(orc_column, orc_type, column_name, type_hint); + /// ORC doesn't support Decimal256 as separate type. We read and write it as binary data. + case TypeIndex::Decimal256: + return readColumnWithBigNumberFromBinaryData>( + orc_column, orc_type, column_name, type_hint); + default:; + } + } + return readColumnWithStringData(orc_column, orc_type, column_name); + } + case orc::CHAR: { + if (type_hint) + { + switch (type_hint->getTypeId()) + { + case TypeIndex::Int128: + return readColumnWithBigNumberFromBinaryData(orc_column, orc_type, column_name, type_hint); + case TypeIndex::UInt128: + return readColumnWithBigNumberFromBinaryData(orc_column, orc_type, column_name, type_hint); + case TypeIndex::Int256: + return readColumnWithBigNumberFromBinaryData(orc_column, orc_type, column_name, type_hint); + case TypeIndex::UInt256: + return readColumnWithBigNumberFromBinaryData(orc_column, orc_type, column_name, type_hint); + default:; + } + } + return readColumnWithFixedStringData(orc_column, orc_type, column_name); + } + case orc::BYTE: + return readColumnWithNumericDataCast(orc_column, orc_type, column_name); + case orc::SHORT: + return readColumnWithNumericDataCast(orc_column, orc_type, column_name); + case orc::INT: { + /// ORC format doesn't have unsigned integers and we output IPv4 as Int32. + /// We should allow to read it back from Int32. + if (type_hint && isIPv4(type_hint)) + return readIPv4ColumnWithInt32Data(orc_column, orc_type, column_name); + return readColumnWithNumericDataCast(orc_column, orc_type, column_name); + } + case orc::LONG: + return readColumnWithNumericData(orc_column, orc_type, column_name); + case orc::FLOAT: + return readColumnWithNumericDataCast(orc_column, orc_type, column_name); + case orc::DOUBLE: + return readColumnWithNumericData(orc_column, orc_type, column_name); + case orc::DATE: + return readColumnWithDateData(orc_column, orc_type, column_name, type_hint); + case orc::TIMESTAMP: + return readColumnWithTimestampData(orc_column, orc_type, column_name); + case orc::DECIMAL: { + auto interal_type = parseORCType(orc_type); + + auto precision = orc_type->getPrecision(); + if (precision == 0) + precision = 38; + + if (precision > 18) + { + if (precision <= DecimalUtils::max_precision) + return readColumnWithDecimalDataCast( + orc_column, orc_type, column_name, interal_type); + else if (precision <= DecimalUtils::max_precision) + return readColumnWithDecimalDataCast( + orc_column, orc_type, column_name, interal_type); + else if (precision <= DecimalUtils::max_precision) + return readColumnWithDecimalDataCast( + orc_column, orc_type, column_name, interal_type); + else + return readColumnWithDecimalDataCast( + orc_column, orc_type, column_name, interal_type); + } + else + { + if (precision <= DecimalUtils::max_precision) + return readColumnWithDecimalDataCast( + orc_column, orc_type, column_name, interal_type); + else if (precision <= DecimalUtils::max_precision) + return readColumnWithDecimalDataCast( + orc_column, orc_type, column_name, interal_type); + else if (precision <= DecimalUtils::max_precision) + return readColumnWithDecimalDataCast( + orc_column, orc_type, column_name, interal_type); + else + return readColumnWithDecimalDataCast( + orc_column, orc_type, column_name, interal_type); + } + } + case orc::MAP: { + DataTypePtr key_type_hint; + DataTypePtr value_type_hint; + if (type_hint) + { + const auto * map_type_hint = typeid_cast(type_hint.get()); + if (map_type_hint) + { + key_type_hint = map_type_hint->getKeyType(); + value_type_hint = map_type_hint->getValueType(); + } + } + + const auto * orc_map_column = dynamic_cast(orc_column); + const auto * orc_key_column = orc_map_column->keys.get(); + const auto * orc_value_column = orc_map_column->elements.get(); + const auto * orc_key_type = orc_type->getSubtype(0); + const auto * orc_value_type = orc_type->getSubtype(1); + + auto key_column = readColumnFromORCColumn( + orc_key_column, orc_key_type, "key", false, allow_null_type, skip_columns_with_unsupported_types, skipped, key_type_hint); + if (skipped) + return {}; + + if (key_type_hint && !key_type_hint->equals(*key_column.type)) + { + /// Cast key column to target type, because it can happen + /// that parsed type cannot be ClickHouse Map key type. + key_column.column = castColumn(key_column, key_type_hint); + key_column.type = key_type_hint; + } + + auto value_column = readColumnFromORCColumn( + orc_value_column, + orc_value_type, + "value", + false, + allow_null_type, + skip_columns_with_unsupported_types, + skipped, + value_type_hint); + if (skipped) + return {}; + + if (value_type_hint && !value_type_hint->equals(*value_column.type)) + { + /// Cast value column to target type, because it can happen + /// that parsed type cannot be ClickHouse Map value type. + value_column.column = castColumn(value_column, value_type_hint); + value_column.type = value_type_hint; + } + + auto offsets_column = readOffsetsFromORCListColumn(orc_map_column); + auto map_column = ColumnMap::create(key_column.column, value_column.column, offsets_column); + auto map_type = std::make_shared(key_column.type, value_column.type); + return {std::move(map_column), std::move(map_type), column_name}; + } + case orc::LIST: { + DataTypePtr nested_type_hint; + if (type_hint) + { + const auto * array_type_hint = typeid_cast(type_hint.get()); + if (array_type_hint) + nested_type_hint = array_type_hint->getNestedType(); + } + + const auto * orc_list_column = dynamic_cast(orc_column); + const auto * orc_nested_column = getNestedORCColumn(orc_list_column); + const auto * orc_nested_type = orc_type->getSubtype(0); + auto nested_column = readColumnFromORCColumn( + orc_nested_column, + orc_nested_type, + column_name, + false, + allow_null_type, + skip_columns_with_unsupported_types, + skipped, + nested_type_hint); + + if (skipped) + return {}; + + auto offsets_column = readOffsetsFromORCListColumn(orc_list_column); + auto array_column = ColumnArray::create(nested_column.column, offsets_column); + auto array_type = std::make_shared(nested_column.type); + return {std::move(array_column), std::move(array_type), column_name}; + } + case orc::STRUCT: { + Columns tuple_elements; + DataTypes tuple_types; + std::vector tuple_names; + const auto * tuple_type_hint = type_hint ? typeid_cast(type_hint.get()) : nullptr; + + const auto * orc_struct_column = dynamic_cast(orc_column); + for (size_t i = 0; i < orc_type->getSubtypeCount(); ++i) + { + const auto & field_name = orc_type->getFieldName(i); + + DataTypePtr nested_type_hint; + if (tuple_type_hint) + { + if (tuple_type_hint->haveExplicitNames()) + { + auto pos = tuple_type_hint->tryGetPositionByName(field_name); + if (pos) + nested_type_hint = tuple_type_hint->getElement(*pos); + } + else if (size_t(i) < tuple_type_hint->getElements().size()) + nested_type_hint = tuple_type_hint->getElement(i); + } + + const auto * nested_orc_column = orc_struct_column->fields[i]; + const auto * nested_orc_type = orc_type->getSubtype(i); + auto element = readColumnFromORCColumn( + nested_orc_column, + nested_orc_type, + field_name, + false, + allow_null_type, + skip_columns_with_unsupported_types, + skipped, + nested_type_hint); + + if (skipped) + return {}; + + tuple_elements.emplace_back(std::move(element.column)); + tuple_types.emplace_back(std::move(element.type)); + tuple_names.emplace_back(std::move(element.name)); + } + + auto tuple_column = ColumnTuple::create(std::move(tuple_elements)); + auto tuple_type = std::make_shared(std::move(tuple_types), std::move(tuple_names)); + return {std::move(tuple_column), std::move(tuple_type), column_name}; + } + default: { + if (skip_columns_with_unsupported_types) + { + skipped = true; + return {}; + } + + throw Exception( + ErrorCodes::UNKNOWN_TYPE, + "Unsupported ORC type '{}' of an input column '{}'. " + "If it happens during schema inference and you want to skip columns with " + "unsupported types, you can enable setting input_format_orc" + "_skip_columns_with_unsupported_types_in_schema_inference", + orc_type->toString(), + column_name); + } + } +} + +void ORCColumnToCHColumn::orcColumnsToCHChunk( + Chunk & res, NameToColumnPtr & name_to_column_ptr, size_t num_rows, BlockMissingValues * block_missing_values) +{ + Columns columns_list; + columns_list.reserve(header.columns()); + std::unordered_map>> nested_tables; + bool skipped = false; + for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i) + { + const ColumnWithTypeAndName & header_column = header.getByPosition(column_i); + + auto search_column_name = header_column.name; + if (case_insensitive_matching) + boost::to_lower(search_column_name); + + ColumnWithTypeAndName column; + if (!name_to_column_ptr.contains(search_column_name)) + { + bool read_from_nested = false; + + /// Check if it's a column from nested table. + if (import_nested) + { + String nested_table_name = Nested::extractTableName(header_column.name); + String search_nested_table_name = nested_table_name; + if (case_insensitive_matching) + boost::to_lower(search_nested_table_name); + if (name_to_column_ptr.contains(search_nested_table_name)) + { + if (!nested_tables.contains(search_nested_table_name)) + { + NamesAndTypesList nested_columns; + for (const auto & name_and_type : header.getNamesAndTypesList()) + { + if (name_and_type.name.starts_with(nested_table_name + ".")) + nested_columns.push_back(name_and_type); + } + auto nested_table_type = Nested::collect(nested_columns).front().type; + + auto orc_column_with_type = name_to_column_ptr[search_nested_table_name]; + ColumnsWithTypeAndName cols = {readColumnFromORCColumn( + orc_column_with_type.first, + orc_column_with_type.second, + nested_table_name, + false, + true, + false, + skipped, + nested_table_type)}; + BlockPtr block_ptr = std::make_shared(cols); + auto column_extractor = std::make_shared(*block_ptr, case_insensitive_matching); + nested_tables[search_nested_table_name] = {block_ptr, column_extractor}; + } + + auto nested_column = nested_tables[search_nested_table_name].second->extractColumn(search_column_name); + if (nested_column) + { + column = *nested_column; + if (case_insensitive_matching) + column.name = header_column.name; + read_from_nested = true; + } + } + } + + if (!read_from_nested) + { + if (!allow_missing_columns) + throw Exception{ErrorCodes::THERE_IS_NO_COLUMN, "Column '{}' is not presented in input data.", header_column.name}; + else + { + column.name = header_column.name; + column.type = header_column.type; + column.column = header_column.column->cloneResized(num_rows); + columns_list.push_back(std::move(column.column)); + if (block_missing_values) + block_missing_values->setBits(column_i, num_rows); + continue; + } + } + } + else + { + auto orc_column_with_type = name_to_column_ptr[search_column_name]; + column = readColumnFromORCColumn( + orc_column_with_type.first, + orc_column_with_type.second, + header_column.name, + false, + true, + false, + skipped, + header_column.type); + } + + if (null_as_default) + insertNullAsDefaultIfNeeded(column, header_column, column_i, block_missing_values); + + try + { + column.column = castColumn(column, header_column.type); + } + catch (Exception & e) + { + e.addMessage(fmt::format( + "while converting column {} from type {} to type {}", + backQuote(header_column.name), + column.type->getName(), + header_column.type->getName())); + throw; + } + + column.type = header_column.type; + columns_list.push_back(std::move(column.column)); + } + + res.setColumns(columns_list, num_rows); +} + + +void registerInputFormatORC(FormatFactory & factory) +{ + factory.registerInputFormat( + "ORC", + [](ReadBuffer & buf, const Block & sample, const RowInputFormatParams &, const FormatSettings & settings) + { return std::make_shared(buf, sample, settings); }); + factory.markFormatSupportsSubcolumns("ORC"); + factory.markFormatSupportsSubsetOfColumns("ORC"); +} + +void registerORCSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader( + "ORC", + [](ReadBuffer & buf, const FormatSettings & settings) { return std::make_shared(buf, settings); }); + + factory.registerAdditionalInfoForSchemaCacheGetter( + "ORC", + [](const FormatSettings & settings) + { return fmt::format("schema_inference_make_columns_nullable={}", settings.schema_inference_make_columns_nullable); }); +} +} + +#else + +namespace DB +{ +class FormatFactory; +void registerInputFormatORC(FormatFactory &) +{ +} + +void registerORCSchemaReader(FormatFactory &) +{ +} +} + + +#endif diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h new file mode 100644 index 00000000000..50104018aee --- /dev/null +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h @@ -0,0 +1,120 @@ +#pragma once +#include "config.h" + +#if USE_ORC +# include +# include +# include +# include + +namespace DB +{ + +class ORCInputStream : public orc::InputStream +{ +public: + explicit ORCInputStream(SeekableReadBuffer & in_); + + uint64_t getLength() const override; + uint64_t getNaturalReadSize() const override; + void read(void * buf, uint64_t length, uint64_t offset) override; + const std::string & getName() const override { return name; } + +private: + SeekableReadBuffer & in; + std::string name = "ORCInputStream"; +}; + +std::unique_ptr asORCInputStream(ReadBuffer & in); + + +class ORCColumnToCHColumn; +class ORCBlockInputFormat : public IInputFormat +{ +public: + ORCBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_); + + String getName() const override { return "ORCBlockInputFormat"; } + + void resetParser() override; + + const BlockMissingValues & getMissingValues() const override; + + size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; } + +protected: + Chunk generate() override; + + void onCancel() override { is_stopped = 1; } + +private: + void prepareFileReader(); + bool prepareStripeReader(); + + std::unique_ptr file_reader; + std::unique_ptr stripe_reader; + std::unique_ptr orc_column_to_ch_column; + std::unique_ptr batch; + + // indices of columns to read from ORC file + std::list include_indices; + + BlockMissingValues block_missing_values; + size_t approx_bytes_read_for_chunk; + + const FormatSettings format_settings; + const std::unordered_set & skip_stripes; + + int total_stripes = 0; + int current_stripe = -1; + std::unique_ptr current_stripe_info; + + std::atomic is_stopped{0}; +}; + +class ORCSchemaReader : public ISchemaReader +{ +public: + ORCSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_); + + NamesAndTypesList readSchema() override; + +private: + const FormatSettings format_settings; +}; + +class ORCColumnToCHColumn +{ +public: + using ORCColumnPtr = const orc::ColumnVectorBatch *; + using ORCTypePtr = const orc::Type *; + using ORCColumnWithType = std::pair; + using NameToColumnPtr = std::unordered_map; + + ORCColumnToCHColumn( + const Block & header_, + bool import_nested_, + bool allow_missing_columns_, + bool null_as_default_, + bool case_insensitive_matching_ = false); + + void orcTableToCHChunk( + Chunk & res, + const orc::Type * schema, + const orc::ColumnVectorBatch * table, + size_t num_rows, + BlockMissingValues * block_missing_values = nullptr); + + void orcColumnsToCHChunk( + Chunk & res, NameToColumnPtr & name_to_column_ptr, size_t num_rows, BlockMissingValues * block_missing_values = nullptr); + +private: + const Block & header; + bool import_nested; + /// If false, throw exception if some columns in header not exists in arrow table. + bool allow_missing_columns; + bool null_as_default; + bool case_insensitive_matching; +}; +} +#endif From 8ee1912de6558ee2f58881565f5f35c148589961 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 14 Aug 2023 17:26:24 +0800 Subject: [PATCH 511/777] remove old orc files --- .../Formats/Impl/ORCBlockInputFormat.cpp | 209 ------------------ .../Formats/Impl/ORCBlockInputFormat.h | 78 ------- 2 files changed, 287 deletions(-) delete mode 100644 src/Processors/Formats/Impl/ORCBlockInputFormat.cpp delete mode 100644 src/Processors/Formats/Impl/ORCBlockInputFormat.h diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp deleted file mode 100644 index 016f07731d5..00000000000 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ /dev/null @@ -1,209 +0,0 @@ -#include "ORCBlockInputFormat.h" -#include -#if USE_ORC - -#include -#include -#include -#include -#include -#include "ArrowBufferedStreams.h" -#include "ArrowColumnToCHColumn.h" -#include "ArrowFieldIndexUtil.h" -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int CANNOT_READ_ALL_DATA; -} - -ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_) - : IInputFormat(std::move(header_), &in_), format_settings(format_settings_), skip_stripes(format_settings.orc.skip_stripes) -{ -} - -Chunk ORCBlockInputFormat::generate() -{ - block_missing_values.clear(); - - if (!file_reader) - prepareReader(); - - if (is_stopped) - return {}; - - for (; stripe_current < stripe_total && skip_stripes.contains(stripe_current); ++stripe_current) - ; - - if (stripe_current >= stripe_total) - return {}; - - auto batch_result = file_reader->ReadStripe(stripe_current, include_indices); - if (!batch_result.ok()) - throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to create batch reader: {}", batch_result.status().ToString()); - - auto batch = batch_result.ValueOrDie(); - if (!batch) - return {}; - - auto table_result = arrow::Table::FromRecordBatches({batch}); - if (!table_result.ok()) - throw ParsingException( - ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading batch of ORC data: {}", table_result.status().ToString()); - - /// We should extract the number of rows directly from the stripe, because in case when - /// record batch contains 0 columns (for example if we requested only columns that - /// are not presented in data) the number of rows in record batch will be 0. - size_t num_rows = file_reader->GetRawORCReader()->getStripe(stripe_current)->getNumberOfRows(); - - auto table = table_result.ValueOrDie(); - if (!table || !num_rows) - return {}; - - approx_bytes_read_for_chunk = file_reader->GetRawORCReader()->getStripe(stripe_current)->getDataLength(); - ++stripe_current; - - Chunk res; - /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. - /// Otherwise fill the missing columns with zero values of its type. - BlockMissingValues * block_missing_values_ptr = format_settings.defaults_for_omitted_fields ? &block_missing_values : nullptr; - arrow_column_to_ch_column->arrowTableToCHChunk(res, table, num_rows, block_missing_values_ptr); - return res; -} - -void ORCBlockInputFormat::resetParser() -{ - IInputFormat::resetParser(); - - file_reader.reset(); - include_indices.clear(); - block_missing_values.clear(); -} - -const BlockMissingValues & ORCBlockInputFormat::getMissingValues() const -{ - return block_missing_values; -} - - -static void getFileReaderAndSchema( - ReadBuffer & in, - std::unique_ptr & file_reader, - std::shared_ptr & schema, - const FormatSettings & format_settings, - std::atomic & is_stopped) -{ - auto arrow_file = asArrowFile(in, format_settings, is_stopped, "ORC", ORC_MAGIC_BYTES); - if (is_stopped) - return; - - auto result = arrow::adapters::orc::ORCFileReader::Open(arrow_file, arrow::default_memory_pool()); - if (!result.ok()) - throw Exception::createDeprecated(result.status().ToString(), ErrorCodes::BAD_ARGUMENTS); - file_reader = std::move(result).ValueOrDie(); - - auto read_schema_result = file_reader->ReadSchema(); - if (!read_schema_result.ok()) - throw Exception::createDeprecated(read_schema_result.status().ToString(), ErrorCodes::BAD_ARGUMENTS); - schema = std::move(read_schema_result).ValueOrDie(); -} - -void ORCBlockInputFormat::prepareReader() -{ - std::shared_ptr schema; - getFileReaderAndSchema(*in, file_reader, schema, format_settings, is_stopped); - if (is_stopped) - return; - - stripe_total = static_cast(file_reader->NumberOfStripes()); - stripe_current = 0; - - arrow_column_to_ch_column = std::make_unique( - getPort().getHeader(), - "ORC", - format_settings.orc.import_nested, - format_settings.orc.allow_missing_columns, - format_settings.null_as_default, - format_settings.orc.case_insensitive_column_matching); - - const bool ignore_case = format_settings.orc.case_insensitive_column_matching; - std::unordered_set nested_table_names; - if (format_settings.orc.import_nested) - nested_table_names = Nested::getAllTableNames(getPort().getHeader(), ignore_case); - - for (int i = 0; i < schema->num_fields(); ++i) - { - const auto & name = schema->field(i)->name(); - if (getPort().getHeader().has(name, ignore_case) || nested_table_names.contains(ignore_case ? boost::to_lower_copy(name) : name)) - include_indices.push_back(i); - } -} - -ORCSchemaReader::ORCSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) - : ISchemaReader(in_), format_settings(format_settings_) -{ -} - -NamesAndTypesList ORCSchemaReader::readSchema() -{ - std::unique_ptr file_reader; - std::shared_ptr schema; - std::atomic is_stopped = 0; - getFileReaderAndSchema(in, file_reader, schema, format_settings, is_stopped); - auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader( - *schema, "ORC", format_settings.orc.skip_columns_with_unsupported_types_in_schema_inference); - if (format_settings.schema_inference_make_columns_nullable) - return getNamesAndRecursivelyNullableTypes(header); - return header.getNamesAndTypesList();} - -void registerInputFormatORC(FormatFactory & factory) -{ - factory.registerInputFormat( - "ORC", - [](ReadBuffer &buf, - const Block &sample, - const RowInputFormatParams &, - const FormatSettings & settings) - { - return std::make_shared(buf, sample, settings); - }); - factory.markFormatSupportsSubcolumns("ORC"); - factory.markFormatSupportsSubsetOfColumns("ORC"); -} - -void registerORCSchemaReader(FormatFactory & factory) -{ - factory.registerSchemaReader( - "ORC", - [](ReadBuffer & buf, const FormatSettings & settings) - { - return std::make_shared(buf, settings); - } - ); - - factory.registerAdditionalInfoForSchemaCacheGetter("ORC", [](const FormatSettings & settings) - { - return fmt::format("schema_inference_make_columns_nullable={}", settings.schema_inference_make_columns_nullable); - }); -} - -} -#else - -namespace DB -{ - class FormatFactory; - void registerInputFormatORC(FormatFactory &) - { - } - - void registerORCSchemaReader(FormatFactory &) - { - } -} - -#endif diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h deleted file mode 100644 index 98561e72e61..00000000000 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ /dev/null @@ -1,78 +0,0 @@ -#pragma once -#include "config.h" -#if USE_ORC - -#include -#include -#include - -#include - -namespace arrow::adapters::orc -{ - class ORCFileReader; -} - -namespace DB -{ - -class ArrowColumnToCHColumn; - -class ORCBlockInputFormat : public IInputFormat -{ -public: - ORCBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_); - - String getName() const override { return "ORCBlockInputFormat"; } - - void resetParser() override; - - const BlockMissingValues & getMissingValues() const override; - - size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; } - -protected: - Chunk generate() override; - - void onCancel() override - { - is_stopped = 1; - } - -private: - void prepareReader(); - - // TODO: check that this class implements every part of its parent - - std::unique_ptr file_reader; - - std::unique_ptr arrow_column_to_ch_column; - - // indices of columns to read from ORC file - std::vector include_indices; - - BlockMissingValues block_missing_values; - size_t approx_bytes_read_for_chunk; - - const FormatSettings format_settings; - const std::unordered_set & skip_stripes; - - int stripe_total = 0; - int stripe_current = 0; - - std::atomic is_stopped{0}; -}; - -class ORCSchemaReader : public ISchemaReader -{ -public: - ORCSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_); - - NamesAndTypesList readSchema() override; - -private: - const FormatSettings format_settings; -}; - -} -#endif From 832768745b6f9f7fa2fff41105cb8bd03ec18e32 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 14 Aug 2023 17:27:01 +0800 Subject: [PATCH 512/777] add examples --- src/Processors/examples/CMakeLists.txt | 6 ++++++ src/Processors/examples/native_orc.cpp | 14 ++++++++++++++ 2 files changed, 20 insertions(+) create mode 100644 src/Processors/examples/native_orc.cpp diff --git a/src/Processors/examples/CMakeLists.txt b/src/Processors/examples/CMakeLists.txt index 5d43a0d7d08..425ef9553d5 100644 --- a/src/Processors/examples/CMakeLists.txt +++ b/src/Processors/examples/CMakeLists.txt @@ -2,3 +2,9 @@ if (TARGET ch_contrib::hivemetastore) clickhouse_add_executable (comma_separated_streams comma_separated_streams.cpp) target_link_libraries (comma_separated_streams PRIVATE dbms) endif() + +if (USE_ORC) + clickhouse_add_executable (native_orc native_orc.cpp) + target_link_libraries (native_orc PRIVATE dbms) + target_include_directories (native_orc PRIVATE ${ClickHouse_SOURCE_DIR}/contrib/orc/c++/include) +endif () \ No newline at end of file diff --git a/src/Processors/examples/native_orc.cpp b/src/Processors/examples/native_orc.cpp new file mode 100644 index 00000000000..2c1715d3ce4 --- /dev/null +++ b/src/Processors/examples/native_orc.cpp @@ -0,0 +1,14 @@ +#include +#include +#include + +using namespace DB; + +int main() +{ + ReadBufferFromFile in("/data1/clickhouse_official/data/user_files/bigolive_audience_stats_orc.orc"); + ORCSchemaReader schema_reader(in, {}); + auto schema = schema_reader.readSchema(); + std::cout << "schema:" << schema.toString() << std::endl; + return 0; +} From 9d5fcbf52466d4268134f0e35fbba212e9d08c87 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 09:16:08 +0000 Subject: [PATCH 513/777] Follow-up to #52695: Move tests to a more appropriate place --- .../test_from_system_tables.py | 12 -------- .../01161_information_schema.reference | 2 ++ .../0_stateless/01161_information_schema.sql | 29 +++++++++++++------ 3 files changed, 22 insertions(+), 21 deletions(-) diff --git a/tests/integration/test_select_access_rights/test_from_system_tables.py b/tests/integration/test_select_access_rights/test_from_system_tables.py index a4dfa662601..3cd6b90109c 100644 --- a/tests/integration/test_select_access_rights/test_from_system_tables.py +++ b/tests/integration/test_select_access_rights/test_from_system_tables.py @@ -190,15 +190,3 @@ def test_information_schema(): ) == "1\n" ) - assert ( - node.query( - "SELECT count() FROM information_schema.TABLES WHERE table_name='TABLES'" - ) - == "2\n" - ) - assert ( - node.query( - "SELECT count() FROM INFORMATION_SCHEMA.tables WHERE table_name='tables'" - ) - == "3\n" - ) diff --git a/tests/queries/0_stateless/01161_information_schema.reference b/tests/queries/0_stateless/01161_information_schema.reference index 4ec33a70be0..32ad3f16abc 100644 --- a/tests/queries/0_stateless/01161_information_schema.reference +++ b/tests/queries/0_stateless/01161_information_schema.reference @@ -33,3 +33,5 @@ default default v default v f 2 0 Float64 \N \N \N \N \N \N \N \N \N \N \N \N \ tmp tmp d 1 0 Date \N \N \N \N \N 0 \N \N \N \N \N \N \N \N \N Date tmp tmp dt 2 0 DateTime \N \N \N \N \N 0 \N \N \N \N \N \N \N \N \N DateTime tmp tmp dtms 3 0 DateTime64(3) \N \N \N \N \N 3 \N \N \N \N \N \N \N \N \N DateTime64(3) +1 +1 diff --git a/tests/queries/0_stateless/01161_information_schema.sql b/tests/queries/0_stateless/01161_information_schema.sql index ed77ef1c1c2..68a3b011ced 100644 --- a/tests/queries/0_stateless/01161_information_schema.sql +++ b/tests/queries/0_stateless/01161_information_schema.sql @@ -1,20 +1,31 @@ -show tables from information_schema; +SHOW TABLES FROM information_schema; SHOW TABLES FROM INFORMATION_SCHEMA; -create table t (n UInt64, f Float32, s String, fs FixedString(42), d Decimal(9, 6)) engine=Memory; -create view v (n Nullable(Int32), f Float64) as select n, f from t; -create materialized view mv engine=Null as select * from system.one; -create temporary table tmp (d Date, dt DateTime, dtms DateTime64(3)); +DROP TABLE IF EXISTS t; +DROP VIEW IF EXISTS v; +DROP VIEW IF EXISTS mv; +DROP TABLE IF EXISTS tmp; + +CREATE TABLE t (n UInt64, f Float32, s String, fs FixedString(42), d Decimal(9, 6)) ENGINE=Memory; +CREATE VIEW v (n Nullable(Int32), f Float64) AS SELECT n, f FROM t; +CREATE MATERIALIZED VIEW mv ENGINE=Null AS SELECT * FROM system.one; +CREATE TEMPORARY TABLE tmp (d Date, dt DateTime, dtms DateTime64(3)); -- FIXME #28687 -select * from information_schema.schemata where schema_name ilike 'information_schema'; +SELECT * FROM information_schema.schemata WHERE schema_name ilike 'information_schema'; -- SELECT * FROM INFORMATION_SCHEMA.TABLES WHERE (TABLE_SCHEMA=currentDatabase() OR TABLE_SCHEMA='') AND TABLE_NAME NOT LIKE '%inner%'; SELECT * FROM INFORMATION_SCHEMA.TABLES WHERE (table_schema=currentDatabase() OR table_schema='') AND table_name NOT LIKE '%inner%'; -select * from information_schema.views where table_schema=currentDatabase(); +SELECT * FROM information_schema.views WHERE table_schema=currentDatabase(); -- SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE (TABLE_SCHEMA=currentDatabase() OR TABLE_SCHEMA='') AND TABLE_NAME NOT LIKE '%inner%'; SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE (table_schema=currentDatabase() OR table_schema='') AND table_name NOT LIKE '%inner%'; -drop table t; -drop view v; +-- mixed upper/lowercase schema and table name: +SELECT count() FROM information_schema.TABLES WHERE table_schema=currentDatabase() AND table_name = 't'; +SELECT count() FROM INFORMATION_SCHEMA.tables WHERE table_schema=currentDatabase() AND table_name = 't'; +SELECT count() FROM INFORMATION_schema.tables WHERE table_schema=currentDatabase() AND table_name = 't'; -- { serverError UNKNOWN_DATABASE } +SELECT count() FROM information_schema.taBLES WHERE table_schema=currentDatabase() AND table_name = 't'; -- { serverError UNKNOWN_TABLE } + drop view mv; +drop view v; +drop table t; From 31aa9f459ed0aae4dcea3218ba790e2abd1080e3 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 14 Aug 2023 09:35:29 +0000 Subject: [PATCH 514/777] minor fixes --- src/Databases/IDatabase.cpp | 7 +++---- src/Interpreters/DatabaseCatalog.cpp | 10 +++------- 2 files changed, 6 insertions(+), 11 deletions(-) diff --git a/src/Databases/IDatabase.cpp b/src/Databases/IDatabase.cpp index 7d12ae6c588..09640d2f86e 100644 --- a/src/Databases/IDatabase.cpp +++ b/src/Databases/IDatabase.cpp @@ -23,11 +23,10 @@ StoragePtr IDatabase::getTable(const String & name, ContextPtr context) const return storage; TableNameHints hints(this->shared_from_this(), context); std::vector names = hints.getHints(name); - if (!names.empty()) - { + if (names.empty()) + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} does not exist", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name)); + else throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} does not exist. Maybe you meant {}?", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name), backQuoteIfNeed(names[0])); - } - else throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} does not exist", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(name)); } std::vector> IDatabase::getTablesForBackup(const FilterByNameFunction &, const ContextPtr &) const diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 3baaa182d51..dad455d487b 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -341,14 +341,10 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( { TableNameHints hints(this->tryGetDatabase(table_id.getDatabaseName()), getContext()); std::vector names = hints.getHints(table_id.getTableName()); - if (!names.empty()) - { - /// There is two options: first is to print just the name of the table - /// and the second is to print the result in format: db_name.table_name. I'll comment out the second option below - /// I also leave possibility to print several suggestions + if (names.empty()) + exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist", table_id.getNameForLogs())); + else exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist. Maybe you meant {}?", table_id.getNameForLogs(), backQuoteIfNeed(names[0]))); - } - else exception->emplace(Exception(ErrorCodes::UNKNOWN_TABLE, "Table {} does not exist", table_id.getNameForLogs())); } return {}; } From 34c3162c7c6848269bcb50a13a5e7c68a0f62446 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 14 Aug 2023 17:42:42 +0800 Subject: [PATCH 515/777] revert old orc files --- .../Formats/Impl/ORCBlockInputFormat.cpp | 209 ++++++++++++++++++ .../Formats/Impl/ORCBlockInputFormat.h | 78 +++++++ 2 files changed, 287 insertions(+) create mode 100644 src/Processors/Formats/Impl/ORCBlockInputFormat.cpp create mode 100644 src/Processors/Formats/Impl/ORCBlockInputFormat.h diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp new file mode 100644 index 00000000000..8768463735f --- /dev/null +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -0,0 +1,209 @@ +#include "ORCBlockInputFormat.h" +#include +#if USE_ORC + +#include +#include +#include +#include +#include +#include "ArrowBufferedStreams.h" +#include "ArrowColumnToCHColumn.h" +#include "ArrowFieldIndexUtil.h" +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int CANNOT_READ_ALL_DATA; +} + +ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_) + : IInputFormat(std::move(header_), &in_), format_settings(format_settings_), skip_stripes(format_settings.orc.skip_stripes) +{ +} + +Chunk ORCBlockInputFormat::generate() +{ + block_missing_values.clear(); + + if (!file_reader) + prepareReader(); + + if (is_stopped) + return {}; + + for (; stripe_current < stripe_total && skip_stripes.contains(stripe_current); ++stripe_current) + ; + + if (stripe_current >= stripe_total) + return {}; + + auto batch_result = file_reader->ReadStripe(stripe_current, include_indices); + if (!batch_result.ok()) + throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to create batch reader: {}", batch_result.status().ToString()); + + auto batch = batch_result.ValueOrDie(); + if (!batch) + return {}; + + auto table_result = arrow::Table::FromRecordBatches({batch}); + if (!table_result.ok()) + throw ParsingException( + ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading batch of ORC data: {}", table_result.status().ToString()); + + /// We should extract the number of rows directly from the stripe, because in case when + /// record batch contains 0 columns (for example if we requested only columns that + /// are not presented in data) the number of rows in record batch will be 0. + size_t num_rows = file_reader->GetRawORCReader()->getStripe(stripe_current)->getNumberOfRows(); + + auto table = table_result.ValueOrDie(); + if (!table || !num_rows) + return {}; + + approx_bytes_read_for_chunk = file_reader->GetRawORCReader()->getStripe(stripe_current)->getDataLength(); + ++stripe_current; + + Chunk res; + /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. + /// Otherwise fill the missing columns with zero values of its type. + BlockMissingValues * block_missing_values_ptr = format_settings.defaults_for_omitted_fields ? &block_missing_values : nullptr; + arrow_column_to_ch_column->arrowTableToCHChunk(res, table, num_rows, block_missing_values_ptr); + return res; +} + +void ORCBlockInputFormat::resetParser() +{ + IInputFormat::resetParser(); + + file_reader.reset(); + include_indices.clear(); + block_missing_values.clear(); +} + +const BlockMissingValues & ORCBlockInputFormat::getMissingValues() const +{ + return block_missing_values; +} + + +static void getFileReaderAndSchema( + ReadBuffer & in, + std::unique_ptr & file_reader, + std::shared_ptr & schema, + const FormatSettings & format_settings, + std::atomic & is_stopped) +{ + auto arrow_file = asArrowFile(in, format_settings, is_stopped, "ORC", ORC_MAGIC_BYTES); + if (is_stopped) + return; + + auto result = arrow::adapters::orc::ORCFileReader::Open(arrow_file, arrow::default_memory_pool()); + if (!result.ok()) + throw Exception::createDeprecated(result.status().ToString(), ErrorCodes::BAD_ARGUMENTS); + file_reader = std::move(result).ValueOrDie(); + + auto read_schema_result = file_reader->ReadSchema(); + if (!read_schema_result.ok()) + throw Exception::createDeprecated(read_schema_result.status().ToString(), ErrorCodes::BAD_ARGUMENTS); + schema = std::move(read_schema_result).ValueOrDie(); +} + +void ORCBlockInputFormat::prepareReader() +{ + std::shared_ptr schema; + getFileReaderAndSchema(*in, file_reader, schema, format_settings, is_stopped); + if (is_stopped) + return; + + stripe_total = static_cast(file_reader->NumberOfStripes()); + stripe_current = 0; + + arrow_column_to_ch_column = std::make_unique( + getPort().getHeader(), + "ORC", + format_settings.orc.import_nested, + format_settings.orc.allow_missing_columns, + format_settings.null_as_default, + format_settings.orc.case_insensitive_column_matching); + + const bool ignore_case = format_settings.orc.case_insensitive_column_matching; + std::unordered_set nested_table_names; + if (format_settings.orc.import_nested) + nested_table_names = Nested::getAllTableNames(getPort().getHeader(), ignore_case); + + for (int i = 0; i < schema->num_fields(); ++i) + { + const auto & name = schema->field(i)->name(); + if (getPort().getHeader().has(name, ignore_case) || nested_table_names.contains(ignore_case ? boost::to_lower_copy(name) : name)) + include_indices.push_back(i); + } +} + +ORCSchemaReader::ORCSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) + : ISchemaReader(in_), format_settings(format_settings_) +{ +} + +NamesAndTypesList ORCSchemaReader::readSchema() +{ + std::unique_ptr file_reader; + std::shared_ptr schema; + std::atomic is_stopped = 0; + getFileReaderAndSchema(in, file_reader, schema, format_settings, is_stopped); + auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader( + *schema, "ORC", format_settings.orc.skip_columns_with_unsupported_types_in_schema_inference); + if (format_settings.schema_inference_make_columns_nullable) + return getNamesAndRecursivelyNullableTypes(header); + return header.getNamesAndTypesList();} + +void registerInputFormatDeprecatedORC(FormatFactory & factory) +{ + factory.registerInputFormat( + "DeprecatedORC", + [](ReadBuffer &buf, + const Block &sample, + const RowInputFormatParams &, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, settings); + }); + factory.markFormatSupportsSubcolumns("DeprecatedORC"); + factory.markFormatSupportsSubsetOfColumns("DeprecatedORC"); +} + +void registerDeprecatedORCSchemaReader(FormatFactory & factory) +{ + factory.registerSchemaReader( + "DeprecatedORC", + [](ReadBuffer & buf, const FormatSettings & settings) + { + return std::make_shared(buf, settings); + } + ); + + factory.registerAdditionalInfoForSchemaCacheGetter("DeprecatedORC", [](const FormatSettings & settings) + { + return fmt::format("schema_inference_make_columns_nullable={}", settings.schema_inference_make_columns_nullable); + }); +} + +} +#else + +namespace DB +{ + class FormatFactory; + void registerInputFormatDeprecatedORC(FormatFactory &) + { + } + + void registerDeprecatedORCSchemaReader(FormatFactory &) + { + } +} + +#endif diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h new file mode 100644 index 00000000000..98561e72e61 --- /dev/null +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -0,0 +1,78 @@ +#pragma once +#include "config.h" +#if USE_ORC + +#include +#include +#include + +#include + +namespace arrow::adapters::orc +{ + class ORCFileReader; +} + +namespace DB +{ + +class ArrowColumnToCHColumn; + +class ORCBlockInputFormat : public IInputFormat +{ +public: + ORCBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_); + + String getName() const override { return "ORCBlockInputFormat"; } + + void resetParser() override; + + const BlockMissingValues & getMissingValues() const override; + + size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; } + +protected: + Chunk generate() override; + + void onCancel() override + { + is_stopped = 1; + } + +private: + void prepareReader(); + + // TODO: check that this class implements every part of its parent + + std::unique_ptr file_reader; + + std::unique_ptr arrow_column_to_ch_column; + + // indices of columns to read from ORC file + std::vector include_indices; + + BlockMissingValues block_missing_values; + size_t approx_bytes_read_for_chunk; + + const FormatSettings format_settings; + const std::unordered_set & skip_stripes; + + int stripe_total = 0; + int stripe_current = 0; + + std::atomic is_stopped{0}; +}; + +class ORCSchemaReader : public ISchemaReader +{ +public: + ORCSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_); + + NamesAndTypesList readSchema() override; + +private: + const FormatSettings format_settings; +}; + +} +#endif From bbe5caa9dd25c8dd84c57ff887dd55e7fb9f1945 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 14 Aug 2023 19:55:45 +0800 Subject: [PATCH 516/777] fix building --- .../Impl/NativeORCBlockInputFormat.cpp | 84 +++++++++---------- .../Formats/Impl/NativeORCBlockInputFormat.h | 2 - 2 files changed, 38 insertions(+), 48 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 54105e80ed4..98efd14f343 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -195,15 +195,12 @@ void ORCBlockInputFormat::prepareFileReader() orc_column_to_ch_column = std::make_unique( getPort().getHeader(), - format_settings.orc.import_nested, format_settings.orc.allow_missing_columns, format_settings.null_as_default, format_settings.orc.case_insensitive_column_matching); const bool ignore_case = format_settings.orc.case_insensitive_column_matching; - std::unordered_set nested_table_names; - if (format_settings.orc.import_nested) - nested_table_names = Nested::getAllTableNames(getPort().getHeader(), ignore_case); + std::unordered_set nested_table_names = Nested::getAllTableNames(getPort().getHeader(), ignore_case); for (size_t i = 0; i < schema.columns(); ++i) { @@ -314,9 +311,8 @@ NamesAndTypesList ORCSchemaReader::readSchema() ORCColumnToCHColumn::ORCColumnToCHColumn( - const Block & header_, bool import_nested_, bool allow_missing_columns_, bool null_as_default_, bool case_insensitive_matching_) + const Block & header_, bool allow_missing_columns_, bool null_as_default_, bool case_insensitive_matching_) : header(header_) - , import_nested(import_nested_) , allow_missing_columns(allow_missing_columns_) , null_as_default(null_as_default_) , case_insensitive_matching(case_insensitive_matching_) @@ -396,7 +392,7 @@ readColumnWithNumericData(const orc::ColumnVectorBatch * orc_column, const orc:: auto & column_data = static_cast(*internal_column).getData(); column_data.reserve(orc_column->numElements); - const auto * orc_int_column = typeid_cast(orc_column); + const auto * orc_int_column = dynamic_cast(orc_column); column_data.insert_assume_reserved(orc_int_column->data.data(), orc_int_column->data.data() + orc_int_column->numElements); return {std::move(internal_column), std::move(internal_type), column_name}; @@ -412,7 +408,7 @@ readColumnWithNumericDataCast(const orc::ColumnVectorBatch * orc_column, const o auto & column_data = static_cast(*internal_column).getData(); column_data.reserve(orc_column->numElements); - const auto * orc_int_column = typeid_cast(orc_column); + const auto * orc_int_column = dynamic_cast(orc_column); for (size_t i = 0; i < orc_int_column->numElements; ++i) column_data.push_back(static_cast(orc_int_column->data[i])); @@ -488,7 +484,7 @@ static ColumnWithTypeAndName readColumnWithDecimalDataCast( auto & column_data = static_cast(*internal_column).getData(); column_data.reserve(orc_column->numElements); - const auto * orc_decimal_column = typeid_cast(orc_column); + auto * orc_decimal_column = const_cast(dynamic_cast(orc_column)); for (size_t i = 0; i < orc_decimal_column->numElements; ++i) { DecimalType decimal_value; @@ -946,47 +942,44 @@ void ORCColumnToCHColumn::orcColumnsToCHChunk( bool read_from_nested = false; /// Check if it's a column from nested table. - if (import_nested) + String nested_table_name = Nested::extractTableName(header_column.name); + String search_nested_table_name = nested_table_name; + if (case_insensitive_matching) + boost::to_lower(search_nested_table_name); + if (name_to_column_ptr.contains(search_nested_table_name)) { - String nested_table_name = Nested::extractTableName(header_column.name); - String search_nested_table_name = nested_table_name; - if (case_insensitive_matching) - boost::to_lower(search_nested_table_name); - if (name_to_column_ptr.contains(search_nested_table_name)) + if (!nested_tables.contains(search_nested_table_name)) { - if (!nested_tables.contains(search_nested_table_name)) + NamesAndTypesList nested_columns; + for (const auto & name_and_type : header.getNamesAndTypesList()) { - NamesAndTypesList nested_columns; - for (const auto & name_and_type : header.getNamesAndTypesList()) - { - if (name_and_type.name.starts_with(nested_table_name + ".")) - nested_columns.push_back(name_and_type); - } - auto nested_table_type = Nested::collect(nested_columns).front().type; - - auto orc_column_with_type = name_to_column_ptr[search_nested_table_name]; - ColumnsWithTypeAndName cols = {readColumnFromORCColumn( - orc_column_with_type.first, - orc_column_with_type.second, - nested_table_name, - false, - true, - false, - skipped, - nested_table_type)}; - BlockPtr block_ptr = std::make_shared(cols); - auto column_extractor = std::make_shared(*block_ptr, case_insensitive_matching); - nested_tables[search_nested_table_name] = {block_ptr, column_extractor}; + if (name_and_type.name.starts_with(nested_table_name + ".")) + nested_columns.push_back(name_and_type); } + auto nested_table_type = Nested::collect(nested_columns).front().type; - auto nested_column = nested_tables[search_nested_table_name].second->extractColumn(search_column_name); - if (nested_column) - { - column = *nested_column; - if (case_insensitive_matching) - column.name = header_column.name; - read_from_nested = true; - } + auto orc_column_with_type = name_to_column_ptr[search_nested_table_name]; + ColumnsWithTypeAndName cols = {readColumnFromORCColumn( + orc_column_with_type.first, + orc_column_with_type.second, + nested_table_name, + false, + true, + false, + skipped, + nested_table_type)}; + BlockPtr block_ptr = std::make_shared(cols); + auto column_extractor = std::make_shared(*block_ptr, case_insensitive_matching); + nested_tables[search_nested_table_name] = {block_ptr, column_extractor}; + } + + auto nested_column = nested_tables[search_nested_table_name].second->extractColumn(search_column_name); + if (nested_column) + { + column = *nested_column; + if (case_insensitive_matching) + column.name = header_column.name; + read_from_nested = true; } } @@ -1051,7 +1044,6 @@ void registerInputFormatORC(FormatFactory & factory) "ORC", [](ReadBuffer & buf, const Block & sample, const RowInputFormatParams &, const FormatSettings & settings) { return std::make_shared(buf, sample, settings); }); - factory.markFormatSupportsSubcolumns("ORC"); factory.markFormatSupportsSubsetOfColumns("ORC"); } diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h index 50104018aee..5fbc901b283 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h @@ -93,7 +93,6 @@ public: ORCColumnToCHColumn( const Block & header_, - bool import_nested_, bool allow_missing_columns_, bool null_as_default_, bool case_insensitive_matching_ = false); @@ -110,7 +109,6 @@ public: private: const Block & header; - bool import_nested; /// If false, throw exception if some columns in header not exists in arrow table. bool allow_missing_columns; bool null_as_default; From 310ac6feaf0c16ee2f962187ba721054f9929d3a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Aug 2023 14:19:08 +0200 Subject: [PATCH 517/777] Tune PRInfo.has_changes_in_documentation --- tests/ci/pr_info.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 86d4985c6b2..dee71b726df 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -279,7 +279,7 @@ class PRInfo: "user_orgs": self.user_orgs, } - def has_changes_in_documentation(self): + def has_changes_in_documentation(self) -> bool: # If the list wasn't built yet the best we can do is to # assume that there were changes. if self.changed_files is None or not self.changed_files: @@ -287,10 +287,9 @@ class PRInfo: for f in self.changed_files: _, ext = os.path.splitext(f) - path_in_docs = "docs" in f - path_in_website = "website" in f + path_in_docs = f.startswith("docs/") if ( - ext in DIFF_IN_DOCUMENTATION_EXT and (path_in_docs or path_in_website) + ext in DIFF_IN_DOCUMENTATION_EXT and path_in_docs ) or "docker/docs" in f: return True return False From 900e38a6768febec05a90d6d79d7cd98e2989b12 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Aug 2023 14:20:40 +0200 Subject: [PATCH 518/777] Fail early on missed documentation for new features --- tests/ci/run_check.py | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 4f022b6c0a5..9e0644d6c6e 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -137,17 +137,19 @@ def main(): if pr_labels_to_remove: remove_labels(gh, pr_info, pr_labels_to_remove) - if FEATURE_LABEL in pr_info.labels: - print(f"The '{FEATURE_LABEL}' in the labels, expect the 'Docs Check' status") + if FEATURE_LABEL in pr_info.labels and not pr_info.has_changes_in_documentation(): + print( + f"The '{FEATURE_LABEL}' in the labels, " + "but there's no changed documentation" + ) post_commit_status( # do not pass pr_info here intentionally commit, - "pending", + "failure", NotSet, f"expect adding docs for {FEATURE_LABEL}", - DOCS_NAME, + CI_STATUS_NAME, ) - elif not description_error: - set_mergeable_check(commit, "skipped") + sys.exit(1) if description_error: print( @@ -173,6 +175,7 @@ def main(): ) sys.exit(1) + set_mergeable_check(commit, "skipped") ci_report_url = create_ci_report(pr_info, []) if not can_run: print("::notice ::Cannot run") From 56a8818cf25b4335c3707ad02f6585c21705bf2b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Aug 2023 14:31:26 +0200 Subject: [PATCH 519/777] Fix logic of Mergeable Check --- tests/ci/run_check.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 9e0644d6c6e..db98a2c1ab5 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -147,7 +147,8 @@ def main(): "failure", NotSet, f"expect adding docs for {FEATURE_LABEL}", - CI_STATUS_NAME, + DOCS_NAME, + pr_info, ) sys.exit(1) From f8b1d7474dffa024ff692bec35578c5172aeea8a Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 14 Aug 2023 12:46:23 +0000 Subject: [PATCH 520/777] Update test_distributed_inter_server_secret to pass with analyzer --- tests/analyzer_integration_broken_tests.txt | 18 ----- .../test.py | 68 +++++++------------ 2 files changed, 25 insertions(+), 61 deletions(-) diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 68822fbf311..3cc4869aa62 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -5,24 +5,6 @@ test_distributed_ddl/test.py::test_default_database[configs_secure] test_distributed_ddl/test.py::test_on_server_fail[configs] test_distributed_ddl/test.py::test_on_server_fail[configs_secure] test_distributed_insert_backward_compatibility/test.py::test_distributed_in_tuple -test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_secure_cluster[default-] -test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_secure_cluster[nopass-] -test_distributed_inter_server_secret/test.py::test_per_user_inline_settings_secure_cluster[pass-foo] -test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_secure_cluster[default-] -test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_secure_cluster[nopass-] -test_distributed_inter_server_secret/test.py::test_per_user_protocol_settings_secure_cluster[pass-foo] -test_distributed_inter_server_secret/test.py::test_user_insecure_cluster[default-] -test_distributed_inter_server_secret/test.py::test_user_insecure_cluster[nopass-] -test_distributed_inter_server_secret/test.py::test_user_insecure_cluster[pass-foo] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster[default-] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster[nopass-] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster[pass-foo] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster_from_backward[default-] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster_from_backward[nopass-] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster_from_backward[pass-foo] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster_with_backward[default-] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster_with_backward[nopass-] -test_distributed_inter_server_secret/test.py::test_user_secure_cluster_with_backward[pass-foo] test_distributed_load_balancing/test.py::test_distributed_replica_max_ignored_errors test_distributed_load_balancing/test.py::test_load_balancing_default test_distributed_load_balancing/test.py::test_load_balancing_priority_round_robin[dist_priority] diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index 36ac07a550a..1aeaddcf3c5 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -110,10 +110,6 @@ def start_cluster(): cluster.shutdown() -def query_with_id(node, id_, query, **kwargs): - return node.query("WITH '{}' AS __id {}".format(id_, query), **kwargs) - - # @return -- [user, initial_user] def get_query_user_info(node, query_pattern): node.query("SYSTEM FLUSH LOGS") @@ -334,7 +330,7 @@ def test_secure_disagree_insert(): @users def test_user_insecure_cluster(user, password): id_ = "query-dist_insecure-" + user - query_with_id(n1, id_, "SELECT * FROM dist_insecure", user=user, password=password) + n1.query(f"SELECT *, '{id_}' FROM dist_insecure", user=user, password=password) assert get_query_user_info(n1, id_) == [ user, user, @@ -345,7 +341,7 @@ def test_user_insecure_cluster(user, password): @users def test_user_secure_cluster(user, password): id_ = "query-dist_secure-" + user - query_with_id(n1, id_, "SELECT * FROM dist_secure", user=user, password=password) + n1.query(f"SELECT *, '{id_}' FROM dist_secure", user=user, password=password) assert get_query_user_info(n1, id_) == [user, user] assert get_query_user_info(n2, id_) == [user, user] @@ -353,16 +349,14 @@ def test_user_secure_cluster(user, password): @users def test_per_user_inline_settings_insecure_cluster(user, password): id_ = "query-ddl-settings-dist_insecure-" + user - query_with_id( - n1, - id_, - """ - SELECT * FROM dist_insecure - SETTINGS - prefer_localhost_replica=0, - max_memory_usage_for_user=1e9, - max_untracked_memory=0 - """, + n1.query( + f""" + SELECT *, '{id_}' FROM dist_insecure + SETTINGS + prefer_localhost_replica=0, + max_memory_usage_for_user=1e9, + max_untracked_memory=0 + """, user=user, password=password, ) @@ -372,16 +366,14 @@ def test_per_user_inline_settings_insecure_cluster(user, password): @users def test_per_user_inline_settings_secure_cluster(user, password): id_ = "query-ddl-settings-dist_secure-" + user - query_with_id( - n1, - id_, - """ - SELECT * FROM dist_secure - SETTINGS - prefer_localhost_replica=0, - max_memory_usage_for_user=1e9, - max_untracked_memory=0 - """, + n1.query( + f""" + SELECT *, '{id_}' FROM dist_secure + SETTINGS + prefer_localhost_replica=0, + max_memory_usage_for_user=1e9, + max_untracked_memory=0 + """, user=user, password=password, ) @@ -393,10 +385,8 @@ def test_per_user_inline_settings_secure_cluster(user, password): @users def test_per_user_protocol_settings_insecure_cluster(user, password): id_ = "query-protocol-settings-dist_insecure-" + user - query_with_id( - n1, - id_, - "SELECT * FROM dist_insecure", + n1.query( + f"SELECT *, '{id_}' FROM dist_insecure", user=user, password=password, settings={ @@ -411,10 +401,8 @@ def test_per_user_protocol_settings_insecure_cluster(user, password): @users def test_per_user_protocol_settings_secure_cluster(user, password): id_ = "query-protocol-settings-dist_secure-" + user - query_with_id( - n1, - id_, - "SELECT * FROM dist_secure", + n1.query( + f"SELECT *, '{id_}' FROM dist_secure", user=user, password=password, settings={ @@ -431,8 +419,8 @@ def test_per_user_protocol_settings_secure_cluster(user, password): @users def test_user_secure_cluster_with_backward(user, password): id_ = "with-backward-query-dist_secure-" + user - query_with_id( - n1, id_, "SELECT * FROM dist_secure_backward", user=user, password=password + n1.query( + f"SELECT *, '{id_}' FROM dist_secure_backward", user=user, password=password ) assert get_query_user_info(n1, id_) == [user, user] assert get_query_user_info(backward, id_) == [user, user] @@ -441,13 +429,7 @@ def test_user_secure_cluster_with_backward(user, password): @users def test_user_secure_cluster_from_backward(user, password): id_ = "from-backward-query-dist_secure-" + user - query_with_id( - backward, - id_, - "SELECT * FROM dist_secure_backward", - user=user, - password=password, - ) + backward.query(f"SELECT *, '{id_}' FROM dist_secure", user=user, password=password) assert get_query_user_info(n1, id_) == [user, user] assert get_query_user_info(backward, id_) == [user, user] From 3655df0f406792d65b212807eb88e81966c95b98 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 14 Aug 2023 16:32:52 +0200 Subject: [PATCH 521/777] Attempt to address reset ENV in init.d script --- programs/install/Install.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index d7086c95beb..e10a9fea86b 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -997,7 +997,9 @@ namespace { /// sudo respects limits in /etc/security/limits.conf e.g. open files, /// that's why we are using it instead of the 'clickhouse su' tool. - command = fmt::format("sudo -u '{}' {}", user, command); + /// by default, sudo resets all the ENV variables, but we should preserve + /// the values /etc/default/clickhouse in /etc/init.d/clickhouse file + command = fmt::format("sudo --preserve-env -u '{}' {}", user, command); } fmt::print("Will run {}\n", command); From ca2f800fa5d739b84d9817263678ba16ae9a8cc4 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 14 Aug 2023 14:35:01 +0000 Subject: [PATCH 522/777] Remove unnecessary code --- .../ClusterProxy/SelectStreamFactory.h | 3 --- src/Interpreters/ClusterProxy/executeQuery.cpp | 16 +--------------- 2 files changed, 1 insertion(+), 18 deletions(-) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index 1cc5a3b1a77..ca07fd5deda 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -60,9 +60,6 @@ public: /// (When there is a local replica with big delay). bool lazy = false; time_t local_delay = 0; - - /// Set only if parallel reading from replicas is used. - std::shared_ptr coordinator; }; using Shards = std::vector; diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 2fed626ffb7..bb5c83eca39 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -281,7 +281,6 @@ void executeQueryWithParallelReplicas( auto all_replicas_count = std::min(static_cast(settings.max_parallel_replicas), new_cluster->getShardCount()); auto coordinator = std::make_shared(all_replicas_count); auto remote_plan = std::make_unique(); - auto plans = std::vector(); /// This is a little bit weird, but we construct an "empty" coordinator without /// any specified reading/coordination method (like Default, InOrder, InReverseOrder) @@ -309,20 +308,7 @@ void executeQueryWithParallelReplicas( &Poco::Logger::get("ReadFromParallelRemoteReplicasStep"), query_info.storage_limits); - remote_plan->addStep(std::move(read_from_remote)); - remote_plan->addInterpreterContext(context); - plans.emplace_back(std::move(remote_plan)); - - if (std::all_of(plans.begin(), plans.end(), [](const QueryPlanPtr & plan) { return !plan; })) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No plans were generated for reading from shard. This is a bug"); - - DataStreams input_streams; - input_streams.reserve(plans.size()); - for (const auto & plan : plans) - input_streams.emplace_back(plan->getCurrentDataStream()); - - auto union_step = std::make_unique(std::move(input_streams)); - query_plan.unitePlans(std::move(union_step), std::move(plans)); + query_plan.addStep(std::move(read_from_remote)); } } From 1738afc1965de150342e0d9a7d52b85fe561d24c Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 14 Aug 2023 16:37:34 +0200 Subject: [PATCH 523/777] Update insert-into.md --- docs/en/sql-reference/statements/insert-into.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index d6e30827f9b..e0cc98c2351 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -11,7 +11,7 @@ Inserts data into a table. **Syntax** ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... ``` You can specify a list of columns to insert using the `(c1, c2, c3)`. You can also use an expression with column [matcher](../../sql-reference/statements/select/index.md#asterisk) such as `*` and/or [modifiers](../../sql-reference/statements/select/index.md#select-modifiers) such as [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#except-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier). @@ -107,7 +107,7 @@ If table has [constraints](../../sql-reference/statements/create/table.md#constr **Syntax** ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] SELECT ... ``` Columns are mapped according to their position in the SELECT clause. However, their names in the SELECT expression and the table for INSERT may differ. If necessary, type casting is performed. @@ -126,7 +126,7 @@ To insert a default value instead of `NULL` into a column with not nullable data **Syntax** ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] FROM INFILE file_name [COMPRESSION type] FORMAT format_name +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] FROM INFILE file_name [COMPRESSION type] FORMAT format_name ``` Use the syntax above to insert data from a file, or files, stored on the **client** side. `file_name` and `type` are string literals. Input file [format](../../interfaces/formats.md) must be set in the `FORMAT` clause. From 3d5c9bfa1ad5f00fb1dc3cd45444ebb23a3219a2 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 14 Aug 2023 16:39:37 +0200 Subject: [PATCH 524/777] Update insert-into.md --- docs/ru/sql-reference/statements/insert-into.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/sql-reference/statements/insert-into.md b/docs/ru/sql-reference/statements/insert-into.md index 4fa6ac4ce66..747e36b8809 100644 --- a/docs/ru/sql-reference/statements/insert-into.md +++ b/docs/ru/sql-reference/statements/insert-into.md @@ -11,7 +11,7 @@ sidebar_label: INSERT INTO **Синтаксис** ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... ``` Вы можете указать список столбцов для вставки, используя синтаксис `(c1, c2, c3)`. Также можно использовать выражение cо [звездочкой](../../sql-reference/statements/select/index.md#asterisk) и/или модификаторами, такими как [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#except-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier). @@ -100,7 +100,7 @@ INSERT INTO t FORMAT TabSeparated **Синтаксис** ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] SELECT ... ``` Соответствие столбцов определяется их позицией в секции SELECT. При этом, их имена в выражении SELECT и в таблице для INSERT, могут отличаться. При необходимости выполняется приведение типов данных, эквивалентное соответствующему оператору CAST. @@ -120,7 +120,7 @@ INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... **Синтаксис** ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] FROM INFILE file_name [COMPRESSION type] FORMAT format_name +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] FROM INFILE file_name [COMPRESSION type] FORMAT format_name ``` Используйте этот синтаксис, чтобы вставить данные из файла, который хранится на стороне **клиента**. `file_name` и `type` задаются в виде строковых литералов. [Формат](../../interfaces/formats.md) входного файла должен быть задан в секции `FORMAT`. From 8f3f47a51fc15a2a5fc7acf98299b187bb69eed3 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Mon, 14 Aug 2023 16:40:36 +0200 Subject: [PATCH 525/777] Update insert-into.md --- docs/zh/sql-reference/statements/insert-into.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/zh/sql-reference/statements/insert-into.md b/docs/zh/sql-reference/statements/insert-into.md index 9acc1655f9a..f80c0a8a8ea 100644 --- a/docs/zh/sql-reference/statements/insert-into.md +++ b/docs/zh/sql-reference/statements/insert-into.md @@ -8,7 +8,7 @@ INSERT INTO 语句主要用于向系统中添加数据. 查询的基本格式: ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... ``` 您可以在查询中指定要插入的列的列表,如:`[(c1, c2, c3)]`。您还可以使用列[匹配器](../../sql-reference/statements/select/index.md#asterisk)的表达式,例如`*`和/或[修饰符](../../sql-reference/statements/select/index.md#select-modifiers),例如 [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#apply-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier)。 @@ -71,7 +71,7 @@ INSERT INTO [db.]table [(c1, c2, c3)] FORMAT format_name data_set 例如,下面的查询所使用的输入格式就与上面INSERT … VALUES的中使用的输入格式相同: ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] FORMAT Values (v11, v12, v13), (v21, v22, v23), ... +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] FORMAT Values (v11, v12, v13), (v21, v22, v23), ... ``` ClickHouse会清除数据前所有的空白字符与一个换行符(如果有换行符的话)。所以在进行查询时,我们建议您将数据放入到输入输出格式名称后的新的一行中去(如果数据是以空白字符开始的,这将非常重要)。 @@ -93,7 +93,7 @@ INSERT INTO t FORMAT TabSeparated ### 使用`SELECT`的结果写入 {#inserting-the-results-of-select} ``` sql -INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... +INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] SELECT ... ``` 写入与SELECT的列的对应关系是使用位置来进行对应的,尽管它们在SELECT表达式与INSERT中的名称可能是不同的。如果需要,会对它们执行对应的类型转换。 From 364b43f19ef4b266291642e5431d9c389bd9c151 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 14:49:40 +0000 Subject: [PATCH 526/777] Cosmetics: add "cache" section marker --- src/Interpreters/Context.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 07e0cac79e3..b9311acc337 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -915,6 +915,8 @@ public: void setSystemZooKeeperLogAfterInitializationIfNeeded(); + /// --- Caches ------------------------------------------------------------------------------------------ + /// Create a cache of uncompressed blocks of specified size. This can be done only once. void setUncompressedCache(const String & uncompressed_cache_policy, size_t max_size_in_bytes); std::shared_ptr getUncompressedCache() const; @@ -962,6 +964,8 @@ public: */ void dropCaches() const; + /// ----------------------------------------------------------------------------------------------------- + /// Settings for MergeTree background tasks stored in config.xml BackgroundTaskSchedulingSettings getBackgroundProcessingTaskSchedulingSettings() const; BackgroundTaskSchedulingSettings getBackgroundMoveTaskSchedulingSettings() const; From a81c762928c3766be025fbb4043081d37f897c02 Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Mon, 14 Aug 2023 14:52:26 +0000 Subject: [PATCH 527/777] Fix style --- src/Interpreters/ClusterProxy/executeQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index bb5c83eca39..f2d7132b174 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -28,7 +28,6 @@ namespace DB namespace ErrorCodes { extern const int TOO_LARGE_DISTRIBUTED_DEPTH; - extern const int LOGICAL_ERROR; extern const int SUPPORT_IS_DISABLED; } From fed995734ec446175bdb009d85bfa883d439e02e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 14:52:37 +0000 Subject: [PATCH 528/777] Cosmetics: Move prefetch threadpool stuff out of "caching" section --- src/Interpreters/Context.cpp | 48 ++++++++++++++++-------------------- src/Interpreters/Context.h | 14 +++++------ 2 files changed, 28 insertions(+), 34 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b8a700f5311..a12cc70a507 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2315,32 +2315,6 @@ ThreadPool & Context::getLoadMarksThreadpool() const return *shared->load_marks_threadpool; } -static size_t getPrefetchThreadpoolSizeFromConfig(const Poco::Util::AbstractConfiguration & config) -{ - return config.getUInt(".prefetch_threadpool_pool_size", 100); -} - -size_t Context::getPrefetchThreadpoolSize() const -{ - const auto & config = getConfigRef(); - return getPrefetchThreadpoolSizeFromConfig(config); -} - -ThreadPool & Context::getPrefetchThreadpool() const -{ - const auto & config = getConfigRef(); - - auto lock = getLock(); - if (!shared->prefetch_threadpool) - { - auto pool_size = getPrefetchThreadpoolSize(); - auto queue_size = config.getUInt(".prefetch_threadpool_queue_size", 1000000); - shared->prefetch_threadpool = std::make_unique( - CurrentMetrics::IOPrefetchThreads, CurrentMetrics::IOPrefetchThreadsActive, pool_size, pool_size, queue_size); - } - return *shared->prefetch_threadpool; -} - void Context::setIndexUncompressedCache(size_t max_size_in_bytes) { auto lock = getLock(); @@ -2351,7 +2325,6 @@ void Context::setIndexUncompressedCache(size_t max_size_in_bytes) shared->index_uncompressed_cache = std::make_shared(max_size_in_bytes); } - UncompressedCachePtr Context::getIndexUncompressedCache() const { auto lock = getLock(); @@ -2468,6 +2441,27 @@ void Context::dropCaches() const shared->mmap_cache->reset(); } +ThreadPool & Context::getPrefetchThreadpool() const +{ + const auto & config = getConfigRef(); + + auto lock = getLock(); + if (!shared->prefetch_threadpool) + { + auto pool_size = getPrefetchThreadpoolSize(); + auto queue_size = config.getUInt(".prefetch_threadpool_queue_size", 1000000); + shared->prefetch_threadpool = std::make_unique( + CurrentMetrics::IOPrefetchThreads, CurrentMetrics::IOPrefetchThreadsActive, pool_size, pool_size, queue_size); + } + return *shared->prefetch_threadpool; +} + +size_t Context::getPrefetchThreadpoolSize() const +{ + const auto & config = getConfigRef(); + return config.getUInt(".prefetch_threadpool_pool_size", 100); +} + BackgroundSchedulePool & Context::getBufferFlushSchedulePool() const { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index b9311acc337..a2f06f4fc14 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -928,13 +928,6 @@ public: void dropMarkCache() const; ThreadPool & getLoadMarksThreadpool() const; - ThreadPool & getPrefetchThreadpool() const; - - /// Note: prefetchThreadpool is different from threadpoolReader - /// in the way that its tasks are - wait for marks to be loaded - /// and make a prefetch by putting a read task to threadpoolReader. - size_t getPrefetchThreadpoolSize() const; - /// Create a cache of index uncompressed blocks of specified size. This can be done only once. void setIndexUncompressedCache(size_t max_size_in_bytes); std::shared_ptr getIndexUncompressedCache() const; @@ -966,6 +959,13 @@ public: /// ----------------------------------------------------------------------------------------------------- + ThreadPool & getPrefetchThreadpool() const; + + /// Note: prefetchThreadpool is different from threadpoolReader + /// in the way that its tasks are - wait for marks to be loaded + /// and make a prefetch by putting a read task to threadpoolReader. + size_t getPrefetchThreadpoolSize() const; + /// Settings for MergeTree background tasks stored in config.xml BackgroundTaskSchedulingSettings getBackgroundProcessingTaskSchedulingSettings() const; BackgroundTaskSchedulingSettings getBackgroundMoveTaskSchedulingSettings() const; From def9bb8d92d6a7dad413d8a3a88eec7e74d5296d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 26 Jun 2023 17:11:25 +0000 Subject: [PATCH 529/777] Exclude the query cache from dropCaches() --- src/Interpreters/Context.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a12cc70a507..1d2774c4d77 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2434,11 +2434,10 @@ void Context::dropCaches() const if (shared->index_mark_cache) shared->index_mark_cache->reset(); - if (shared->query_cache) - shared->query_cache->reset(); - if (shared->mmap_cache) shared->mmap_cache->reset(); + + /// Intentionally not dropping the query cache which is transactionally inconsistent by design. } ThreadPool & Context::getPrefetchThreadpool() const From 646c51ea40038a3d5e6f2f275732c9fa4507ddbd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 26 Jun 2023 17:22:15 +0000 Subject: [PATCH 530/777] Cosmetics: Move methods into a more logical order. --- src/Interpreters/Context.cpp | 46 ++++++++++++++++++------------------ 1 file changed, 23 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 1d2774c4d77..5a7ca4a7221 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2363,6 +2363,29 @@ void Context::dropIndexMarkCache() const shared->index_mark_cache->reset(); } +void Context::setMMappedFileCache(size_t cache_size_in_num_entries) +{ + auto lock = getLock(); + + if (shared->mmap_cache) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Mapped file cache has been already created."); + + shared->mmap_cache = std::make_shared(cache_size_in_num_entries); +} + +MMappedFileCachePtr Context::getMMappedFileCache() const +{ + auto lock = getLock(); + return shared->mmap_cache; +} + +void Context::dropMMappedFileCache() const +{ + auto lock = getLock(); + if (shared->mmap_cache) + shared->mmap_cache->reset(); +} + void Context::setQueryCache(const Poco::Util::AbstractConfiguration & config) { auto lock = getLock(); @@ -2394,29 +2417,6 @@ void Context::dropQueryCache() const shared->query_cache->reset(); } -void Context::setMMappedFileCache(size_t cache_size_in_num_entries) -{ - auto lock = getLock(); - - if (shared->mmap_cache) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Mapped file cache has been already created."); - - shared->mmap_cache = std::make_shared(cache_size_in_num_entries); -} - -MMappedFileCachePtr Context::getMMappedFileCache() const -{ - auto lock = getLock(); - return shared->mmap_cache; -} - -void Context::dropMMappedFileCache() const -{ - auto lock = getLock(); - if (shared->mmap_cache) - shared->mmap_cache->reset(); -} - void Context::dropCaches() const { From 3472a928d804c7fea0a3ddd6b356abf91bdbf630 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 29 Jun 2023 10:02:41 +0000 Subject: [PATCH 531/777] Cosmetics: Rename dropCache() to clearCaches() "dropCache()" is a bad name, it sounds like that the cache can no longer be used after the call. Not true, we only "clear" the cache. --- src/Interpreters/Context.cpp | 15 +++++++-------- src/Interpreters/Context.h | 14 +++++++------- src/Interpreters/InterpreterDropQuery.cpp | 4 ++-- src/Interpreters/InterpreterSystemQuery.cpp | 12 ++++++------ src/Storages/MergeTree/MergeTreeData.cpp | 8 ++++---- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageStripeLog.cpp | 2 +- 8 files changed, 29 insertions(+), 30 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5a7ca4a7221..39284b4c1b2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2269,7 +2269,7 @@ UncompressedCachePtr Context::getUncompressedCache() const } -void Context::dropUncompressedCache() const +void Context::clearUncompressedCache() const { auto lock = getLock(); if (shared->uncompressed_cache) @@ -2293,7 +2293,7 @@ MarkCachePtr Context::getMarkCache() const return shared->mark_cache; } -void Context::dropMarkCache() const +void Context::clearMarkCache() const { auto lock = getLock(); if (shared->mark_cache) @@ -2332,7 +2332,7 @@ UncompressedCachePtr Context::getIndexUncompressedCache() const } -void Context::dropIndexUncompressedCache() const +void Context::clearIndexUncompressedCache() const { auto lock = getLock(); if (shared->index_uncompressed_cache) @@ -2356,7 +2356,7 @@ MarkCachePtr Context::getIndexMarkCache() const return shared->index_mark_cache; } -void Context::dropIndexMarkCache() const +void Context::clearIndexMarkCache() const { auto lock = getLock(); if (shared->index_mark_cache) @@ -2379,7 +2379,7 @@ MMappedFileCachePtr Context::getMMappedFileCache() const return shared->mmap_cache; } -void Context::dropMMappedFileCache() const +void Context::clearMMappedFileCache() const { auto lock = getLock(); if (shared->mmap_cache) @@ -2410,15 +2410,14 @@ QueryCachePtr Context::getQueryCache() const return shared->query_cache; } -void Context::dropQueryCache() const +void Context::clearQueryCache() const { auto lock = getLock(); if (shared->query_cache) shared->query_cache->reset(); } - -void Context::dropCaches() const +void Context::clearCaches() const { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index a2f06f4fc14..c725d032bbe 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -920,34 +920,34 @@ public: /// Create a cache of uncompressed blocks of specified size. This can be done only once. void setUncompressedCache(const String & uncompressed_cache_policy, size_t max_size_in_bytes); std::shared_ptr getUncompressedCache() const; - void dropUncompressedCache() const; + void clearUncompressedCache() const; /// Create a cache of marks of specified size. This can be done only once. void setMarkCache(const String & mark_cache_policy, size_t cache_size_in_bytes); std::shared_ptr getMarkCache() const; - void dropMarkCache() const; + void clearMarkCache() const; ThreadPool & getLoadMarksThreadpool() const; /// Create a cache of index uncompressed blocks of specified size. This can be done only once. void setIndexUncompressedCache(size_t max_size_in_bytes); std::shared_ptr getIndexUncompressedCache() const; - void dropIndexUncompressedCache() const; + void clearIndexUncompressedCache() const; /// Create a cache of index marks of specified size. This can be done only once. void setIndexMarkCache(size_t cache_size_in_bytes); std::shared_ptr getIndexMarkCache() const; - void dropIndexMarkCache() const; + void clearIndexMarkCache() const; /// Create a cache of mapped files to avoid frequent open/map/unmap/close and to reuse from several threads. void setMMappedFileCache(size_t cache_size_in_num_entries); std::shared_ptr getMMappedFileCache() const; - void dropMMappedFileCache() const; + void clearMMappedFileCache() const; /// Create a cache of query results for statements which run repeatedly. void setQueryCache(const Poco::Util::AbstractConfiguration & config); void updateQueryCacheConfiguration(const Poco::Util::AbstractConfiguration & config); std::shared_ptr getQueryCache() const; - void dropQueryCache() const; + void clearQueryCache() const; /** Clear the caches of the uncompressed blocks and marks. * This is usually done when renaming tables, changing the type of columns, deleting a table. @@ -955,7 +955,7 @@ public: * (when deleting a table - it is necessary, since in its place another can appear) * const - because the change in the cache is not considered significant. */ - void dropCaches() const; + void clearCaches() const; /// ----------------------------------------------------------------------------------------------------- diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index ed927d550a8..ea4dfc4df65 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -247,10 +247,10 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue DatabaseCatalog::instance().removeDependencies(table_id, check_ref_deps, check_loading_deps, is_drop_or_detach_database); database->dropTable(context_, table_id.table_name, query.sync); - /// We have to drop mmapio cache when dropping table from Ordinary database + /// We have to clear mmapio cache when dropping table from Ordinary database /// to avoid reading old data if new table with the same name is created if (database->getUUID() == UUIDHelpers::Nil) - context_->dropMMappedFileCache(); + context_->clearMMappedFileCache(); } db = database; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index c0341330846..9c8bc256fa2 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -319,27 +319,27 @@ BlockIO InterpreterSystemQuery::execute() } case Type::DROP_MARK_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_MARK_CACHE); - system_context->dropMarkCache(); + system_context->clearMarkCache(); break; case Type::DROP_UNCOMPRESSED_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_UNCOMPRESSED_CACHE); - system_context->dropUncompressedCache(); + system_context->clearUncompressedCache(); break; case Type::DROP_INDEX_MARK_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_MARK_CACHE); - system_context->dropIndexMarkCache(); + system_context->clearIndexMarkCache(); break; case Type::DROP_INDEX_UNCOMPRESSED_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_UNCOMPRESSED_CACHE); - system_context->dropIndexUncompressedCache(); + system_context->clearIndexUncompressedCache(); break; case Type::DROP_MMAP_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_MMAP_CACHE); - system_context->dropMMappedFileCache(); + system_context->clearMMappedFileCache(); break; case Type::DROP_QUERY_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_QUERY_CACHE); - getContext()->dropQueryCache(); + getContext()->clearQueryCache(); break; #if USE_EMBEDDED_COMPILER case Type::DROP_COMPILED_EXPRESSION_CACHE: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index db0a7b34d7e..7b1fbd0deb0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2328,7 +2328,7 @@ size_t MergeTreeData::clearOldPartsFromFilesystem(bool force) removePartsFinally(parts_to_remove); /// This is needed to close files to avoid they reside on disk after being deleted. /// NOTE: we can drop files from cache more selectively but this is good enough. - getContext()->dropMMappedFileCache(); + getContext()->clearMMappedFileCache(); return parts_to_remove.size(); } @@ -2799,7 +2799,7 @@ void MergeTreeData::rename(const String & new_table_path, const StorageID & new_ } if (!getStorageID().hasUUID()) - getContext()->dropCaches(); + getContext()->clearCaches(); /// TODO: remove const_cast for (const auto & part : data_parts_by_info) @@ -2840,9 +2840,9 @@ void MergeTreeData::dropAllData() } /// Tables in atomic databases have UUID and stored in persistent locations. - /// No need to drop caches (that are keyed by filesystem path) because collision is not possible. + /// No need to clear caches (that are keyed by filesystem path) because collision is not possible. if (!getStorageID().hasUUID()) - getContext()->dropCaches(); + getContext()->clearCaches(); /// Removing of each data part before recursive removal of directory is to speed-up removal, because there will be less number of syscalls. NameSet part_names_failed; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 87aa71f3e8d..90cf55e53b2 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -777,7 +777,7 @@ void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr num_marks_saved = 0; total_rows = 0; total_bytes = 0; - getContext()->dropMMappedFileCache(); + getContext()->clearMMappedFileCache(); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ad9013d9f13..dc23e3ea635 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -2001,7 +2001,7 @@ PartitionCommandsResultInfo StorageMergeTree::attachPartition( } /// New parts with other data may appear in place of deleted parts. - local_context->dropCaches(); + local_context->clearCaches(); return results; } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 0bfef5ed5e5..83336cbd22e 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -426,7 +426,7 @@ void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont num_indices_saved = 0; total_rows = 0; total_bytes = 0; - getContext()->dropMMappedFileCache(); + getContext()->clearMMappedFileCache(); } From 54e54aed7489915a97314ac54c16c398a7cd0efc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 29 Jun 2023 10:19:40 +0000 Subject: [PATCH 532/777] Cosmetics: Introduce constants for default cache sizes and policies --- programs/local/LocalServer.cpp | 31 ++++++------------- programs/server/Server.cpp | 18 +++-------- src/Common/CacheBase.h | 5 +-- src/Core/Defines.h | 16 ++++++++++ src/Core/ServerSettings.h | 15 ++++----- src/Interpreters/Cache/QueryCache.cpp | 8 ++--- src/Interpreters/Context.cpp | 14 ++++----- .../JIT/CompiledExpressionCache.h | 2 +- 8 files changed, 54 insertions(+), 55 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 587c88a2745..b5c7c790a83 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -655,43 +655,32 @@ void LocalServer::processConfig() /// There is no need for concurrent queries, override max_concurrent_queries. global_context->getProcessList().setMaxSize(0); - /// Size of cache for uncompressed blocks. Zero means disabled. - String uncompressed_cache_policy = config().getString("uncompressed_cache_policy", ""); - size_t uncompressed_cache_size = config().getUInt64("uncompressed_cache_size", 0); + String uncompressed_cache_policy = config().getString("uncompressed_cache_policy", DEFAULT_UNCOMPRESSED_CACHE_POLICY); + size_t uncompressed_cache_size = config().getUInt64("uncompressed_cache_size", DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE); if (uncompressed_cache_size) global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size); - /// Size of cache for marks (index of MergeTree family of tables). - String mark_cache_policy = config().getString("mark_cache_policy", ""); - size_t mark_cache_size = config().getUInt64("mark_cache_size", 5368709120); + String mark_cache_policy = config().getString("mark_cache_policy", DEFAULT_MARK_CACHE_POLICY); + size_t mark_cache_size = config().getUInt64("mark_cache_size", DEFAULT_MARK_CACHE_MAX_SIZE); if (mark_cache_size) global_context->setMarkCache(mark_cache_policy, mark_cache_size); - /// Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled. - size_t index_uncompressed_cache_size = config().getUInt64("index_uncompressed_cache_size", 0); + size_t index_uncompressed_cache_size = config().getUInt64("index_uncompressed_cache_size", DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE); if (index_uncompressed_cache_size) global_context->setIndexUncompressedCache(index_uncompressed_cache_size); - /// Size of cache for index marks (index of MergeTree skip indices). - size_t index_mark_cache_size = config().getUInt64("index_mark_cache_size", 0); + size_t index_mark_cache_size = config().getUInt64("index_mark_cache_size", DEFAULT_INDEX_MARK_CACHE_MAX_SIZE); if (index_mark_cache_size) global_context->setIndexMarkCache(index_mark_cache_size); - /// A cache for mmapped files. - size_t mmap_cache_size = config().getUInt64("mmap_cache_size", 1000); /// The choice of default is arbitrary. + size_t mmap_cache_size = config().getUInt64("mmap_cache_size", DEFAULT_MMAP_CACHE_MAX_SIZE); if (mmap_cache_size) global_context->setMMappedFileCache(mmap_cache_size); #if USE_EMBEDDED_COMPILER - /// 128 MB - constexpr size_t compiled_expression_cache_size_default = 1024 * 1024 * 128; - size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", compiled_expression_cache_size_default); - - constexpr size_t compiled_expression_cache_elements_size_default = 10000; - size_t compiled_expression_cache_elements_size - = config().getUInt64("compiled_expression_cache_elements_size", compiled_expression_cache_elements_size_default); - - CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_size, compiled_expression_cache_elements_size); + size_t compiled_expression_cache_max_size_in_bytes = config().getUInt64("compiled_expression_cache_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE); + size_t compiled_expression_cache_max_elements = config().getUInt64("compiled_expression_cache_elements_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_ENTRIES); + CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_max_size_in_bytes, compiled_expression_cache_max_elements); #endif /// NOTE: it is important to apply any overrides before diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 6237186e3a6..e5604ca422d 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1488,13 +1488,11 @@ try size_t max_cache_size = static_cast(memory_amount * server_settings.cache_size_to_ram_max_ratio); String uncompressed_cache_policy = server_settings.uncompressed_cache_policy; - LOG_INFO(log, "Uncompressed cache policy name {}", uncompressed_cache_policy); size_t uncompressed_cache_size = server_settings.uncompressed_cache_size; if (uncompressed_cache_size > max_cache_size) { uncompressed_cache_size = max_cache_size; - LOG_INFO(log, "Uncompressed cache size was lowered to {} because the system has low amount of memory", - formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + LOG_INFO(log, "Uncompressed cache size was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); } global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size); @@ -1520,8 +1518,7 @@ try if (mark_cache_size > max_cache_size) { mark_cache_size = max_cache_size; - LOG_INFO(log, "Mark cache size was lowered to {} because the system has low amount of memory", - formatReadableSizeWithBinarySuffix(mark_cache_size)); + LOG_INFO(log, "Mark cache size was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(mark_cache_size)); } global_context->setMarkCache(mark_cache_policy, mark_cache_size); @@ -1538,14 +1535,9 @@ try global_context->setQueryCache(config()); #if USE_EMBEDDED_COMPILER - /// 128 MB - constexpr size_t compiled_expression_cache_size_default = 1024 * 1024 * 128; - size_t compiled_expression_cache_size = config().getUInt64("compiled_expression_cache_size", compiled_expression_cache_size_default); - - constexpr size_t compiled_expression_cache_elements_size_default = 10000; - size_t compiled_expression_cache_elements_size = config().getUInt64("compiled_expression_cache_elements_size", compiled_expression_cache_elements_size_default); - - CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_size, compiled_expression_cache_elements_size); + size_t compiled_expression_cache_max_size_in_bytes = config().getUInt64("compiled_expression_cache_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE); + size_t compiled_expression_cache_max_elements = config().getUInt64("compiled_expression_cache_elements_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_ENTRIES); + CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_max_size_in_bytes, compiled_expression_cache_max_elements); #endif /// Set path for format schema files diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index 84cbd5b5c6f..aa7b3ea10cf 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -51,10 +51,11 @@ public: { auto on_weight_loss_function = [&](size_t weight_loss) { onRemoveOverflowWeightLoss(weight_loss); }; - static constexpr std::string_view default_cache_policy = "SLRU"; - if (cache_policy_name.empty()) + { + static constexpr auto default_cache_policy = "SLRU"; cache_policy_name = default_cache_policy; + } if (cache_policy_name == "LRU") { diff --git a/src/Core/Defines.h b/src/Core/Defines.h index efe14b93a3d..3039f0a67cf 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -1,6 +1,7 @@ #pragma once #include +#include #define DBMS_DEFAULT_PORT 9000 #define DBMS_DEFAULT_SECURE_PORT 9440 @@ -64,6 +65,21 @@ /// Max depth of hierarchical dictionary #define DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH 1000 +/// Default maximum (total and entry) sizes and policies of various caches +static constexpr auto DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE = 0_MiB; +static constexpr auto DEFAULT_UNCOMPRESSED_CACHE_POLICY = "SLRU"; +static constexpr auto DEFAULT_MARK_CACHE_MAX_SIZE = 5368_MiB; +static constexpr auto DEFAULT_MARK_CACHE_POLICY = "SLRU"; +static constexpr auto DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE = 0_MiB; +static constexpr auto DEFAULT_INDEX_MARK_CACHE_MAX_SIZE = 0_MiB; +static constexpr auto DEFAULT_MMAP_CACHE_MAX_SIZE = 1_KiB; /// chosen by rolling dice +static constexpr auto DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE = 128_MiB; +static constexpr auto DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_ENTRIES = 10'000; +static constexpr auto DEFAULT_QUERY_CACHE_MAX_SIZE = 1_GiB; +static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRIES = 1024uz; +static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_BYTES = 1_MiB; +static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_ROWS = 30'000'000uz; + /// Query profiler cannot work with sanitizers. /// Sanitizers are using quick "frame walking" stack unwinding (this implies -fno-omit-frame-pointer) /// And they do unwinding frequently (on every malloc/free, thread/mutex operations, etc). diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 7678e8c3f24..f759cd150a8 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -2,6 +2,7 @@ #include +#include namespace Poco::Util @@ -56,13 +57,13 @@ namespace DB M(UInt64, max_concurrent_select_queries, 0, "Limit on total number of concurrently select queries. Zero means Unlimited.", 0) \ \ M(Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro ram max ratio. Allows to lower cache size on low-memory systems.", 0) \ - M(String, uncompressed_cache_policy, "SLRU", "Uncompressed cache policy name.", 0) \ - M(UInt64, uncompressed_cache_size, 0, "Size of cache for uncompressed blocks. Zero means disabled.", 0) \ - M(UInt64, mark_cache_size, 5368709120, "Size of cache for marks (index of MergeTree family of tables).", 0) \ - M(String, mark_cache_policy, "SLRU", "Mark cache policy name.", 0) \ - M(UInt64, index_uncompressed_cache_size, 0, "Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled.", 0) \ - M(UInt64, index_mark_cache_size, 0, "Size of cache for index marks. Zero means disabled.", 0) \ - M(UInt64, mmap_cache_size, 1000, "A cache for mmapped files.", 0) /* The choice of default is arbitrary. */ \ + M(String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0) \ + M(UInt64, uncompressed_cache_size, DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks. Zero means disabled.", 0) \ + M(UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0) \ + M(String, mark_cache_policy, DEFAULT_MARK_CACHE_POLICY, "Mark cache policy name.", 0) \ + M(UInt64, index_uncompressed_cache_size, DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled.", 0) \ + M(UInt64, index_mark_cache_size, DEFAULT_INDEX_MARK_CACHE_MAX_SIZE, "Size of cache for index marks. Zero means disabled.", 0) \ + M(UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0) \ \ M(Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0) \ M(Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0) \ diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 5982a5ade50..182a186d4e1 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -531,14 +531,14 @@ void QueryCache::updateConfiguration(const Poco::Util::AbstractConfiguration & c { std::lock_guard lock(mutex); - size_t max_size_in_bytes = config.getUInt64("query_cache.max_size_in_bytes", 1_GiB); + size_t max_size_in_bytes = config.getUInt64("query_cache.max_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_SIZE); cache.setMaxSize(max_size_in_bytes); - size_t max_entries = config.getUInt64("query_cache.max_entries", 1024); + size_t max_entries = config.getUInt64("query_cache.max_entries", DEFAULT_QUERY_CACHE_MAX_ENTRIES); cache.setMaxCount(max_entries); - max_entry_size_in_bytes = config.getUInt64("query_cache.max_entry_size_in_bytes", 1_MiB); - max_entry_size_in_rows = config.getUInt64("query_cache.max_entry_rows_in_rows", 30'000'000); + max_entry_size_in_bytes = config.getUInt64("query_cache.max_entry_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_BYTES); + max_entry_size_in_rows = config.getUInt64("query_cache.max_entry_rows_in_rows", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_ROWS); } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 39284b4c1b2..b77b4b79cb9 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -245,9 +245,9 @@ struct ContextSharedPart : boost::noncopyable std::optional backups_worker; - String default_profile_name; /// Default profile name used for default values. - String system_profile_name; /// Profile used by system processes - String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying + String default_profile_name; /// Default profile name used for default values. + String system_profile_name; /// Profile used by system processes + String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying std::unique_ptr access_control; mutable ResourceManagerPtr resource_manager; mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. @@ -261,11 +261,11 @@ struct ContextSharedPart : boost::noncopyable ProcessList process_list; /// Executing queries at the moment. SessionTracker session_tracker; GlobalOvercommitTracker global_overcommit_tracker; - MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) - MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) + MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) + MovesList moves_list; /// The list of executing moves (for (Replicated)?MergeTree) ReplicatedFetchList replicated_fetch_list; - ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections. - InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. + ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections. + InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. mutable std::unique_ptr buffer_flush_schedule_pool; /// A thread pool that can do background flush for Buffer tables. mutable std::unique_ptr schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables) diff --git a/src/Interpreters/JIT/CompiledExpressionCache.h b/src/Interpreters/JIT/CompiledExpressionCache.h index 21f7c67226c..8354b33bae9 100644 --- a/src/Interpreters/JIT/CompiledExpressionCache.h +++ b/src/Interpreters/JIT/CompiledExpressionCache.h @@ -19,7 +19,7 @@ public: size_t getCompiledExpressionSize() const { return compiled_expression_size; } - virtual ~CompiledExpressionCacheEntry() {} + virtual ~CompiledExpressionCacheEntry() = default; private: From 9e4b4541bc34a58bbc564d63fec4e0710eb45e33 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 15:13:54 +0000 Subject: [PATCH 533/777] Cosmetics: More aesthetic code --- src/Interpreters/Context.cpp | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index b77b4b79cb9..0cc3750dc26 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -250,15 +250,15 @@ struct ContextSharedPart : boost::noncopyable String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying std::unique_ptr access_control; mutable ResourceManagerPtr resource_manager; - mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. - mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files. - mutable std::unique_ptr load_marks_threadpool; /// Threadpool for loading marks cache. - mutable std::unique_ptr prefetch_threadpool; /// Threadpool for loading marks cache. - mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices. - mutable MarkCachePtr index_mark_cache; /// Cache of marks in compressed files of MergeTree indices. - mutable QueryCachePtr query_cache; /// Cache of query results. - mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. - ProcessList process_list; /// Executing queries at the moment. + mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. + mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files. + mutable std::unique_ptr load_marks_threadpool; /// Threadpool for loading marks cache. + mutable std::unique_ptr prefetch_threadpool; /// Threadpool for loading marks cache. + mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices. + mutable QueryCachePtr query_cache; /// Cache of query results. + mutable MarkCachePtr index_mark_cache; /// Cache of marks in compressed files of MergeTree indices. + mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. + ProcessList process_list; /// Executing queries at the moment. SessionTracker session_tracker; GlobalOvercommitTracker global_overcommit_tracker; MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) From f126e54b0d76e21f93104d42b3620dfa36fb436a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 29 Jun 2023 18:33:48 +0000 Subject: [PATCH 534/777] Respect max_cache_size for all caches So far, max_cache_size was only respected in Server.cpp, and in this file only for the mark cache and the uncompressed cache. This commit respects it also for the other cache types in Server.cpp, as well for all caches in LocalServer.cpp. --- programs/local/LocalServer.cpp | 31 +++++++++++++++++++++++++++++ programs/server/Server.cpp | 36 +++++++++++++++++++++++++--------- 2 files changed, 58 insertions(+), 9 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index b5c7c790a83..d97a4760f49 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -2,6 +2,8 @@ #include #include +#include +#include #include #include #include @@ -655,25 +657,54 @@ void LocalServer::processConfig() /// There is no need for concurrent queries, override max_concurrent_queries. global_context->getProcessList().setMaxSize(0); + const size_t memory_amount = getMemoryAmount(); + const double cache_size_to_ram_max_ratio = config().getDouble("cache_size_to_ram_max_ratio", 0.5); + const size_t max_cache_size = static_cast(memory_amount * cache_size_to_ram_max_ratio); + String uncompressed_cache_policy = config().getString("uncompressed_cache_policy", DEFAULT_UNCOMPRESSED_CACHE_POLICY); size_t uncompressed_cache_size = config().getUInt64("uncompressed_cache_size", DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE); + if (uncompressed_cache_size > max_cache_size) + { + uncompressed_cache_size = max_cache_size; + LOG_INFO(log, "Lowered uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } if (uncompressed_cache_size) global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size); String mark_cache_policy = config().getString("mark_cache_policy", DEFAULT_MARK_CACHE_POLICY); size_t mark_cache_size = config().getUInt64("mark_cache_size", DEFAULT_MARK_CACHE_MAX_SIZE); + if (mark_cache_size > max_cache_size) + { + mark_cache_size = max_cache_size; + LOG_INFO(log, "Lowered mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(mark_cache_size)); + } if (mark_cache_size) global_context->setMarkCache(mark_cache_policy, mark_cache_size); size_t index_uncompressed_cache_size = config().getUInt64("index_uncompressed_cache_size", DEFAULT_INDEX_UNCOMPRESSED_CACHE_MAX_SIZE); + if (index_uncompressed_cache_size > max_cache_size) + { + index_uncompressed_cache_size = max_cache_size; + LOG_INFO(log, "Lowered index uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } if (index_uncompressed_cache_size) global_context->setIndexUncompressedCache(index_uncompressed_cache_size); size_t index_mark_cache_size = config().getUInt64("index_mark_cache_size", DEFAULT_INDEX_MARK_CACHE_MAX_SIZE); + if (index_mark_cache_size > max_cache_size) + { + index_mark_cache_size = max_cache_size; + LOG_INFO(log, "Lowered index mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } if (index_mark_cache_size) global_context->setIndexMarkCache(index_mark_cache_size); size_t mmap_cache_size = config().getUInt64("mmap_cache_size", DEFAULT_MMAP_CACHE_MAX_SIZE); + if (mmap_cache_size > max_cache_size) + { + mmap_cache_size = max_cache_size; + LOG_INFO(log, "Lowered mmap file cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } if (mmap_cache_size) global_context->setMMappedFileCache(mmap_cache_size); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index e5604ca422d..d5ae2513fa0 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -658,7 +659,7 @@ try global_context->addWarningMessage("Server was built with sanitizer. It will work slowly."); #endif - const auto memory_amount = getMemoryAmount(); + const size_t memory_amount = getMemoryAmount(); LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.", formatReadableSizeWithBinarySuffix(memory_amount), @@ -1485,14 +1486,14 @@ try /// Set up caches. - size_t max_cache_size = static_cast(memory_amount * server_settings.cache_size_to_ram_max_ratio); + const size_t max_cache_size = static_cast(memory_amount * server_settings.cache_size_to_ram_max_ratio); String uncompressed_cache_policy = server_settings.uncompressed_cache_policy; size_t uncompressed_cache_size = server_settings.uncompressed_cache_size; if (uncompressed_cache_size > max_cache_size) { uncompressed_cache_size = max_cache_size; - LOG_INFO(log, "Uncompressed cache size was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + LOG_INFO(log, "Lowered uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); } global_context->setUncompressedCache(uncompressed_cache_policy, uncompressed_cache_size); @@ -1511,27 +1512,44 @@ try server_settings.async_insert_queue_flush_on_shutdown)); } - size_t mark_cache_size = server_settings.mark_cache_size; String mark_cache_policy = server_settings.mark_cache_policy; + size_t mark_cache_size = server_settings.mark_cache_size; if (!mark_cache_size) LOG_ERROR(log, "Too low mark cache size will lead to severe performance degradation."); if (mark_cache_size > max_cache_size) { mark_cache_size = max_cache_size; - LOG_INFO(log, "Mark cache size was lowered to {} because the system has low amount of memory", formatReadableSizeWithBinarySuffix(mark_cache_size)); + LOG_INFO(log, "Lowered mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(mark_cache_size)); } global_context->setMarkCache(mark_cache_policy, mark_cache_size); - if (server_settings.index_uncompressed_cache_size) + size_t index_uncompressed_cache_size = server_settings.index_uncompressed_cache_size; + if (index_uncompressed_cache_size > max_cache_size) + { + index_uncompressed_cache_size = max_cache_size; + LOG_INFO(log, "Lowered index uncompressed cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } + if (index_uncompressed_cache_size) global_context->setIndexUncompressedCache(server_settings.index_uncompressed_cache_size); - if (server_settings.index_mark_cache_size) + size_t index_mark_cache_size = server_settings.index_mark_cache_size; + if (index_mark_cache_size > max_cache_size) + { + index_mark_cache_size = max_cache_size; + LOG_INFO(log, "Lowered index mark cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } + if (index_mark_cache_size) global_context->setIndexMarkCache(server_settings.index_mark_cache_size); - if (server_settings.mmap_cache_size) + size_t mmap_cache_size = server_settings.mmap_cache_size; + if (mmap_cache_size > max_cache_size) + { + mmap_cache_size = max_cache_size; + LOG_INFO(log, "Lowered mmap file cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } + if (mmap_cache_size) global_context->setMMappedFileCache(server_settings.mmap_cache_size); - /// A cache for query results. global_context->setQueryCache(config()); #if USE_EMBEDDED_COMPILER From bb53d635dfead645fc246b8f38bef99239290014 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 29 Jun 2023 18:47:03 +0000 Subject: [PATCH 535/777] More consistency: Run same mark cache size check in ch-local Server.cpp had a check that the configured mark cache size is not 0. Run the same check in LocalServer.cpp. --- programs/local/LocalServer.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index d97a4760f49..5e2c8f927bc 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -673,6 +673,8 @@ void LocalServer::processConfig() String mark_cache_policy = config().getString("mark_cache_policy", DEFAULT_MARK_CACHE_POLICY); size_t mark_cache_size = config().getUInt64("mark_cache_size", DEFAULT_MARK_CACHE_MAX_SIZE); + if (!mark_cache_size) + LOG_ERROR(log, "Too low mark cache size will lead to severe performance degradation."); if (mark_cache_size > max_cache_size) { mark_cache_size = max_cache_size; From ed90463888742f60939650480104b96242f19996 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 29 Jun 2023 18:55:09 +0000 Subject: [PATCH 536/777] Add comment --- programs/local/LocalServer.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 5e2c8f927bc..0dea7e8e643 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -710,6 +710,8 @@ void LocalServer::processConfig() if (mmap_cache_size) global_context->setMMappedFileCache(mmap_cache_size); + /// not initializing the query cache in clickhouse-local + #if USE_EMBEDDED_COMPILER size_t compiled_expression_cache_max_size_in_bytes = config().getUInt64("compiled_expression_cache_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE); size_t compiled_expression_cache_max_elements = config().getUInt64("compiled_expression_cache_elements_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_ENTRIES); From 7ed5166c8aca62e7b9397b8b108afa6b26098bde Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 29 Jun 2023 10:56:32 +0000 Subject: [PATCH 537/777] Cosmetics: Make re-cfg of query cache similar to re-cfg of other caches This doesn't change semantics, it only changes in which places the configuration is parsed. This makes the logic more consistent across caches. --- programs/server/Server.cpp | 11 ++++++++++- src/Interpreters/Cache/QueryCache.cpp | 17 ++++++----------- src/Interpreters/Cache/QueryCache.h | 13 +++++-------- src/Interpreters/Context.cpp | 13 +++++++++---- src/Interpreters/Context.h | 2 +- 5 files changed, 31 insertions(+), 25 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index d5ae2513fa0..b823cbbe367 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1550,7 +1550,16 @@ try if (mmap_cache_size) global_context->setMMappedFileCache(server_settings.mmap_cache_size); - global_context->setQueryCache(config()); + size_t query_cache_max_size_in_bytes = config().getUInt64("query_cache.max_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_SIZE); + size_t query_cache_max_entries = config().getUInt64("query_cache.max_entries", DEFAULT_QUERY_CACHE_MAX_ENTRIES); + size_t query_cache_query_cache_max_entry_size_in_bytes = config().getUInt64("query_cache.max_entry_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_BYTES); + size_t query_cache_max_entry_size_in_rows = config().getUInt64("query_cache.max_entry_rows_in_rows", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_ROWS); + if (query_cache_max_size_in_bytes > max_cache_size) + { + query_cache_max_size_in_bytes = max_cache_size; + LOG_INFO(log, "Lowered query cache size to {} because the system has limited RAM", formatReadableSizeWithBinarySuffix(uncompressed_cache_size)); + } + global_context->setQueryCache(query_cache_max_size_in_bytes, query_cache_max_entries, query_cache_query_cache_max_entry_size_in_bytes, query_cache_max_entry_size_in_rows); #if USE_EMBEDDED_COMPILER size_t compiled_expression_cache_max_size_in_bytes = config().getUInt64("compiled_expression_cache_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE); diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 182a186d4e1..134aa0956d1 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -493,7 +493,6 @@ void QueryCache::reset() cache.reset(); std::lock_guard lock(mutex); times_executed.clear(); - cache_size_in_bytes = 0; } size_t QueryCache::weight() const @@ -511,7 +510,7 @@ size_t QueryCache::recordQueryRun(const Key & key) std::lock_guard lock(mutex); size_t times = ++times_executed[key]; // Regularly drop times_executed to avoid DOS-by-unlimited-growth. - static constexpr size_t TIMES_EXECUTED_MAX_SIZE = 10'000; + static constexpr auto TIMES_EXECUTED_MAX_SIZE = 10'000uz; if (times_executed.size() > TIMES_EXECUTED_MAX_SIZE) times_executed.clear(); return times; @@ -522,23 +521,19 @@ std::vector QueryCache::dump() const return cache.dump(); } -QueryCache::QueryCache() +QueryCache::QueryCache(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes_, size_t max_entry_size_in_rows_) : cache(std::make_unique>(std::make_unique())) { + updateConfiguration(max_size_in_bytes, max_entries, max_entry_size_in_bytes_, max_entry_size_in_rows_); } -void QueryCache::updateConfiguration(const Poco::Util::AbstractConfiguration & config) +void QueryCache::updateConfiguration(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes_, size_t max_entry_size_in_rows_) { std::lock_guard lock(mutex); - - size_t max_size_in_bytes = config.getUInt64("query_cache.max_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_SIZE); cache.setMaxSize(max_size_in_bytes); - - size_t max_entries = config.getUInt64("query_cache.max_entries", DEFAULT_QUERY_CACHE_MAX_ENTRIES); cache.setMaxCount(max_entries); - - max_entry_size_in_bytes = config.getUInt64("query_cache.max_entry_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_BYTES); - max_entry_size_in_rows = config.getUInt64("query_cache.max_entry_rows_in_rows", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_ROWS); + max_entry_size_in_bytes = max_entry_size_in_bytes_; + max_entry_size_in_rows = max_entry_size_in_rows_; } } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index eaa54c503fa..0c0674c6302 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -4,7 +4,6 @@ #include #include #include -#include #include #include @@ -110,9 +109,6 @@ private: /// query --> query result using Cache = CacheBase; - /// query --> query execution count - using TimesExecuted = std::unordered_map; - public: /// Buffers multiple partial query result chunks (buffer()) and eventually stores them as cache entry (finalizeWrite()). /// @@ -177,9 +173,9 @@ public: friend class QueryCache; /// for createReader() }; - QueryCache(); + QueryCache(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes_, size_t max_entry_size_in_rows_); - void updateConfiguration(const Poco::Util::AbstractConfiguration & config); + void updateConfiguration(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes_, size_t max_entry_size_in_rows_); Reader createReader(const Key & key); Writer createWriter(const Key & key, std::chrono::milliseconds min_query_runtime, bool squash_partial_results, size_t max_block_size, size_t max_query_cache_size_in_bytes_quota, size_t max_query_cache_entries_quota); @@ -199,14 +195,15 @@ private: Cache cache; /// has its own locking --> not protected by mutex mutable std::mutex mutex; + + /// query --> query execution count + using TimesExecuted = std::unordered_map; TimesExecuted times_executed TSA_GUARDED_BY(mutex); /// Cache configuration size_t max_entry_size_in_bytes TSA_GUARDED_BY(mutex) = 0; size_t max_entry_size_in_rows TSA_GUARDED_BY(mutex) = 0; - size_t cache_size_in_bytes TSA_GUARDED_BY(mutex) = 0; /// Updated in each cache insert/delete - friend class StorageSystemQueryCache; }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 0cc3750dc26..0886d699734 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2386,22 +2386,27 @@ void Context::clearMMappedFileCache() const shared->mmap_cache->reset(); } -void Context::setQueryCache(const Poco::Util::AbstractConfiguration & config) +void Context::setQueryCache(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes, size_t max_entry_size_in_rows) { auto lock = getLock(); if (shared->query_cache) throw Exception(ErrorCodes::LOGICAL_ERROR, "Query cache has been already created."); - shared->query_cache = std::make_shared(); - shared->query_cache->updateConfiguration(config); + shared->query_cache = std::make_shared(max_size_in_bytes, max_entries, max_entry_size_in_bytes, max_entry_size_in_rows); } void Context::updateQueryCacheConfiguration(const Poco::Util::AbstractConfiguration & config) { auto lock = getLock(); if (shared->query_cache) - shared->query_cache->updateConfiguration(config); + { + size_t max_size_in_bytes = config.getUInt64("query_cache.max_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_SIZE); + size_t max_entries = config.getUInt64("query_cache.max_entries", DEFAULT_QUERY_CACHE_MAX_ENTRIES); + size_t max_entry_size_in_bytes = config.getUInt64("query_cache.max_entry_size_in_bytes", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_BYTES); + size_t max_entry_size_in_rows = config.getUInt64("query_cache.max_entry_rows_in_rows", DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_ROWS); + shared->query_cache->updateConfiguration(max_size_in_bytes, max_entries, max_entry_size_in_bytes, max_entry_size_in_rows); + } } QueryCachePtr Context::getQueryCache() const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index c725d032bbe..587fe402d4e 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -944,7 +944,7 @@ public: void clearMMappedFileCache() const; /// Create a cache of query results for statements which run repeatedly. - void setQueryCache(const Poco::Util::AbstractConfiguration & config); + void setQueryCache(size_t max_size_in_bytes, size_t max_entries, size_t max_entry_size_in_bytes, size_t max_entry_size_in_rows); void updateQueryCacheConfiguration(const Poco::Util::AbstractConfiguration & config); std::shared_ptr getQueryCache() const; void clearQueryCache() const; From c6dc7a8a0bc1aaffeaf3d967f260c8630fb52154 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 14 Aug 2023 16:04:58 +0000 Subject: [PATCH 538/777] Update test --- tests/queries/0_stateless/02404_memory_bound_merging.reference | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.reference b/tests/queries/0_stateless/02404_memory_bound_merging.reference index d9fac433189..41a3b6bf8ec 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.reference +++ b/tests/queries/0_stateless/02404_memory_bound_merging.reference @@ -118,8 +118,7 @@ ExpressionTransform MergingAggregatedBucketTransform × 4 Resize 1 → 4 GroupingAggregatedTransform 3 → 1 - (Union) - (ReadFromRemoteParallelReplicas) + (ReadFromRemoteParallelReplicas) select a, count() from pr_t group by a order by a limit 5 offset 500; 500 1000 501 1000 From 9dafc596d06ece75d1c53bfc287159b8ed849033 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Sat, 12 Aug 2023 01:04:08 +0200 Subject: [PATCH 539/777] Analyzer: fix quotas for system tables --- .../InterpreterSelectQueryAnalyzer.cpp | 2 +- src/Interpreters/executeQuery.cpp | 6 ++++ src/Planner/Planner.cpp | 6 ++-- src/Planner/Planner.h | 8 ++--- src/Planner/PlannerJoinTree.cpp | 26 ++++++++++++++- src/Planner/PlannerJoinTree.h | 2 +- tests/analyzer_integration_broken_tests.txt | 32 ------------------- 7 files changed, 40 insertions(+), 42 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 8db1d27c073..b8cace5e0ad 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -184,7 +184,7 @@ InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( , context(buildContext(context_, select_query_options_)) , select_query_options(select_query_options_) , query_tree(query_tree_) - , planner(query_tree_, select_query_options_) + , planner(query_tree_, select_query_options) { } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 578ca3b41f9..597c5bda245 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -45,6 +45,7 @@ #include #include #include +#include #include #include #include @@ -1033,6 +1034,11 @@ static std::tuple executeQueryImpl( } + // InterpreterSelectQueryAnalyzer does not build QueryPlan in the constructor. + // We need to force to build it here to check if we need to ingore quota. + if (auto * interpreter_with_analyzer = dynamic_cast(interpreter.get())) + interpreter_with_analyzer->getQueryPlan(); + if (!interpreter->ignoreQuota() && !quota_checked) { quota = context->getQuota(); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 9f6c22f90f3..7cce495dfb8 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1047,7 +1047,7 @@ PlannerContextPtr buildPlannerContext(const QueryTreeNodePtr & query_tree_node, } Planner::Planner(const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_) + SelectQueryOptions & select_query_options_) : query_tree(query_tree_) , select_query_options(select_query_options_) , planner_context(buildPlannerContext(query_tree, select_query_options, std::make_shared())) @@ -1055,7 +1055,7 @@ Planner::Planner(const QueryTreeNodePtr & query_tree_, } Planner::Planner(const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_, + SelectQueryOptions & select_query_options_, GlobalPlannerContextPtr global_planner_context_) : query_tree(query_tree_) , select_query_options(select_query_options_) @@ -1064,7 +1064,7 @@ Planner::Planner(const QueryTreeNodePtr & query_tree_, } Planner::Planner(const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_, + SelectQueryOptions & select_query_options_, PlannerContextPtr planner_context_) : query_tree(query_tree_) , select_query_options(select_query_options_) diff --git a/src/Planner/Planner.h b/src/Planner/Planner.h index 783a07f6e99..f8d151365cf 100644 --- a/src/Planner/Planner.h +++ b/src/Planner/Planner.h @@ -22,16 +22,16 @@ class Planner public: /// Initialize planner with query tree after analysis phase Planner(const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_); + SelectQueryOptions & select_query_options_); /// Initialize planner with query tree after query analysis phase and global planner context Planner(const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_, + SelectQueryOptions & select_query_options_, GlobalPlannerContextPtr global_planner_context_); /// Initialize planner with query tree after query analysis phase and planner context Planner(const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_, + SelectQueryOptions & select_query_options_, PlannerContextPtr planner_context_); const QueryPlan & getQueryPlan() const @@ -66,7 +66,7 @@ private: void buildPlanForQueryNode(); QueryTreeNodePtr query_tree; - SelectQueryOptions select_query_options; + SelectQueryOptions & select_query_options; PlannerContextPtr planner_context; QueryPlan query_plan; StorageLimitsList storage_limits; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 56a48ce8328..11de6fcfabe 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -113,6 +113,20 @@ void checkAccessRights(const TableNode & table_node, const Names & column_names, query_context->checkAccess(AccessType::SELECT, storage_id, column_names); } +bool shouldIgnoreQuotaAndLimits(const TableNode & table_node) +{ + const auto & storage_id = table_node.getStorageID(); + if (!storage_id.hasDatabase()) + return false; + if (storage_id.database_name == DatabaseCatalog::SYSTEM_DATABASE) + { + static const boost::container::flat_set tables_ignoring_quota{"quotas", "quota_limits", "quota_usage", "quotas_usage", "one"}; + if (tables_ignoring_quota.count(storage_id.table_name)) + return true; + } + return false; +} + NameAndTypePair chooseSmallestColumnToReadFromStorage(const StoragePtr & storage, const StorageSnapshotPtr & storage_snapshot) { /** We need to read at least one column to find the number of rows. @@ -1375,7 +1389,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node, const SelectQueryInfo & select_query_info, - const SelectQueryOptions & select_query_options, + SelectQueryOptions & select_query_options, const ColumnIdentifierSet & outer_scope_columns, PlannerContextPtr & planner_context) { @@ -1386,6 +1400,16 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node, std::vector table_expressions_outer_scope_columns(table_expressions_stack_size); ColumnIdentifierSet current_outer_scope_columns = outer_scope_columns; + if (is_single_table_expression) + { + auto * table_node = table_expressions_stack[0]->as(); + if (table_node && shouldIgnoreQuotaAndLimits(*table_node)) + { + select_query_options.ignore_quota = true; + select_query_options.ignore_limits = true; + } + } + /// For each table, table function, query, union table expressions prepare before query plan build for (size_t i = 0; i < table_expressions_stack_size; ++i) { diff --git a/src/Planner/PlannerJoinTree.h b/src/Planner/PlannerJoinTree.h index acbc96ddae0..9d3b98175d0 100644 --- a/src/Planner/PlannerJoinTree.h +++ b/src/Planner/PlannerJoinTree.h @@ -20,7 +20,7 @@ struct JoinTreeQueryPlan /// Build JOIN TREE query plan for query node JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node, const SelectQueryInfo & select_query_info, - const SelectQueryOptions & select_query_options, + SelectQueryOptions & select_query_options, const ColumnIdentifierSet & outer_scope_columns, PlannerContextPtr & planner_context); diff --git a/tests/analyzer_integration_broken_tests.txt b/tests/analyzer_integration_broken_tests.txt index 68822fbf311..b485f3f60cc 100644 --- a/tests/analyzer_integration_broken_tests.txt +++ b/tests/analyzer_integration_broken_tests.txt @@ -96,22 +96,6 @@ test_executable_table_function/test.py::test_executable_function_input_python test_settings_profile/test.py::test_show_profiles test_sql_user_defined_functions_on_cluster/test.py::test_sql_user_defined_functions_on_cluster test_postgresql_protocol/test.py::test_python_client -test_quota/test.py::test_add_remove_interval -test_quota/test.py::test_add_remove_quota -test_quota/test.py::test_consumption_of_show_clusters -test_quota/test.py::test_consumption_of_show_databases -test_quota/test.py::test_consumption_of_show_privileges -test_quota/test.py::test_consumption_of_show_processlist -test_quota/test.py::test_consumption_of_show_tables -test_quota/test.py::test_dcl_introspection -test_quota/test.py::test_dcl_management -test_quota/test.py::test_exceed_quota -test_quota/test.py::test_query_inserts -test_quota/test.py::test_quota_from_users_xml -test_quota/test.py::test_reload_users_xml_by_timer -test_quota/test.py::test_simpliest_quota -test_quota/test.py::test_tracking_quota -test_quota/test.py::test_users_xml_is_readonly test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database test_profile_events_s3/test.py::test_profile_events test_user_defined_object_persistence/test.py::test_persistence @@ -121,22 +105,6 @@ test_select_access_rights/test_main.py::test_alias_columns test_select_access_rights/test_main.py::test_select_count test_select_access_rights/test_main.py::test_select_join test_postgresql_protocol/test.py::test_python_client -test_quota/test.py::test_add_remove_interval -test_quota/test.py::test_add_remove_quota -test_quota/test.py::test_consumption_of_show_clusters -test_quota/test.py::test_consumption_of_show_databases -test_quota/test.py::test_consumption_of_show_privileges -test_quota/test.py::test_consumption_of_show_processlist -test_quota/test.py::test_consumption_of_show_tables -test_quota/test.py::test_dcl_introspection -test_quota/test.py::test_dcl_management -test_quota/test.py::test_exceed_quota -test_quota/test.py::test_query_inserts -test_quota/test.py::test_quota_from_users_xml -test_quota/test.py::test_reload_users_xml_by_timer -test_quota/test.py::test_simpliest_quota -test_quota/test.py::test_tracking_quota -test_quota/test.py::test_users_xml_is_readonly test_replicating_constants/test.py::test_different_versions test_merge_tree_s3/test.py::test_heavy_insert_select_check_memory[node] test_wrong_db_or_table_name/test.py::test_wrong_table_name From a366c1c532d6cb176c8c4ba72e8a3ca6f5ca7f2d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Sun, 13 Aug 2023 01:04:33 +0200 Subject: [PATCH 540/777] Update src/Interpreters/executeQuery.cpp --- src/Interpreters/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 597c5bda245..a56007375f4 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1035,7 +1035,7 @@ static std::tuple executeQueryImpl( } // InterpreterSelectQueryAnalyzer does not build QueryPlan in the constructor. - // We need to force to build it here to check if we need to ingore quota. + // We need to force to build it here to check if we need to ignore quota. if (auto * interpreter_with_analyzer = dynamic_cast(interpreter.get())) interpreter_with_analyzer->getQueryPlan(); From 12448285555abc54bf14a3a35f38ced6db736b06 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 14 Aug 2023 19:27:05 +0200 Subject: [PATCH 541/777] Analyzer: fix virtual columns in StorageDistributed --- src/Storages/StorageDistributed.cpp | 6 +++++- .../0_stateless/02844_distributed_virtual_columns.reference | 0 .../0_stateless/02844_distributed_virtual_columns.sql | 5 +++++ 3 files changed, 10 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02844_distributed_virtual_columns.reference create mode 100644 tests/queries/0_stateless/02844_distributed_virtual_columns.sql diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index a7aeb11e2d8..f80e498efa8 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -691,7 +691,11 @@ QueryTreeNodePtr buildQueryTreeDistributed(SelectQueryInfo & query_info, if (remote_storage_id.hasDatabase()) resolved_remote_storage_id = query_context->resolveStorageID(remote_storage_id); - auto storage = std::make_shared(resolved_remote_storage_id, distributed_storage_snapshot->metadata->getColumns(), distributed_storage_snapshot->object_columns); + auto get_column_options = GetColumnsOptions(GetColumnsOptions::All).withExtendedObjects().withVirtuals(); + + auto column_names_and_types = distributed_storage_snapshot->getColumns(get_column_options); + + auto storage = std::make_shared(resolved_remote_storage_id, ColumnsDescription{column_names_and_types}); auto table_node = std::make_shared(std::move(storage), query_context); if (table_expression_modifiers) diff --git a/tests/queries/0_stateless/02844_distributed_virtual_columns.reference b/tests/queries/0_stateless/02844_distributed_virtual_columns.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02844_distributed_virtual_columns.sql b/tests/queries/0_stateless/02844_distributed_virtual_columns.sql new file mode 100644 index 00000000000..31a6780f19e --- /dev/null +++ b/tests/queries/0_stateless/02844_distributed_virtual_columns.sql @@ -0,0 +1,5 @@ +drop table if exists data_01072; +drop table if exists dist_01072; +create table data_01072 (key Int) Engine=MergeTree() ORDER BY key; +create table dist_01072 (key Int) Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01072, key); +select * from dist_01072 where key=0 and _part='0'; From 2c1bd7d3bcba800150812b32d7b01a2098e0cb62 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Mon, 14 Aug 2023 18:15:31 +0000 Subject: [PATCH 542/777] Remove duplicate function specialization --- src/Common/TransformEndianness.hpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index 9d34ce6ba77..7c77e918199 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -77,11 +77,4 @@ inline void transformEndianness(CityHash_v1_0_2::uint128 & x) transformEndianness(x.low64); transformEndianness(x.high64); } - -template -inline void transformEndianness(CityHash_v1_0_2::uint128 & x) -{ - transformEndianness(x.low64); - transformEndianness(x.high64); -} } From 368f6d7b1390b98ccac2610eb88a4237abcab439 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Aug 2023 20:46:41 +0200 Subject: [PATCH 543/777] fix --- src/Functions/transform.cpp | 4 ++++ tests/queries/0_stateless/02443_detach_attach_partition.sh | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index 16326dd5a44..62ab51abd76 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -776,8 +776,12 @@ namespace UInt64 key = 0; auto * dst = reinterpret_cast(&key); const auto ref = cache.from_column->getDataAt(i); + +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wunreachable-code" if constexpr (std::endian::native == std::endian::big) dst += sizeof(key) - ref.size; +#pragma clang diagnostic pop memcpy(dst, ref.data, ref.size); table[key] = i; diff --git a/tests/queries/0_stateless/02443_detach_attach_partition.sh b/tests/queries/0_stateless/02443_detach_attach_partition.sh index 36bc3309924..13ea966dbf5 100755 --- a/tests/queries/0_stateless/02443_detach_attach_partition.sh +++ b/tests/queries/0_stateless/02443_detach_attach_partition.sh @@ -55,7 +55,7 @@ wait $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table0" $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table1" -$CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table0 ATTACH PARTITION ID 'all'" +while ! $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table0 ATTACH PARTITION ID 'all'" 2>/dev/null; do sleep 0.5; done $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table1 ATTACH PARTITION ID 'all'" 2>/dev/null $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table1" $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table1 ATTACH PARTITION ID 'all'" From 1916aec2e24fca860a3ec0fcae172450f9210ed0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Aug 2023 19:06:32 +0200 Subject: [PATCH 544/777] add trash for Replicated database --- src/Interpreters/InterpreterCreateQuery.cpp | 30 ++++++++++++++++--- .../configs/config.xml | 1 + .../test_replicated_database/test.py | 2 +- 3 files changed, 28 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f4aee9ad4db..f1c01d1aadc 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -71,7 +72,6 @@ #include #include -#include #include #include @@ -1329,10 +1329,32 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, } data_path = database->getTableDataPath(create); + auto full_data_path = fs::path{getContext()->getPath()} / data_path; - if (!create.attach && !data_path.empty() && fs::exists(fs::path{getContext()->getPath()} / data_path)) - throw Exception(storage_already_exists_error_code, - "Directory for {} data {} already exists", Poco::toLower(storage_name), String(data_path)); + if (!create.attach && !data_path.empty() && fs::exists(full_data_path)) + { + if (getContext()->getZooKeeperMetadataTransaction() && + !getContext()->getZooKeeperMetadataTransaction()->isInitialQuery() && + !DatabaseCatalog::instance().hasUUIDMapping(create.uuid) && + Context::getGlobalContextInstance()->isServerCompletelyStarted() && + Context::getGlobalContextInstance()->getConfigRef().getBool("allow_moving_table_dir_to_trash", false)) + { + /// This is a secondary query from a Replicated database. It cannot be retried with another UUID, we must execute it as is. + /// We don't have a table with this UUID (and all metadata is loaded), + /// so the existing directory probably contains some leftovers from previous unsuccessful attempts to create the table + + fs::path trash_path = fs::path{getContext()->getPath()} / "trash" / data_path / getHexUIntLowercase(thread_local_rng()); + LOG_WARNING(&Poco::Logger::get("InterpreterCreateQuery"), "Directory for {} data {} already exists. Will move it to {}", + Poco::toLower(storage_name), String(data_path), trash_path); + fs::create_directories(trash_path.parent_path()); + renameNoReplace(full_data_path, trash_path); + } + else + { + throw Exception(storage_already_exists_error_code, + "Directory for {} data {} already exists", Poco::toLower(storage_name), String(data_path)); + } + } bool from_path = create.attach_from_path.has_value(); String actual_data_path = data_path; diff --git a/tests/integration/test_replicated_database/configs/config.xml b/tests/integration/test_replicated_database/configs/config.xml index 16cd942e975..7a67d69c031 100644 --- a/tests/integration/test_replicated_database/configs/config.xml +++ b/tests/integration/test_replicated_database/configs/config.xml @@ -1,5 +1,6 @@ 10 + 1 10 diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index d0a04f40b69..f45841124d9 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1262,7 +1262,7 @@ def test_recover_digest_mismatch(started_cluster): "mv /var/lib/clickhouse/metadata/recover_digest_mismatch/t1.sql /var/lib/clickhouse/metadata/recover_digest_mismatch/m1.sql", "sed --follow-symlinks -i 's/Int32/String/' /var/lib/clickhouse/metadata/recover_digest_mismatch/mv1.sql", "rm -f /var/lib/clickhouse/metadata/recover_digest_mismatch/d1.sql", - # f"rm -rf /var/lib/clickhouse/metadata/recover_digest_mismatch/", # Directory already exists + "rm -rf /var/lib/clickhouse/metadata/recover_digest_mismatch/", # Will trigger "Directory already exists" "rm -rf /var/lib/clickhouse/store", ] From 3d8f2c335feb46c7dcd16ef9ea38d1df0a81c2c2 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 14 Aug 2023 18:54:15 +0000 Subject: [PATCH 545/777] Protect against invalid asynchronous_metrics_update_period_s settings --- src/Interpreters/ServerAsynchronousMetrics.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 68411e80755..1b78ff4d2b6 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -24,6 +24,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int INVALID_SETTING_VALUE; +} + namespace { @@ -52,7 +57,11 @@ ServerAsynchronousMetrics::ServerAsynchronousMetrics( : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_) , WithContext(global_context_) , heavy_metric_update_period(heavy_metrics_update_period_seconds) -{} +{ + /// sanity check + if (update_period_seconds == 0 || heavy_metrics_update_period_seconds == 0) + throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Setting asynchronous_metrics_update_period_s and asynchronous_heavy_metrics_update_period_s must not be zero"); +} void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) { From 894ec8e0d084848a6bd9b47f191a7a879fdb56ae Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Aug 2023 18:57:40 +0200 Subject: [PATCH 546/777] less exceptions with runtime format string --- programs/keeper/Keeper.cpp | 7 ++- programs/server/Server.cpp | 9 ++- src/Backups/BackupCoordinationRemote.cpp | 6 +- src/Backups/BackupCoordinationStageSync.cpp | 6 +- src/Backups/RestoreCoordinationRemote.cpp | 12 ++-- src/Common/DNSResolver.cpp | 8 +-- src/Common/DNSResolver.h | 6 +- src/Common/DateLUTImpl.cpp | 12 +++- src/Common/NamePrompter.h | 5 ++ src/Common/ZooKeeper/IKeeper.cpp | 34 +++++----- src/Common/ZooKeeper/IKeeper.h | 44 +++++++++++-- src/Common/ZooKeeper/TestKeeper.cpp | 8 +-- src/Common/ZooKeeper/ZooKeeper.cpp | 62 +++++++++---------- src/Common/ZooKeeper/ZooKeeperArgs.cpp | 6 +- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 19 +++--- src/Common/ZooKeeper/ZooKeeperCommon.h | 4 +- src/Common/ZooKeeper/ZooKeeperConstants.cpp | 2 +- src/Common/ZooKeeper/ZooKeeperIO.cpp | 4 +- src/Common/ZooKeeper/ZooKeeperIO.h | 6 +- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 28 ++++----- .../ZooKeeper/ZooKeeperWithFaultInjection.h | 12 ++-- src/Coordination/KeeperDispatcher.cpp | 2 +- src/Core/SettingsFields.h | 10 ++- src/Databases/DatabaseReplicatedWorker.cpp | 2 +- .../MySQL/MaterializedMySQLSyncThread.cpp | 4 +- ...erDefinedSQLObjectsLoaderFromZooKeeper.cpp | 8 +-- src/Interpreters/DDLWorker.cpp | 2 +- src/Interpreters/TransactionLog.cpp | 4 +- src/Interpreters/executeDDLQueryOnCluster.cpp | 2 +- src/Storages/AlterCommands.cpp | 5 +- src/Storages/ColumnsDescription.cpp | 9 +-- src/Storages/ColumnsDescription.h | 5 +- src/Storages/MergeTree/LeaderElection.h | 2 +- .../PartMovesBetweenShardsOrchestrator.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- src/Storages/MergeTree/ZooKeeperRetries.h | 2 +- src/Storages/ProjectionsDescription.cpp | 10 ++- src/Storages/S3Queue/S3QueueFilesMetadata.cpp | 2 +- src/Storages/StorageKeeperMap.cpp | 4 +- src/Storages/StorageReplicatedMergeTree.cpp | 22 +++---- .../System/StorageSystemDDLWorkerQueue.cpp | 4 +- ...nd_exception_messages_formatting.reference | 1 + ..._log_and_exception_messages_formatting.sql | 9 ++- .../0_stateless/00921_datetime64_basic.sql | 6 +- .../01281_parseDateTime64BestEffort.sql | 2 +- .../0_stateless/02244_make_datetime.sql | 2 +- .../0_stateless/02245_make_datetime64.sql | 2 +- ...2505_forbid_paths_in_datetime_timezone.sql | 10 +-- 49 files changed, 243 insertions(+), 194 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 1723c274fdb..77fbc9430ef 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -114,15 +114,16 @@ void Keeper::createServer(const std::string & listen_host, const char * port_nam if (listen_try) { - LOG_WARNING(&logger(), "{}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to " + LOG_WARNING(&logger(), "Listen [{}]:{} failed: {}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, " + "then consider to " "specify not disabled IPv4 or IPv6 address to listen in element of configuration " "file. Example for disabled IPv6: 0.0.0.0 ." " Example for disabled IPv4: ::", - message); + listen_host, port, getCurrentExceptionMessage(false)); } else { - throw Exception::createDeprecated(message, ErrorCodes::NETWORK_ERROR); + throw Exception(ErrorCodes::NETWORK_ERROR, "Listen [{}]:{} failed: {}", listen_host, port, getCurrentExceptionMessage(false)); } } } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 6237186e3a6..b9ecbe7852f 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -325,19 +325,18 @@ void Server::createServer( } catch (const Poco::Exception &) { - std::string message = "Listen [" + listen_host + "]:" + std::to_string(port) + " failed: " + getCurrentExceptionMessage(false); - if (listen_try) { - LOG_WARNING(&logger(), "{}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, then consider to " + LOG_WARNING(&logger(), "Listen [{}]:{} failed: {}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, " + "then consider to " "specify not disabled IPv4 or IPv6 address to listen in element of configuration " "file. Example for disabled IPv6: 0.0.0.0 ." " Example for disabled IPv4: ::", - message); + listen_host, port, getCurrentExceptionMessage(false)); } else { - throw Exception::createDeprecated(message, ErrorCodes::NETWORK_ERROR); + throw Exception(ErrorCodes::NETWORK_ERROR, "Listen [{}]:{} failed: {}", listen_host, port, getCurrentExceptionMessage(false)); } } } diff --git a/src/Backups/BackupCoordinationRemote.cpp b/src/Backups/BackupCoordinationRemote.cpp index 214873fb765..e5fcbf26781 100644 --- a/src/Backups/BackupCoordinationRemote.cpp +++ b/src/Backups/BackupCoordinationRemote.cpp @@ -187,7 +187,7 @@ BackupCoordinationRemote::BackupCoordinationRemote( if (code == Coordination::Error::ZNODEEXISTS) zk->handleEphemeralNodeExistenceNoFailureInjection(alive_node_path, ""); else if (code != Coordination::Error::ZOK) - throw zkutil::KeeperException(code, alive_node_path); + throw zkutil::KeeperException::fromPath(code, alive_node_path); } }) { @@ -745,7 +745,7 @@ bool BackupCoordinationRemote::startWritingFile(size_t data_file_index) else if (code == Coordination::Error::ZNODEEXISTS) host_is_assigned = (zk->get(full_path) == host_index_str); /// The previous retry could write this ZooKeeper's node and then fail. else - throw zkutil::KeeperException(code, full_path); + throw zkutil::KeeperException::fromPath(code, full_path); }); if (!host_is_assigned) @@ -815,7 +815,7 @@ bool BackupCoordinationRemote::hasConcurrentBackups(const std::atomic &) break; bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1); if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt) - throw zkutil::KeeperException(code, backup_stage_path); + throw zkutil::KeeperException::fromPath(code, backup_stage_path); } }); diff --git a/src/Backups/BackupCoordinationStageSync.cpp b/src/Backups/BackupCoordinationStageSync.cpp index ebddbb8b82f..9b9ddc8515c 100644 --- a/src/Backups/BackupCoordinationStageSync.cpp +++ b/src/Backups/BackupCoordinationStageSync.cpp @@ -56,7 +56,7 @@ void BackupCoordinationStageSync::set(const String & current_host, const String { auto code = zookeeper->trySet(zookeeper_path, new_stage); if (code != Coordination::Error::ZOK) - throw zkutil::KeeperException(code, zookeeper_path); + throw zkutil::KeeperException::fromPath(code, zookeeper_path); } else { @@ -64,7 +64,7 @@ void BackupCoordinationStageSync::set(const String & current_host, const String String alive_node_path = zookeeper_path + "/alive|" + current_host; auto code = zookeeper->tryCreate(alive_node_path, "", zkutil::CreateMode::Ephemeral); if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNODEEXISTS) - throw zkutil::KeeperException(code, alive_node_path); + throw zkutil::KeeperException::fromPath(code, alive_node_path); zookeeper->createIfNotExists(zookeeper_path + "/started|" + current_host, ""); zookeeper->createIfNotExists(zookeeper_path + "/current|" + current_host + "|" + new_stage, message); @@ -90,7 +90,7 @@ void BackupCoordinationStageSync::setError(const String & current_host, const Ex /// so the following line tries to preserve the error status. auto code = zookeeper->trySet(zookeeper_path, Stage::ERROR); if (code != Coordination::Error::ZOK) - throw zkutil::KeeperException(code, zookeeper_path); + throw zkutil::KeeperException::fromPath(code, zookeeper_path); }); } diff --git a/src/Backups/RestoreCoordinationRemote.cpp b/src/Backups/RestoreCoordinationRemote.cpp index f95969b52a1..37abebb26b7 100644 --- a/src/Backups/RestoreCoordinationRemote.cpp +++ b/src/Backups/RestoreCoordinationRemote.cpp @@ -46,7 +46,7 @@ RestoreCoordinationRemote::RestoreCoordinationRemote( if (code == Coordination::Error::ZNODEEXISTS) zk->handleEphemeralNodeExistenceNoFailureInjection(alive_node_path, ""); else if (code != Coordination::Error::ZOK) - throw zkutil::KeeperException(code, alive_node_path); + throw zkutil::KeeperException::fromPath(code, alive_node_path); } }) { @@ -129,7 +129,7 @@ bool RestoreCoordinationRemote::acquireCreatingTableInReplicatedDatabase(const S path += "/" + escapeForFileName(table_name); auto code = zk->tryCreate(path, toString(current_host_index), zkutil::CreateMode::Persistent); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); if (code == Coordination::Error::ZOK) { @@ -155,7 +155,7 @@ bool RestoreCoordinationRemote::acquireInsertingDataIntoReplicatedTable(const St String path = zookeeper_path + "/repl_tables_data_acquired/" + escapeForFileName(table_zk_path); auto code = zk->tryCreate(path, toString(current_host_index), zkutil::CreateMode::Persistent); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); if (code == Coordination::Error::ZOK) { @@ -181,7 +181,7 @@ bool RestoreCoordinationRemote::acquireReplicatedAccessStorage(const String & ac String path = zookeeper_path + "/repl_access_storages_acquired/" + escapeForFileName(access_storage_zk_path); auto code = zk->tryCreate(path, toString(current_host_index), zkutil::CreateMode::Persistent); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); if (code == Coordination::Error::ZOK) { @@ -217,7 +217,7 @@ bool RestoreCoordinationRemote::acquireReplicatedSQLObjects(const String & loade auto code = zk->tryCreate(path, "", zkutil::CreateMode::Persistent); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); if (code == Coordination::Error::ZOK) { @@ -302,7 +302,7 @@ bool RestoreCoordinationRemote::hasConcurrentRestores(const std::atomic break; bool is_last_attempt = (attempt == MAX_ZOOKEEPER_ATTEMPTS - 1); if ((code != Coordination::Error::ZBADVERSION) || is_last_attempt) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); } }); diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index a8ff347f399..285362e32f1 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -313,8 +313,8 @@ bool DNSResolver::updateCacheImpl( UpdateF && update_func, ElemsT && elems, UInt32 max_consecutive_failures, - const String & notfound_log_msg, - const String & dropped_log_msg) + FormatStringHelper notfound_log_msg, + FormatStringHelper dropped_log_msg) { bool updated = false; String lost_elems; @@ -351,7 +351,7 @@ bool DNSResolver::updateCacheImpl( } if (!lost_elems.empty()) - LOG_INFO(log, fmt::runtime(notfound_log_msg), lost_elems); + LOG_INFO(log, notfound_log_msg.format(std::move(lost_elems))); if (elements_to_drop.size()) { updated = true; @@ -363,7 +363,7 @@ bool DNSResolver::updateCacheImpl( deleted_elements += cacheElemToString(it->first); elems.erase(it); } - LOG_INFO(log, fmt::runtime(dropped_log_msg), deleted_elements); + LOG_INFO(log, dropped_log_msg.format(std::move(deleted_elements))); } return updated; diff --git a/src/Common/DNSResolver.h b/src/Common/DNSResolver.h index 84715b392a8..1017607a5bd 100644 --- a/src/Common/DNSResolver.h +++ b/src/Common/DNSResolver.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace Poco { class Logger; } @@ -61,13 +62,12 @@ public: private: template - bool updateCacheImpl( UpdateF && update_func, ElemsT && elems, UInt32 max_consecutive_failures, - const String & notfound_log_msg, - const String & dropped_log_msg); + FormatStringHelper notfound_log_msg, + FormatStringHelper dropped_log_msg); DNSResolver(); diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index 4c21d9c9783..bb677b3a62d 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include @@ -12,6 +12,14 @@ #include +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} +} + /// Embedded timezones. std::string_view getTimeZone(const char * name); @@ -66,7 +74,7 @@ DateLUTImpl::DateLUTImpl(const std::string & time_zone_) cctz::time_zone cctz_time_zone; if (!cctz::load_time_zone(time_zone, &cctz_time_zone)) - throw Poco::Exception("Cannot load time zone " + time_zone_); + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot load time zone {}", time_zone_); constexpr cctz::civil_day epoch{1970, 1, 1}; constexpr cctz::civil_day lut_start{DATE_LUT_MIN_YEAR, 1, 1}; diff --git a/src/Common/NamePrompter.h b/src/Common/NamePrompter.h index 79e78529d95..65a2c76a102 100644 --- a/src/Common/NamePrompter.h +++ b/src/Common/NamePrompter.h @@ -117,6 +117,11 @@ public: DB::appendHintsMessage(error_message, hints); } + String getHintsMessage(const String & name) const + { + return getHintsErrorMessageSuffix(getHints(name)); + } + IHints() = default; IHints(const IHints &) = default; diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index f0a07241735..5897d04b8a2 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -21,29 +21,33 @@ namespace ProfileEvents namespace Coordination { -Exception::Exception(const std::string & msg, const Error code_, int) - : DB::Exception(msg, DB::ErrorCodes::KEEPER_EXCEPTION), code(code_) +void Exception::incrementErrorMetrics(const Error code_) { - if (Coordination::isUserError(code)) + if (Coordination::isUserError(code_)) ProfileEvents::increment(ProfileEvents::ZooKeeperUserExceptions); - else if (Coordination::isHardwareError(code)) + else if (Coordination::isHardwareError(code_)) ProfileEvents::increment(ProfileEvents::ZooKeeperHardwareExceptions); else ProfileEvents::increment(ProfileEvents::ZooKeeperOtherExceptions); } -Exception::Exception(const std::string & msg, const Error code_) - : Exception(msg + " (" + errorMessage(code_) + ")", code_, 0) +Exception::Exception(const std::string & msg, const Error code_, int) + : DB::Exception(msg, DB::ErrorCodes::KEEPER_EXCEPTION) + , code(code_) { + incrementErrorMetrics(code); +} + +Exception::Exception(PreformattedMessage && msg, const Error code_) + : DB::Exception(std::move(msg), DB::ErrorCodes::KEEPER_EXCEPTION) + , code(code_) +{ + extendedMessage(errorMessage(code)); + incrementErrorMetrics(code); } Exception::Exception(const Error code_) - : Exception(errorMessage(code_), code_, 0) -{ -} - -Exception::Exception(const Error code_, const std::string & path) - : Exception(std::string{errorMessage(code_)} + ", path: " + path, code_, 0) + : Exception(code_, "Coordination error: {}", errorMessage(code_)) { } @@ -56,10 +60,10 @@ using namespace DB; static void addRootPath(String & path, const String & root_path) { if (path.empty()) - throw Exception("Path cannot be empty", Error::ZBADARGUMENTS); + throw Exception::fromMessage(Error::ZBADARGUMENTS, "Path cannot be empty"); if (path[0] != '/') - throw Exception("Path must begin with /, got path '" + path + "'", Error::ZBADARGUMENTS); + throw Exception(Error::ZBADARGUMENTS, "Path must begin with /, got path '{}'", path); if (root_path.empty()) return; @@ -76,7 +80,7 @@ static void removeRootPath(String & path, const String & root_path) return; if (path.size() <= root_path.size()) - throw Exception("Received path is not longer than root_path", Error::ZDATAINCONSISTENCY); + throw Exception::fromMessage(Error::ZDATAINCONSISTENCY, "Received path is not longer than root_path"); path = path.substr(root_path.size()); } diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 43f0f9c5f6f..e2c9afdd5da 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -17,6 +17,13 @@ * - ZooKeeper emulation layer on top of Etcd, FoundationDB, whatever. */ +namespace DB +{ +namespace ErrorCodes +{ + extern const int KEEPER_EXCEPTION; +} +} namespace Coordination { @@ -450,17 +457,46 @@ class Exception : public DB::Exception private: /// Delegate constructor, used to minimize repetition; last parameter used for overload resolution. Exception(const std::string & msg, const Error code_, int); /// NOLINT + Exception(PreformattedMessage && msg, const Error code_); + + /// Message must be a compile-time constant + template + requires std::is_convertible_v + Exception(T && message, const Error code_) : DB::Exception(DB::ErrorCodes::KEEPER_EXCEPTION, std::forward(message)), code(code_) + { + incrementErrorMetrics(code); + } + + static void incrementErrorMetrics(const Error code_); public: explicit Exception(const Error code_); /// NOLINT - Exception(const std::string & msg, const Error code_); /// NOLINT - Exception(const Error code_, const std::string & path); /// NOLINT Exception(const Exception & exc); template - Exception(const Error code_, fmt::format_string fmt, Args &&... args) - : Exception(fmt::format(fmt, std::forward(args)...), code_) + Exception(const Error code_, FormatStringHelper fmt, Args &&... args) + : DB::Exception(DB::ErrorCodes::KEEPER_EXCEPTION, std::move(fmt), std::forward(args)...) + , code(code_) { + incrementErrorMetrics(code); + } + + inline static Exception createDeprecated(const std::string & msg, const Error code_) + { + return Exception(msg, code_, 0); + } + + inline static Exception fromPath(const Error code_, const std::string & path) + { + return Exception(code_, "Coordination error: {}, path {}", errorMessage(code_), path); + } + + /// Message must be a compile-time constant + template + requires std::is_convertible_v + inline static Exception fromMessage(const Error code_, T && message) + { + return Exception(std::forward(message), code_); } const char * name() const noexcept override { return "Coordination::Exception"; } diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index dabc0ae4eef..9e355093c9d 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -177,7 +177,7 @@ struct TestKeeperMultiRequest final : MultiRequest, TestKeeperRequest requests.push_back(std::make_shared(*concrete_request_check)); } else - throw Exception("Illegal command as part of multi ZooKeeper request", Error::ZBADARGUMENTS); + throw Exception::fromMessage(Error::ZBADARGUMENTS, "Illegal command as part of multi ZooKeeper request"); } } @@ -389,7 +389,7 @@ std::pair TestKeeperListRequest::process(TestKeeper::Containe { auto path_prefix = path; if (path_prefix.empty()) - throw Exception("Logical error: path cannot be empty", Error::ZSESSIONEXPIRED); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "Logical error: path cannot be empty"); if (path_prefix.back() != '/') path_prefix += '/'; @@ -705,10 +705,10 @@ void TestKeeper::pushRequest(RequestInfo && request) std::lock_guard lock(push_request_mutex); if (expired) - throw Exception("Session expired", Error::ZSESSIONEXPIRED); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "Session expired"); if (!requests_queue.tryPush(std::move(request), args.operation_timeout_ms)) - throw Exception("Cannot push request to queue within operation timeout", Error::ZOPERATIONTIMEOUT); + throw Exception::fromMessage(Error::ZOPERATIONTIMEOUT, "Cannot push request to queue within operation timeout"); } catch (...) { diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 10331a4e410..b7f85293e94 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -51,7 +51,7 @@ const int CreateMode::EphemeralSequential = 3; static void check(Coordination::Error code, const std::string & path) { if (code != Coordination::Error::ZOK) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); } @@ -64,7 +64,7 @@ void ZooKeeper::init(ZooKeeperArgs args_) if (args.implementation == "zookeeper") { if (args.hosts.empty()) - throw KeeperException("No hosts passed to ZooKeeper constructor.", Coordination::Error::ZBADARGUMENTS); + throw KeeperException::fromMessage(Coordination::Error::ZBADARGUMENTS, "No hosts passed to ZooKeeper constructor."); Coordination::ZooKeeper::Nodes nodes; nodes.reserve(args.hosts.size()); @@ -107,9 +107,9 @@ void ZooKeeper::init(ZooKeeperArgs args_) { /// For DNS errors we throw exception with ZCONNECTIONLOSS code, so it will be considered as hardware error, not user error if (dns_error) - throw KeeperException("Cannot resolve any of provided ZooKeeper hosts due to DNS error", Coordination::Error::ZCONNECTIONLOSS); + throw KeeperException::fromMessage(Coordination::Error::ZCONNECTIONLOSS, "Cannot resolve any of provided ZooKeeper hosts due to DNS error"); else - throw KeeperException("Cannot use any of provided ZooKeeper nodes", Coordination::Error::ZCONNECTIONLOSS); + throw KeeperException::fromMessage(Coordination::Error::ZCONNECTIONLOSS, "Cannot use any of provided ZooKeeper nodes"); } impl = std::make_unique(nodes, args, zk_log, [this](size_t node_idx, const Coordination::ZooKeeper::Node & node) @@ -145,11 +145,11 @@ void ZooKeeper::init(ZooKeeperArgs args_) auto future = asyncExists("/"); auto res = future.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)); if (res != std::future_status::ready) - throw KeeperException("Cannot check if zookeeper root exists.", Coordination::Error::ZOPERATIONTIMEOUT); + throw KeeperException::fromMessage(Coordination::Error::ZOPERATIONTIMEOUT, "Cannot check if zookeeper root exists."); auto code = future.get().error; if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) - throw KeeperException(code, "/"); + throw KeeperException::fromPath(code, "/"); if (code == Coordination::Error::ZNONODE) throw KeeperException(Coordination::Error::ZNONODE, "ZooKeeper root doesn't exist. You should create root node {} before start.", args.chroot); @@ -260,7 +260,7 @@ Coordination::Error ZooKeeper::tryGetChildren( Coordination::Error code = getChildrenImpl(path, res, stat, callbackForEvent(watch), list_request_type); if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); return code; } @@ -275,7 +275,7 @@ Coordination::Error ZooKeeper::tryGetChildrenWatch( Coordination::Error code = getChildrenImpl(path, res, stat, watch_callback, list_request_type); if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); return code; } @@ -314,7 +314,7 @@ Coordination::Error ZooKeeper::tryCreate(const std::string & path, const std::st code == Coordination::Error::ZNONODE || code == Coordination::Error::ZNODEEXISTS || code == Coordination::Error::ZNOCHILDRENFOREPHEMERALS)) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); return code; } @@ -333,7 +333,7 @@ void ZooKeeper::createIfNotExists(const std::string & path, const std::string & if (code == Coordination::Error::ZOK || code == Coordination::Error::ZNODEEXISTS) return; else - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); } void ZooKeeper::createAncestors(const std::string & path) @@ -355,14 +355,14 @@ void ZooKeeper::createAncestors(const std::string & path) /// The parent node doesn't exist. Save the current node and try with the parent last_pos = current_node.rfind('/'); if (last_pos == std::string::npos || last_pos == 0) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); pending_nodes.emplace_back(std::move(current_node)); current_node = path.substr(0, last_pos); } else if (code == Coordination::Error::ZOK || code == Coordination::Error::ZNODEEXISTS) break; else - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); } for (const std::string & pending : pending_nodes | std::views::reverse) @@ -423,7 +423,7 @@ Coordination::Error ZooKeeper::tryRemove(const std::string & path, int32_t versi code == Coordination::Error::ZNONODE || code == Coordination::Error::ZBADVERSION || code == Coordination::Error::ZNOTEMPTY)) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); return code; } @@ -457,7 +457,7 @@ bool ZooKeeper::existsWatch(const std::string & path, Coordination::Stat * stat, Coordination::Error code = existsImpl(path, stat, watch_callback); if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); return code != Coordination::Error::ZNONODE; } @@ -524,7 +524,7 @@ bool ZooKeeper::tryGetWatch( Coordination::Error code = getImpl(path, res, stat, watch_callback); if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); if (return_code) *return_code = code; @@ -566,7 +566,7 @@ void ZooKeeper::createOrUpdate(const std::string & path, const std::string & dat create(path, data, mode); } else if (code != Coordination::Error::ZOK) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); } Coordination::Error ZooKeeper::trySet(const std::string & path, const std::string & data, @@ -577,7 +577,7 @@ Coordination::Error ZooKeeper::trySet(const std::string & path, const std::strin if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE || code == Coordination::Error::ZBADVERSION)) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); return code; } @@ -756,7 +756,7 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab continue; } - throw KeeperException(res.error, batch[i]); + throw KeeperException::fromPath(res.error, batch[i]); } } return removed_as_expected; @@ -823,7 +823,7 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & return true; if (state->code) - throw KeeperException(static_cast(state->code.load(std::memory_order_seq_cst)), path); + throw KeeperException::fromPath(static_cast(state->code.load(std::memory_order_seq_cst)), path); if (state->event_type == Coordination::DELETED) return true; @@ -844,7 +844,7 @@ void ZooKeeper::handleEphemeralNodeExistence(const std::string & path, const std { auto code = tryRemove(path, stat.version); if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE) - throw Coordination::Exception(code, path); + throw Coordination::Exception::fromPath(code, path); } else { @@ -893,7 +893,7 @@ std::future ZooKeeper::asyncCreate(const std::stri auto callback = [promise, path](const Coordination::CreateResponse & response) mutable { if (response.error != Coordination::Error::ZOK) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -924,7 +924,7 @@ std::future ZooKeeper::asyncGet(const std::string & p auto callback = [promise, path](const Coordination::GetResponse & response) mutable { if (response.error != Coordination::Error::ZOK) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -956,7 +956,7 @@ std::future ZooKeeper::asyncTryGet(const std::string auto callback = [promise, path](const Coordination::GetResponse & response) mutable { if (response.error != Coordination::Error::ZOK && response.error != Coordination::Error::ZNONODE) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -973,7 +973,7 @@ std::future ZooKeeper::asyncExists(const std::stri auto callback = [promise, path](const Coordination::ExistsResponse & response) mutable { if (response.error != Coordination::Error::ZOK && response.error != Coordination::Error::ZNONODE) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -1004,7 +1004,7 @@ std::future ZooKeeper::asyncSet(const std::string & p auto callback = [promise, path](const Coordination::SetResponse & response) mutable { if (response.error != Coordination::Error::ZOK) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -1037,7 +1037,7 @@ std::future ZooKeeper::asyncGetChildren( auto callback = [promise, path](const Coordination::ListResponse & response) mutable { if (response.error != Coordination::Error::ZOK) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -1070,7 +1070,7 @@ ZooKeeper::asyncTryGetChildren(const std::string & path, Coordination::ListReque auto callback = [promise, path](const Coordination::ListResponse & response) mutable { if (response.error != Coordination::Error::ZOK && response.error != Coordination::Error::ZNONODE) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -1087,7 +1087,7 @@ std::future ZooKeeper::asyncRemove(const std::stri auto callback = [promise, path](const Coordination::RemoveResponse & response) mutable { if (response.error != Coordination::Error::ZOK) - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); else promise->set_value(response); }; @@ -1108,7 +1108,7 @@ std::future ZooKeeper::asyncTryRemove(const std::s && response.error != Coordination::Error::ZBADVERSION && response.error != Coordination::Error::ZNOTEMPTY) { - promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException::fromPath(response.error, path))); } else promise->set_value(response); @@ -1244,10 +1244,10 @@ size_t getFailedOpIndex(Coordination::Error exception_code, const Coordination:: KeeperMultiException::KeeperMultiException(Coordination::Error exception_code, const Coordination::Requests & requests_, const Coordination::Responses & responses_) - : KeeperException("Transaction failed", exception_code), + : KeeperException(exception_code, "Transaction failed: Op #{}, path", failed_op_index), requests(requests_), responses(responses_), failed_op_index(getFailedOpIndex(exception_code, responses)) { - addMessage("Op #" + std::to_string(failed_op_index) + ", path: " + getPathForFirstFailedOp()); + addMessage(getPathForFirstFailedOp()); } diff --git a/src/Common/ZooKeeper/ZooKeeperArgs.cpp b/src/Common/ZooKeeper/ZooKeeperArgs.cpp index 4c73b9ffc6d..5d01294e9b0 100644 --- a/src/Common/ZooKeeper/ZooKeeperArgs.cpp +++ b/src/Common/ZooKeeper/ZooKeeperArgs.cpp @@ -36,7 +36,7 @@ ZooKeeperArgs::ZooKeeperArgs(const Poco::Util::AbstractConfiguration & config, c } if (session_timeout_ms < 0 || operation_timeout_ms < 0 || connection_timeout_ms < 0) - throw KeeperException("Timeout cannot be negative", Coordination::Error::ZBADARGUMENTS); + throw KeeperException::fromMessage(Coordination::Error::ZBADARGUMENTS, "Timeout cannot be negative"); /// init get_priority_load_balancing get_priority_load_balancing.hostname_differences.resize(hosts.size()); @@ -63,7 +63,7 @@ void ZooKeeperArgs::initFromKeeperServerSection(const Poco::Util::AbstractConfig auto tcp_port_secure = config.getString(key); if (tcp_port_secure.empty()) - throw KeeperException("Empty tcp_port_secure in config file", Coordination::Error::ZBADARGUMENTS); + throw KeeperException::fromMessage(Coordination::Error::ZBADARGUMENTS, "Empty tcp_port_secure in config file"); } bool secure{false}; @@ -81,7 +81,7 @@ void ZooKeeperArgs::initFromKeeperServerSection(const Poco::Util::AbstractConfig } if (tcp_port.empty()) - throw KeeperException("No tcp_port or tcp_port_secure in config file", Coordination::Error::ZBADARGUMENTS); + throw KeeperException::fromMessage(Coordination::Error::ZBADARGUMENTS, "No tcp_port or tcp_port_secure in config file"); if (auto coordination_key = std::string{config_name} + ".coordination_settings"; config.has(coordination_key)) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index e88d66e5444..8341199cd1e 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -461,8 +461,7 @@ void ZooKeeperErrorResponse::readImpl(ReadBuffer & in) Coordination::read(read_error, in); if (read_error != error) - throw Exception(fmt::format("Error code in ErrorResponse ({}) doesn't match error code in header ({})", read_error, error), - Error::ZMARSHALLINGERROR); + throw Exception(Error::ZMARSHALLINGERROR, "Error code in ErrorResponse ({}) doesn't match error code in header ({})", read_error, error); } void ZooKeeperErrorResponse::writeImpl(WriteBuffer & out) const @@ -534,7 +533,7 @@ ZooKeeperMultiRequest::ZooKeeperMultiRequest(const Requests & generic_requests, requests.push_back(std::make_shared(*concrete_request_list)); } else - throw Exception("Illegal command as part of multi ZooKeeper request", Error::ZBADARGUMENTS); + throw Exception::fromMessage(Error::ZBADARGUMENTS, "Illegal command as part of multi ZooKeeper request"); } } @@ -577,9 +576,9 @@ void ZooKeeperMultiRequest::readImpl(ReadBuffer & in) if (done) { if (op_num != OpNum::Error) - throw Exception("Unexpected op_num received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Unexpected op_num received at the end of results for multi transaction"); if (error != -1) - throw Exception("Unexpected error value received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Unexpected error value received at the end of results for multi transaction"); break; } @@ -588,7 +587,7 @@ void ZooKeeperMultiRequest::readImpl(ReadBuffer & in) requests.push_back(request); if (in.eof()) - throw Exception("Not enough results received for multi transaction", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Not enough results received for multi transaction"); } } @@ -621,7 +620,7 @@ void ZooKeeperMultiResponse::readImpl(ReadBuffer & in) Coordination::read(op_error, in); if (done) - throw Exception("Not enough results received for multi transaction", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Not enough results received for multi transaction"); /// op_num == -1 is special for multi transaction. /// For unknown reason, error code is duplicated in header and in response body. @@ -657,11 +656,11 @@ void ZooKeeperMultiResponse::readImpl(ReadBuffer & in) Coordination::read(error_read, in); if (!done) - throw Exception("Too many results received for multi transaction", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Too many results received for multi transaction"); if (op_num != OpNum::Error) - throw Exception("Unexpected op_num received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Unexpected op_num received at the end of results for multi transaction"); if (error_read != -1) - throw Exception("Unexpected error value received at the end of results for multi transaction", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Unexpected error value received at the end of results for multi transaction"); } } diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index e4b2cc97744..5b662c7f4c1 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -163,7 +163,7 @@ struct ZooKeeperWatchResponse final : WatchResponse, ZooKeeperResponse OpNum getOpNum() const override { chassert(false); - throw Exception("OpNum for watch response doesn't exist", Error::ZRUNTIMEINCONSISTENCY); + throw Exception::fromMessage(Error::ZRUNTIMEINCONSISTENCY, "OpNum for watch response doesn't exist"); } void fillLogElements(LogElements & elems, size_t idx) const override; @@ -214,7 +214,7 @@ struct ZooKeeperCloseResponse final : ZooKeeperResponse { void readImpl(ReadBuffer &) override { - throw Exception("Received response for close request", Error::ZRUNTIMEINCONSISTENCY); + throw Exception::fromMessage(Error::ZRUNTIMEINCONSISTENCY, "Received response for close request"); } void writeImpl(WriteBuffer &) const override {} diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.cpp b/src/Common/ZooKeeper/ZooKeeperConstants.cpp index 9bb9c7b0488..7fffea1d08d 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.cpp +++ b/src/Common/ZooKeeper/ZooKeeperConstants.cpp @@ -33,7 +33,7 @@ static const std::unordered_set VALID_OPERATIONS = OpNum getOpNum(int32_t raw_op_num) { if (!VALID_OPERATIONS.contains(raw_op_num)) - throw Exception("Operation " + std::to_string(raw_op_num) + " is unknown", Error::ZUNIMPLEMENTED); + throw Exception(Error::ZUNIMPLEMENTED, "Operation {} is unknown", raw_op_num); return static_cast(raw_op_num); } diff --git a/src/Common/ZooKeeper/ZooKeeperIO.cpp b/src/Common/ZooKeeper/ZooKeeperIO.cpp index 2911d511254..6a51ffb36fa 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.cpp +++ b/src/Common/ZooKeeper/ZooKeeperIO.cpp @@ -62,10 +62,10 @@ void read(std::string & s, ReadBuffer & in) } if (size < 0) - throw Exception("Negative size while reading string from ZooKeeper", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Negative size while reading string from ZooKeeper"); if (size > MAX_STRING_OR_ARRAY_SIZE) - throw Exception("Too large string size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR,"Too large string size while reading from ZooKeeper"); s.resize(size); size_t read_bytes = in.read(s.data(), size); diff --git a/src/Common/ZooKeeper/ZooKeeperIO.h b/src/Common/ZooKeeper/ZooKeeperIO.h index 81b56a02e27..83973c1ae22 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.h +++ b/src/Common/ZooKeeper/ZooKeeperIO.h @@ -62,7 +62,7 @@ void read(std::array & s, ReadBuffer & in) int32_t size = 0; read(size, in); if (size != N) - throw Exception("Unexpected array size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Unexpected array size while reading from ZooKeeper"); in.readStrict(s.data(), N); } @@ -72,9 +72,9 @@ void read(std::vector & arr, ReadBuffer & in) int32_t size = 0; read(size, in); if (size < 0) - throw Exception("Negative size while reading array from ZooKeeper", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Negative size while reading array from ZooKeeper"); if (size > MAX_STRING_OR_ARRAY_SIZE) - throw Exception("Too large array size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); + throw Exception::fromMessage(Error::ZMARSHALLINGERROR, "Too large array size while reading from ZooKeeper"); arr.resize(size); for (auto & elem : arr) read(elem, in); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index d84d5fa3a69..4dbdcf51b24 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -289,7 +289,7 @@ static void removeRootPath(String & path, const String & chroot) return; if (path.size() <= chroot.size()) - throw Exception(Error::ZDATAINCONSISTENCY, "Received path is not longer than chroot"); + throw Exception::fromMessage(Error::ZDATAINCONSISTENCY, "Received path is not longer than chroot"); path = path.substr(chroot.size()); } @@ -387,7 +387,7 @@ void ZooKeeper::connect( Poco::Timespan connection_timeout) { if (nodes.empty()) - throw Exception(Error::ZBADARGUMENTS, "No nodes passed to ZooKeeper constructor"); + throw Exception::fromMessage(Error::ZBADARGUMENTS, "No nodes passed to ZooKeeper constructor"); static constexpr size_t num_tries = 3; bool connected = false; @@ -479,8 +479,6 @@ void ZooKeeper::connect( if (!connected) { WriteBufferFromOwnString message; - - message << "All connection tries failed while connecting to ZooKeeper. nodes: "; bool first = true; for (const auto & node : nodes) { @@ -496,7 +494,7 @@ void ZooKeeper::connect( } message << fail_reasons.str() << "\n"; - throw Exception(Error::ZCONNECTIONLOSS, message.str()); + throw Exception(Error::ZCONNECTIONLOSS, "All connection tries failed while connecting to ZooKeeper. nodes: {}", message.str()); } else { @@ -543,7 +541,7 @@ void ZooKeeper::receiveHandshake() /// It's better for faster failover than just connection drop. /// Implemented in clickhouse-keeper. if (protocol_version_read == KEEPER_PROTOCOL_VERSION_CONNECTION_REJECT) - throw Exception(Error::ZCONNECTIONLOSS, + throw Exception::fromMessage(Error::ZCONNECTIONLOSS, "Keeper server rejected the connection during the handshake. " "Possibly it's overloaded, doesn't see leader or stale"); else @@ -800,7 +798,7 @@ void ZooKeeper::receiveEvent() auto it = operations.find(xid); if (it == operations.end()) - throw Exception("Received response for unknown xid " + DB::toString(xid), Error::ZRUNTIMEINCONSISTENCY); + throw Exception(Error::ZRUNTIMEINCONSISTENCY, "Received response for unknown xid {}", xid); /// After this point, we must invoke callback, that we've grabbed from 'operations'. /// Invariant: all callbacks are invoked either in case of success or in case of error. @@ -1088,9 +1086,9 @@ void ZooKeeper::pushRequest(RequestInfo && info) { info.request->xid = next_xid.fetch_add(1); if (info.request->xid == CLOSE_XID) - throw Exception(Error::ZSESSIONEXPIRED, "xid equal to close_xid"); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "xid equal to close_xid"); if (info.request->xid < 0) - throw Exception(Error::ZSESSIONEXPIRED, "XID overflow"); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "XID overflow"); if (auto * multi_request = dynamic_cast(info.request.get())) { @@ -1104,7 +1102,7 @@ void ZooKeeper::pushRequest(RequestInfo && info) if (!requests_queue.tryPush(std::move(info), args.operation_timeout_ms)) { if (requests_queue.isFinished()) - throw Exception(Error::ZSESSIONEXPIRED, "Session expired"); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "Session expired"); throw Exception(Error::ZOPERATIONTIMEOUT, "Cannot push request to queue within operation timeout of {} ms", args.operation_timeout_ms); } @@ -1297,7 +1295,7 @@ void ZooKeeper::list( if (!isFeatureEnabled(KeeperFeatureFlag::FILTERED_LIST)) { if (list_request_type != ListRequestType::ALL) - throw Exception(Error::ZBADARGUMENTS, "Filtered list request type cannot be used because it's not supported by the server"); + throw Exception::fromMessage(Error::ZBADARGUMENTS, "Filtered list request type cannot be used because it's not supported by the server"); request = std::make_shared(); } @@ -1380,7 +1378,7 @@ void ZooKeeper::multi( ZooKeeperMultiRequest request(requests, default_acls); if (request.getOpNum() == OpNum::MultiRead && !isFeatureEnabled(KeeperFeatureFlag::MULTI_READ)) - throw Exception(Error::ZBADARGUMENTS, "MultiRead request type cannot be used because it's not supported by the server"); + throw Exception::fromMessage(Error::ZBADARGUMENTS, "MultiRead request type cannot be used because it's not supported by the server"); RequestInfo request_info; request_info.request = std::make_shared(std::move(request)); @@ -1502,7 +1500,7 @@ void ZooKeeper::setupFaultDistributions() void ZooKeeper::checkSessionDeadline() const { if (unlikely(hasReachedDeadline())) - throw Exception(Error::ZSESSIONEXPIRED, "Session expired (force expiry client-side)"); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "Session expired (force expiry client-side)"); } bool ZooKeeper::hasReachedDeadline() const @@ -1513,13 +1511,13 @@ bool ZooKeeper::hasReachedDeadline() const void ZooKeeper::maybeInjectSendFault() { if (unlikely(inject_setup.test() && send_inject_fault && send_inject_fault.value()(thread_local_rng))) - throw Exception(Error::ZSESSIONEXPIRED, "Session expired (fault injected on recv)"); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "Session expired (fault injected on recv)"); } void ZooKeeper::maybeInjectRecvFault() { if (unlikely(inject_setup.test() && recv_inject_fault && recv_inject_fault.value()(thread_local_rng))) - throw Exception(Error::ZSESSIONEXPIRED, "Session expired (fault injected on recv)"); + throw Exception::fromMessage(Error::ZSESSIONEXPIRED, "Session expired (fault injected on recv)"); } void ZooKeeper::maybeInjectSendSleep() diff --git a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h index 9d02d674010..4887e896e9b 100644 --- a/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h +++ b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h @@ -29,7 +29,7 @@ public: if (distribution(rndgen) || must_fail_before_op) { must_fail_before_op = false; - throw zkutil::KeeperException("Fault injection before operation", Coordination::Error::ZSESSIONEXPIRED); + throw zkutil::KeeperException::fromMessage(Coordination::Error::ZSESSIONEXPIRED, "Fault injection before operation"); } } void afterOperation() @@ -37,7 +37,7 @@ public: if (distribution(rndgen) || must_fail_after_op) { must_fail_after_op = false; - throw zkutil::KeeperException("Fault injection after operation", Coordination::Error::ZOPERATIONTIMEOUT); + throw zkutil::KeeperException::fromMessage(Coordination::Error::ZOPERATIONTIMEOUT, "Fault injection after operation"); } } @@ -263,7 +263,7 @@ public: auto code = tryCreate(path, data, mode, path_created); if (code != Coordination::Error::ZOK) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); return path_created; } @@ -327,7 +327,7 @@ public: if (code == Coordination::Error::ZOK || code == Coordination::Error::ZNODEEXISTS) return; - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); } Coordination::Responses multi(const Coordination::Requests & requests) @@ -507,8 +507,8 @@ private: ++calls_total; if (!keeper) - throw zkutil::KeeperException( - "Session is considered to be expired due to fault injection", Coordination::Error::ZSESSIONEXPIRED); + throw zkutil::KeeperException::fromMessage(Coordination::Error::ZSESSIONEXPIRED, + "Session is considered to be expired due to fault injection"); if constexpr (inject_failure_before_op) { diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 99c28674273..1fd67aef72c 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -687,7 +687,7 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) } if (response->error != Coordination::Error::ZOK) - promise->set_exception(std::make_exception_ptr(zkutil::KeeperException("SessionID request failed with error", response->error))); + promise->set_exception(std::make_exception_ptr(zkutil::KeeperException::fromMessage(response->error, "SessionID request failed with error"))); promise->set_value(session_id_response.session_id); }; diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index db14f91512f..0e1f35d4352 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -428,9 +428,8 @@ constexpr auto getEnumValues() auto it = map.find(value); \ if (it != map.end()) \ return it->second; \ - throw Exception::createDeprecated( \ - "Unexpected value of " #NEW_NAME ":" + std::to_string(std::underlying_type::type(value)), \ - ERROR_CODE_FOR_UNEXPECTED_NAME); \ + throw Exception(ERROR_CODE_FOR_UNEXPECTED_NAME, \ + "Unexpected value of " #NEW_NAME ":{}", std::to_string(std::underlying_type::type(value))); \ } \ \ typename SettingField##NEW_NAME::EnumType SettingField##NEW_NAME##Traits::fromString(std::string_view str) \ @@ -444,7 +443,7 @@ constexpr auto getEnumValues() auto it = map.find(str); \ if (it != map.end()) \ return it->second; \ - String msg = "Unexpected value of " #NEW_NAME ": '" + String{str} + "'. Must be one of ["; \ + String msg; \ bool need_comma = false; \ for (auto & name : map | boost::adaptors::map_keys) \ { \ @@ -452,8 +451,7 @@ constexpr auto getEnumValues() msg += ", "; \ msg += "'" + String{name} + "'"; \ } \ - msg += "]"; \ - throw Exception::createDeprecated(msg, ERROR_CODE_FOR_UNEXPECTED_NAME); \ + throw Exception(ERROR_CODE_FOR_UNEXPECTED_NAME, "Unexpected value of " #NEW_NAME ": '{}'. Must be one of [{}]", String{str}, msg); \ } // Mostly like SettingFieldEnum, but can have multiple enum values (or none) set at once. diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 4976f54e417..367f772d402 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -356,7 +356,7 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na /// We use tryRemove(...) because multiple hosts (including initiator) may try to do it concurrently. auto code = zookeeper->tryRemove(try_node_path); if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE) - throw Coordination::Exception(code, try_node_path); + throw Coordination::Exception::fromPath(code, try_node_path); if (!zookeeper->exists(fs::path(entry_path) / "committed")) { diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index c9e10b27caa..3578362b8dd 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -141,7 +141,6 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const S { bool first = true; WriteBufferFromOwnString error_message; - error_message << "Illegal MySQL variables, the MaterializedMySQL engine requires "; for (const auto & [variable_name, variable_error_val] : variables_error_message) { error_message << (first ? "" : ", ") << variable_name << "='" << variable_error_val << "'"; @@ -150,7 +149,8 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const S first = false; } - throw Exception::createDeprecated(error_message.str(), ErrorCodes::ILLEGAL_MYSQL_VARIABLE); + throw Exception(ErrorCodes::ILLEGAL_MYSQL_VARIABLE, "Illegal MySQL variables, the MaterializedMySQL engine requires {}", + error_message.str()); } } diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp index 76ebcf769d5..29aff666da5 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsLoaderFromZooKeeper.cpp @@ -223,7 +223,7 @@ bool UserDefinedSQLObjectsLoaderFromZooKeeper::storeObject( { auto code = zookeeper->tryCreate(path, create_statement, zkutil::CreateMode::Persistent); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNODEEXISTS)) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); if (code == Coordination::Error::ZNODEEXISTS) { @@ -234,14 +234,14 @@ bool UserDefinedSQLObjectsLoaderFromZooKeeper::storeObject( code = zookeeper->trySet(path, create_statement); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNONODE)) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); } if (code == Coordination::Error::ZOK) break; if (!--num_attempts) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); } LOG_DEBUG(log, "Object {} stored", backQuote(object_name)); @@ -262,7 +262,7 @@ bool UserDefinedSQLObjectsLoaderFromZooKeeper::removeObject( auto code = zookeeper->tryRemove(path); if ((code != Coordination::Error::ZOK) && (code != Coordination::Error::ZNONODE)) - throw zkutil::KeeperException(code, path); + throw zkutil::KeeperException::fromPath(code, path); if (code == Coordination::Error::ZNONODE) { diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 92e6bcb326c..8be334d6223 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -574,7 +574,7 @@ void DDLWorker::processTask(DDLTaskBase & task, const ZooKeeperPtr & zookeeper) if (create_active_res != Coordination::Error::ZNONODE && create_active_res != Coordination::Error::ZNODEEXISTS) { chassert(Coordination::isHardwareError(create_active_res)); - throw Coordination::Exception(create_active_res, active_node_path); + throw Coordination::Exception::fromPath(create_active_res, active_node_path); } /// Status dirs were not created in enqueueQuery(...) or someone is removing entry diff --git a/src/Interpreters/TransactionLog.cpp b/src/Interpreters/TransactionLog.cpp index 631e7f5c746..66b23f09ba0 100644 --- a/src/Interpreters/TransactionLog.cpp +++ b/src/Interpreters/TransactionLog.cpp @@ -409,7 +409,7 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn, bool { std::bernoulli_distribution fault(fault_probability_before_commit); if (fault(thread_local_rng)) - throw Coordination::Exception("Fault injected (before commit)", Coordination::Error::ZCONNECTIONLOSS); + throw Coordination::Exception::fromMessage(Coordination::Error::ZCONNECTIONLOSS, "Fault injected (before commit)"); } /// Commit point @@ -419,7 +419,7 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn, bool { std::bernoulli_distribution fault(fault_probability_after_commit); if (fault(thread_local_rng)) - throw Coordination::Exception("Fault injected (after commit)", Coordination::Error::ZCONNECTIONLOSS); + throw Coordination::Exception::fromMessage(Coordination::Error::ZCONNECTIONLOSS, "Fault injected (after commit)"); } } catch (const Coordination::Exception & e) diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index b34108644fb..750affdfe71 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -559,7 +559,7 @@ Strings DDLQueryStatusSource::getChildrenAllowNoNode(const std::shared_ptrtryGetChildren(node_path, res); if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE) - throw Coordination::Exception(code, node_path); + throw Coordination::Exception::fromPath(code, node_path); return res; } diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index a9247f9b898..96dcd74f947 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1077,9 +1077,8 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const { if (!command.if_exists) { - String exception_message = fmt::format("Wrong column. Cannot find column {} to modify", backQuote(column_name)); - all_columns.appendHintsMessage(exception_message, column_name); - throw Exception::createDeprecated(exception_message, ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); + throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Wrong column. Cannot find column {} to modify{}", + backQuote(column_name), all_columns.getHintsMessage(column_name)); } else continue; diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 045afd7e6e6..0c918bda5fd 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -232,9 +232,7 @@ void ColumnsDescription::remove(const String & column_name) auto range = getNameRange(columns, column_name); if (range.first == range.second) { - String exception_message = fmt::format("There is no column {} in table", column_name); - appendHintsMessage(exception_message, column_name); - throw Exception::createDeprecated(exception_message, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no column {} in table{}", column_name, getHintsMessage(column_name)); } for (auto list_it = range.first; list_it != range.second;) @@ -249,9 +247,8 @@ void ColumnsDescription::rename(const String & column_from, const String & colum auto it = columns.get<1>().find(column_from); if (it == columns.get<1>().end()) { - String exception_message = fmt::format("Cannot find column {} in ColumnsDescription", column_from); - appendHintsMessage(exception_message, column_from); - throw Exception::createDeprecated(exception_message, ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find column {} in ColumnsDescription{}", + column_from, getHintsMessage(column_from)); } columns.get<1>().modify_key(it, [&column_to] (String & old_name) diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 365a999673e..fb1eeed3127 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -160,9 +160,8 @@ public: auto it = columns.get<1>().find(column_name); if (it == columns.get<1>().end()) { - String exception_message = fmt::format("Cannot find column {} in ColumnsDescription", column_name); - appendHintsMessage(exception_message, column_name); - throw Exception::createDeprecated(exception_message, ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find column {} in ColumnsDescription{}", + column_name, getHintsMessage(column_name)); } removeSubcolumns(it->name); diff --git a/src/Storages/MergeTree/LeaderElection.h b/src/Storages/MergeTree/LeaderElection.h index f694ecab8e3..2e48892563b 100644 --- a/src/Storages/MergeTree/LeaderElection.h +++ b/src/Storages/MergeTree/LeaderElection.h @@ -36,7 +36,7 @@ void checkNoOldLeaders(Poco::Logger * log, ZooKeeper & zookeeper, const String p if (code == Coordination::Error::ZNONODE) return; else if (code != Coordination::Error::ZOK) - throw KeeperException(code, path); + throw KeeperException::fromPath(code, path); Coordination::Requests ops; diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index 3b382b7b32d..76b8080f64c 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -684,7 +684,7 @@ CancellationCode PartMovesBetweenShardsOrchestrator::killPartMoveToShard(const U continue; } else - throw Coordination::Exception(code, entry.znode_path); + throw Coordination::Exception::fromPath(code, entry.znode_path); } } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 21d5597e614..84307a3ca7a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -2169,7 +2169,7 @@ CommittingBlocks BaseMergePredicate::getCommitti { auto & response = locks_children[i]; if (response.error != Coordination::Error::ZOK && !partition_ids_hint) - throw Coordination::Exception(response.error, paths[i]); + throw Coordination::Exception::fromPath(response.error, paths[i]); if (response.error != Coordination::Error::ZOK) { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 0db3464a637..2ad629c1493 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -351,7 +351,7 @@ size_t ReplicatedMergeTreeSinkImpl::checkQuorumPrecondition(const if (active_replicas < quorum_size) { if (Coordination::isHardwareError(keeper_error)) - throw Coordination::Exception("Failed to check number of alive replicas", keeper_error); + throw Coordination::Exception::fromMessage(keeper_error, "Failed to check number of alive replicas"); throw Exception(ErrorCodes::TOO_FEW_LIVE_REPLICAS, "Number of alive replicas ({}) is less than requested quorum ({}/{}).", active_replicas, quorum_size, replicas_number); diff --git a/src/Storages/MergeTree/ZooKeeperRetries.h b/src/Storages/MergeTree/ZooKeeperRetries.h index 512c0800de7..e46c3f974c7 100644 --- a/src/Storages/MergeTree/ZooKeeperRetries.h +++ b/src/Storages/MergeTree/ZooKeeperRetries.h @@ -159,7 +159,7 @@ public: void setKeeperError(Coordination::Error code, std::string message) { - setKeeperError(std::make_exception_ptr(zkutil::KeeperException(message, code)), code, message); + setKeeperError(std::make_exception_ptr(zkutil::KeeperException::createDeprecated(message, code)), code, message); } template diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 48825361a16..cddf252a7e1 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -356,9 +356,8 @@ const ProjectionDescription & ProjectionsDescription::get(const String & project auto it = map.find(projection_name); if (it == map.end()) { - String exception_message = fmt::format("There is no projection {} in table", projection_name); - appendHintsMessage(exception_message, projection_name); - throw Exception::createDeprecated(exception_message, ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE); + throw Exception(ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE, "There is no projection {} in table{}", + projection_name, getHintsMessage(projection_name)); } return *(it->second); @@ -401,9 +400,8 @@ void ProjectionsDescription::remove(const String & projection_name, bool if_exis if (if_exists) return; - String exception_message = fmt::format("There is no projection {} in table", projection_name); - appendHintsMessage(exception_message, projection_name); - throw Exception::createDeprecated(exception_message, ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE); + throw Exception(ErrorCodes::NO_SUCH_PROJECTION_IN_TABLE, "There is no projection {} in table{}", + projection_name, getHintsMessage(projection_name)); } projections.erase(it->second); diff --git a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp index de4e0a806e0..4624566a517 100644 --- a/src/Storages/S3Queue/S3QueueFilesMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueFilesMetadata.cpp @@ -339,7 +339,7 @@ std::shared_ptr S3QueueFilesMetadata::acquireLock(z } else if (code != Coordination::Error::ZOK) { - throw Coordination::Exception(code, zookeeper_lock_path); + throw Coordination::Exception::fromPath(code, zookeeper_lock_path); } else { diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index f2b1b907832..f98728c012e 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -419,7 +419,7 @@ StorageKeeperMap::StorageKeeperMap( } else if (code != Coordination::Error::ZOK) { - throw Coordination::Exception(code, dropped_lock_path); + throw Coordination::Exception::fromPath(code, dropped_lock_path); } else { @@ -918,7 +918,7 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca { auto code = client->tryRemove(delete_request->getPath()); if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE) - throw zkutil::KeeperException(code, delete_request->getPath()); + throw zkutil::KeeperException::fromPath(code, delete_request->getPath()); } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7fce373e26b..39c911d4f20 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -713,7 +713,7 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes() { auto res = future.get(); if (res.error != Coordination::Error::ZOK && res.error != Coordination::Error::ZNODEEXISTS) - throw Coordination::Exception(fmt::format("Failed to create new nodes at {}", zookeeper_path), res.error); + throw Coordination::Exception(res.error, "Failed to create new nodes at {}", zookeeper_path); } } @@ -749,7 +749,7 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr } else if (code != Coordination::Error::ZOK) { - throw Coordination::Exception(code, drop_lock_path); + throw Coordination::Exception::fromPath(code, drop_lock_path); } else { @@ -4307,7 +4307,7 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name, bool is_ continue; } else - throw Coordination::Exception(code, quorum_status_path); + throw Coordination::Exception::fromPath(code, quorum_status_path); } else { @@ -4331,7 +4331,7 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name, bool is_ continue; } else - throw Coordination::Exception(code, quorum_status_path); + throw Coordination::Exception::fromPath(code, quorum_status_path); } } } @@ -4389,7 +4389,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) continue; } else - throw Coordination::Exception(code, quorum_last_part_path); + throw Coordination::Exception::fromPath(code, quorum_last_part_path); } } @@ -5888,7 +5888,7 @@ void StorageReplicatedMergeTree::alter( } else { - throw Coordination::Exception("Alter cannot be assigned because of Zookeeper error", rc); + throw Coordination::Exception::fromMessage(rc, "Alter cannot be assigned because of Zookeeper error"); } } @@ -7049,7 +7049,7 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, Conte continue; } else - throw Coordination::Exception("Unable to create a mutation znode", rc); + throw Coordination::Exception::fromMessage(rc, "Unable to create a mutation znode"); } merge_selecting_task->schedule(); @@ -8672,7 +8672,7 @@ void StorageReplicatedMergeTree::createTableSharedID() const } else if (code != Coordination::Error::ZOK) { - throw zkutil::KeeperException(code, zookeeper_table_id_path); + throw zkutil::KeeperException::fromPath(code, zookeeper_table_id_path); } } @@ -9115,7 +9115,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( } else { - throw zkutil::KeeperException(ec, zookeeper_part_replica_node); + throw zkutil::KeeperException::fromPath(ec, zookeeper_part_replica_node); } } @@ -9150,7 +9150,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( } else { - throw zkutil::KeeperException(error_code, zookeeper_part_uniq_node); + throw zkutil::KeeperException::fromPath(error_code, zookeeper_part_uniq_node); } @@ -9182,7 +9182,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( } else { - throw zkutil::KeeperException(error_code, zookeeper_part_uniq_node); + throw zkutil::KeeperException::fromPath(error_code, zookeeper_part_uniq_node); } } else diff --git a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp index 67867b6c577..bae7a266dcd 100644 --- a/src/Storages/System/StorageSystemDDLWorkerQueue.cpp +++ b/src/Storages/System/StorageSystemDDLWorkerQueue.cpp @@ -290,7 +290,7 @@ void StorageSystemDDLWorkerQueue::fillData(MutableColumns & res_columns, Context } else { - throw Coordination::Exception(maybe_finished_hosts.error, fs::path(task.entry_path) / "finished"); + throw Coordination::Exception::fromPath(maybe_finished_hosts.error, fs::path(task.entry_path) / "finished"); } /// Process active nodes @@ -322,7 +322,7 @@ void StorageSystemDDLWorkerQueue::fillData(MutableColumns & res_columns, Context } else { - throw Coordination::Exception(maybe_active_hosts.error, fs::path(task.entry_path) / "active"); + throw Coordination::Exception::fromPath(maybe_active_hosts.error, fs::path(task.entry_path) / "active"); } /// Process the rest hosts diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference index 11b660b54a3..00a2cd14700 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference @@ -1,5 +1,6 @@ runtime messages 0.001 runtime exceptions 0.05 +unknown runtime exceptions 0.01 messages shorter than 10 1 messages shorter than 16 3 exceptions shorter than 30 3 diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index eb8e9826eff..2ae95928b75 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -12,7 +12,14 @@ create view logs as select * from system.text_log where now() - toIntervalMinute select 'runtime messages', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.001) from logs; -- Check the same for exceptions. The value was 0.03 -select 'runtime exceptions', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.05) from logs where message like '%DB::Exception%'; +select 'runtime exceptions', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.05) from logs + where (message like '%DB::Exception%' or message like '%Coordination::Exception%') + and message not like '%Received from CLICKHOUSE_CI_LOGS_HOST:9440%'; + +select 'unknown runtime exceptions', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.01) from logs where + (message like '%DB::Exception%' or message like '%Coordination::Exception%') + and message not like '%Received from CLICKHOUSE_CI_LOGS_HOST:9440%' + and message not like '% Received from %' and message not like '%(SYNTAX_ERROR)%'; -- FIXME some of the following messages are not informative and it has to be fixed create temporary table known_short_messages (s String) as select * from (select diff --git a/tests/queries/0_stateless/00921_datetime64_basic.sql b/tests/queries/0_stateless/00921_datetime64_basic.sql index 28205c72ef0..13abe3e64d0 100644 --- a/tests/queries/0_stateless/00921_datetime64_basic.sql +++ b/tests/queries/0_stateless/00921_datetime64_basic.sql @@ -3,17 +3,17 @@ DROP TABLE IF EXISTS A; SELECT CAST(1 as DateTime64('abc')); -- { serverError 43 } # Invalid scale parameter type SELECT CAST(1 as DateTime64(100)); -- { serverError 69 } # too big scale SELECT CAST(1 as DateTime64(-1)); -- { serverError 43 } # signed scale parameter type -SELECT CAST(1 as DateTime64(3, 'qqq')); -- { serverError 1000 } # invalid timezone +SELECT CAST(1 as DateTime64(3, 'qqq')); -- { serverError BAD_ARGUMENTS } # invalid timezone SELECT toDateTime64('2019-09-16 19:20:11.234', 'abc'); -- { serverError 43 } # invalid scale SELECT toDateTime64('2019-09-16 19:20:11.234', 100); -- { serverError 69 } # too big scale SELECT toDateTime64(CAST([['CLb5Ph ']], 'String'), uniqHLL12('2Gs1V', 752)); -- { serverError 44 } # non-const string and non-const scale -SELECT toDateTime64('2019-09-16 19:20:11.234', 3, 'qqq'); -- { serverError 1000 } # invalid timezone +SELECT toDateTime64('2019-09-16 19:20:11.234', 3, 'qqq'); -- { serverError BAD_ARGUMENTS } # invalid timezone SELECT ignore(now64(gccMurmurHash())); -- { serverError 43 } # Illegal argument type SELECT ignore(now64('abcd')); -- { serverError 43 } # Illegal argument type SELECT ignore(now64(number)) FROM system.numbers LIMIT 10; -- { serverError 43 } # Illegal argument type -SELECT ignore(now64(3, 'invalid timezone')); -- { serverError 1000 } +SELECT ignore(now64(3, 'invalid timezone')); -- { serverError BAD_ARGUMENTS } SELECT ignore(now64(3, 1111)); -- { serverError 44 } # invalid timezone parameter type WITH 'UTC' as timezone SELECT timezone, timeZoneOf(now64(3, timezone)) == timezone; diff --git a/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql b/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql index ac1186284be..c1cec6ea212 100644 --- a/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql +++ b/tests/queries/0_stateless/01281_parseDateTime64BestEffort.sql @@ -5,7 +5,7 @@ SELECT parseDateTime64BestEffort('foo'); -- {serverError 41} SELECT parseDateTime64BestEffort('2020-05-14T03:37:03.253184Z', 'bar'); -- {serverError 43} -- invalid scale parameter SELECT parseDateTime64BestEffort('2020-05-14T03:37:03.253184Z', 3, 4); -- {serverError 43} -- invalid timezone parameter -SELECT parseDateTime64BestEffort('2020-05-14T03:37:03.253184Z', 3, 'baz'); -- {serverError 1000} -- unknown timezone +SELECT parseDateTime64BestEffort('2020-05-14T03:37:03.253184Z', 3, 'baz'); -- {serverError BAD_ARGUMENTS} -- unknown timezone SELECT parseDateTime64BestEffort('2020-05-14T03:37:03.253184Z', materialize(3), 4); -- {serverError 44} -- non-const precision SELECT parseDateTime64BestEffort('2020-05-14T03:37:03.253184Z', 3, materialize('UTC')); -- {serverError 44} -- non-const timezone diff --git a/tests/queries/0_stateless/02244_make_datetime.sql b/tests/queries/0_stateless/02244_make_datetime.sql index 9b8f561994b..a3d88d89682 100644 --- a/tests/queries/0_stateless/02244_make_datetime.sql +++ b/tests/queries/0_stateless/02244_make_datetime.sql @@ -20,7 +20,7 @@ select makeDateTime(1984, 1, 41, 0, 0, 0, 'UTC'); select makeDateTime(1984, 1, 1, 25, 0, 0, 'UTC'); select makeDateTime(1984, 1, 1, 0, 70, 0, 'UTC'); select makeDateTime(1984, 1, 1, 0, 0, 70, 'UTC'); -select makeDateTime(1984, 1, 1, 0, 0, 0, 'not a timezone'); -- { serverError 1000 } +select makeDateTime(1984, 1, 1, 0, 0, 0, 'not a timezone'); -- { serverError BAD_ARGUMENTS } select makeDateTime(1984, 1, 1, 0, 0, 0, 'UTC'); select makeDateTime(1983, 2, 29, 0, 0, 0, 'UTC'); diff --git a/tests/queries/0_stateless/02245_make_datetime64.sql b/tests/queries/0_stateless/02245_make_datetime64.sql index 054cbd8956d..a7b3a3d23c5 100644 --- a/tests/queries/0_stateless/02245_make_datetime64.sql +++ b/tests/queries/0_stateless/02245_make_datetime64.sql @@ -38,7 +38,7 @@ select makeDateTime64(1984, 1, 41, 0, 0, 0, 0, 9, 'UTC'); select makeDateTime64(1984, 1, 1, 25, 0, 0, 0, 9, 'UTC'); select makeDateTime64(1984, 1, 1, 0, 70, 0, 0, 9, 'UTC'); select makeDateTime64(1984, 1, 1, 0, 0, 70, 0, 9, 'UTC'); -select makeDateTime64(1984, 1, 1, 0, 0, 0, 0, 9, 'not a timezone'); -- { serverError 1000 } +select makeDateTime64(1984, 1, 1, 0, 0, 0, 0, 9, 'not a timezone'); -- { serverError BAD_ARGUMENTS } select makeDateTime64(1984, 1, 1, 2, 3, 4, 5, 9, 'UTC'); select makeDateTime64(1984, 2, 29, 2, 3, 4, 5, 9, 'UTC'); diff --git a/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql b/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql index b08abcb8a19..63edad6c9e3 100644 --- a/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql +++ b/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql @@ -1,6 +1,6 @@ -select toDateTime(0, '/abc'); -- { serverError POCO_EXCEPTION } -select toDateTime(0, './abc'); -- { serverError POCO_EXCEPTION } -select toDateTime(0, '../abc'); -- { serverError POCO_EXCEPTION } -select toDateTime(0, '~/abc'); -- { serverError POCO_EXCEPTION } -select toDateTime(0, 'abc/../../cba'); -- { serverError POCO_EXCEPTION } +select toDateTime(0, '/abc'); -- { serverError BAD_ARGUMENTS } +select toDateTime(0, './abc'); -- { serverError BAD_ARGUMENTS } +select toDateTime(0, '../abc'); -- { serverError BAD_ARGUMENTS } +select toDateTime(0, '~/abc'); -- { serverError BAD_ARGUMENTS } +select toDateTime(0, 'abc/../../cba'); -- { serverError BAD_ARGUMENTS } From 7312de59c508932934c3bc8aa03818d74215e343 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Aug 2023 23:33:30 +0200 Subject: [PATCH 547/777] empty commit From 1e3f9c8cfeb9a3e6e51069881155fbc9dad53143 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 15 Aug 2023 00:41:24 +0300 Subject: [PATCH 548/777] Merging #53142 (#53431) * Added session_log events to text_log * user error severity instead of debug for failure * updated test expectation * added user_id to logout message * empty commit --------- Co-authored-by: Alexey Gerasimchuck --- src/Interpreters/Session.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index f8bd70afdb6..bcfaae40a03 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -299,6 +299,7 @@ Session::~Session() if (notified_session_log_about_login) { + LOG_DEBUG(log, "{} Logout, user_id: {}", toString(auth_id), toString(*user_id)); if (auto session_log = getSessionLog()) { /// TODO: We have to ensure that the same info is added to the session log on a LoginSuccess event and on the corresponding Logout event. @@ -320,6 +321,7 @@ AuthenticationType Session::getAuthenticationTypeOrLogInFailure(const String & u } catch (const Exception & e) { + LOG_ERROR(log, "{} Authentication failed with error: {}", toString(auth_id), e.what()); if (auto session_log = getSessionLog()) session_log->addLoginFailure(auth_id, getClientInfo(), user_name, e); From 84131740fdfb7fd7f4c1240f019b239d71d60f2f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 15 Aug 2023 00:22:05 +0200 Subject: [PATCH 549/777] Fix sanitizer error --- src/Planner/PlannerJoinTree.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 11de6fcfabe..f6ce029a295 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -842,8 +842,9 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } else { + SelectQueryOptions analyze_query_options = SelectQueryOptions(from_stage).analyze(); Planner planner(select_query_info.query_tree, - SelectQueryOptions(from_stage).analyze(), + analyze_query_options, select_query_info.planner_context); planner.buildQueryPlanIfNeeded(); From 376202f7392032131026aa5f46389f99f66638b8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Aug 2023 23:29:28 +0200 Subject: [PATCH 550/777] fix creation of empty parts --- src/Storages/MergeTree/MergeTreeData.cpp | 5 ++-- src/Storages/MergeTree/MergeTreeData.h | 4 ++- src/Storages/StorageMergeTree.cpp | 31 ++++++++------------- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 4 files changed, 18 insertions(+), 24 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index db0a7b34d7e..da0a6328894 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8435,7 +8435,7 @@ void MergeTreeData::incrementMergedPartsProfileEvent(MergeTreeDataPartType type) } } -MergeTreeData::MutableDataPartPtr MergeTreeData::createEmptyPart( +std::pair MergeTreeData::createEmptyPart( MergeTreePartInfo & new_part_info, const MergeTreePartition & partition, const String & new_part_name, const MergeTreeTransactionPtr & txn) { @@ -8454,6 +8454,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createEmptyPart( ReservationPtr reservation = reserveSpacePreferringTTLRules(metadata_snapshot, 0, move_ttl_infos, time(nullptr), 0, true); VolumePtr data_part_volume = createVolumeFromReservation(reservation, volume); + auto tmp_dir_holder = getTemporaryPartDirectoryHolder(EMPTY_PART_TMP_PREFIX + new_part_name); auto new_data_part = getDataPartBuilder(new_part_name, data_part_volume, EMPTY_PART_TMP_PREFIX + new_part_name) .withBytesAndRowsOnDisk(0, 0) .withPartInfo(new_part_info) @@ -8513,7 +8514,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createEmptyPart( out.finalizePart(new_data_part, sync_on_insert); new_data_part_storage->precommitTransaction(); - return new_data_part; + return std::make_pair(std::move(new_data_part), std::move(tmp_dir_holder)); } bool MergeTreeData::allowRemoveStaleMovingParts() const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 9ee61134740..e4801cffa36 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -936,7 +936,9 @@ public: WriteAheadLogPtr getWriteAheadLog(); constexpr static auto EMPTY_PART_TMP_PREFIX = "tmp_empty_"; - MergeTreeData::MutableDataPartPtr createEmptyPart(MergeTreePartInfo & new_part_info, const MergeTreePartition & partition, const String & new_part_name, const MergeTreeTransactionPtr & txn); + std::pair createEmptyPart( + MergeTreePartInfo & new_part_info, const MergeTreePartition & partition, + const String & new_part_name, const MergeTreeTransactionPtr & txn); MergeTreeDataFormatVersion format_version; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ad9013d9f13..a22c1355015 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1653,11 +1653,7 @@ struct FutureNewEmptyPart MergeTreePartition partition; std::string part_name; - scope_guard tmp_dir_guard; - StorageMergeTree::MutableDataPartPtr data_part; - - std::string getDirName() const { return StorageMergeTree::EMPTY_PART_TMP_PREFIX + part_name; } }; using FutureNewEmptyParts = std::vector; @@ -1688,19 +1684,19 @@ FutureNewEmptyParts initCoverageWithNewEmptyParts(const DataPartsVector & old_pa return future_parts; } -StorageMergeTree::MutableDataPartsVector createEmptyDataParts(MergeTreeData & data, FutureNewEmptyParts & future_parts, const MergeTreeTransactionPtr & txn) +std::pair> createEmptyDataParts( + MergeTreeData & data, FutureNewEmptyParts & future_parts, const MergeTreeTransactionPtr & txn) { - StorageMergeTree::MutableDataPartsVector data_parts; + std::pair> data_parts; for (auto & part: future_parts) - data_parts.push_back(data.createEmptyPart(part.part_info, part.partition, part.part_name, txn)); + { + auto [new_data_part, tmp_dir_holder] = data.createEmptyPart(part.part_info, part.partition, part.part_name, txn); + data_parts.first.emplace_back(std::move(new_data_part)); + data_parts.second.emplace_back(std::move(tmp_dir_holder)); + } return data_parts; } -void captureTmpDirectoryHolders(MergeTreeData & data, FutureNewEmptyParts & future_parts) -{ - for (auto & part : future_parts) - part.tmp_dir_guard = data.getTemporaryPartDirectoryHolder(part.getDirName()); -} void StorageMergeTree::renameAndCommitEmptyParts(MutableDataPartsVector & new_parts, Transaction & transaction) { @@ -1767,9 +1763,7 @@ void StorageMergeTree::truncate(const ASTPtr &, const StorageMetadataPtr &, Cont fmt::join(getPartsNames(future_parts), ", "), fmt::join(getPartsNames(parts), ", "), transaction.getTID()); - captureTmpDirectoryHolders(*this, future_parts); - - auto new_data_parts = createEmptyDataParts(*this, future_parts, txn); + auto [new_data_parts, tmp_dir_holders] = createEmptyDataParts(*this, future_parts, txn); renameAndCommitEmptyParts(new_data_parts, transaction); PartLog::addNewParts(query_context, PartLog::createPartLogEntries(new_data_parts, watch.elapsed(), profile_events_scope.getSnapshot())); @@ -1828,9 +1822,7 @@ void StorageMergeTree::dropPart(const String & part_name, bool detach, ContextPt fmt::join(getPartsNames(future_parts), ", "), fmt::join(getPartsNames({part}), ", "), transaction.getTID()); - captureTmpDirectoryHolders(*this, future_parts); - - auto new_data_parts = createEmptyDataParts(*this, future_parts, txn); + auto [new_data_parts, tmp_dir_holders] = createEmptyDataParts(*this, future_parts, txn); renameAndCommitEmptyParts(new_data_parts, transaction); PartLog::addNewParts(query_context, PartLog::createPartLogEntries(new_data_parts, watch.elapsed(), profile_events_scope.getSnapshot())); @@ -1914,9 +1906,8 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont fmt::join(getPartsNames(future_parts), ", "), fmt::join(getPartsNames(parts), ", "), transaction.getTID()); - captureTmpDirectoryHolders(*this, future_parts); - auto new_data_parts = createEmptyDataParts(*this, future_parts, txn); + auto [new_data_parts, tmp_dir_holders] = createEmptyDataParts(*this, future_parts, txn); renameAndCommitEmptyParts(new_data_parts, transaction); PartLog::addNewParts(query_context, PartLog::createPartLogEntries(new_data_parts, watch.elapsed(), profile_events_scope.getSnapshot())); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7fce373e26b..a1bf04c0ead 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -9509,7 +9509,7 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP } } - MergeTreeData::MutableDataPartPtr new_data_part = createEmptyPart(new_part_info, partition, lost_part_name, NO_TRANSACTION_PTR); + auto [new_data_part, tmp_dir_holder] = createEmptyPart(new_part_info, partition, lost_part_name, NO_TRANSACTION_PTR); new_data_part->setName(lost_part_name); try From c94994afcfea3ad3948cd9bd789f584a76b782a0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 14 Aug 2023 22:50:46 +0000 Subject: [PATCH 551/777] Propagate setting cluster_for_parallel_replicas to shards --- src/Interpreters/Cluster.h | 2 ++ src/Interpreters/ClusterProxy/executeQuery.cpp | 9 ++++++--- src/Processors/QueryPlan/ReadFromRemote.cpp | 17 ++++++++++++++++- src/Processors/QueryPlan/ReadFromRemote.h | 6 ++++-- src/Storages/StorageMergeTree.cpp | 13 ++++++++++++- src/Storages/StorageReplicatedMergeTree.cpp | 13 ++++++++++++- .../test.py | 10 +++++----- ...02835_parallel_replicas_over_distributed.sql | 8 ++++---- 8 files changed, 61 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index b90acd1d576..f509a2d9847 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -273,6 +273,8 @@ public: /// Are distributed DDL Queries (ON CLUSTER Clause) allowed for this cluster bool areDistributedDDLQueriesAllowed() const { return allow_distributed_ddl_queries; } + String getName() const { return name; } + private: SlotToShard slot_to_shard; diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index fbc7bbd5bbb..e335df3b2c5 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -234,7 +234,8 @@ void executeQuery( std::move(external_tables), log, shards, - query_info.storage_limits); + query_info.storage_limits, + query_info.getCluster()->getName()); read_from_remote->setStepDescription("Read from remote replica"); plan->addStep(std::move(read_from_remote)); @@ -266,14 +267,16 @@ void executeQueryWithParallelReplicas( const StorageID & main_table, const ASTPtr & table_func_ptr, SelectStreamFactory & stream_factory, - const ASTPtr & query_ast, ContextPtr context, const SelectQueryInfo & query_info, + const ASTPtr & query_ast, + ContextPtr context, + const SelectQueryInfo & query_info, const ClusterPtr & not_optimized_cluster) { const auto & settings = context->getSettingsRef(); auto new_context = Context::createCopy(context); auto scalars = new_context->hasQueryContext() ? new_context->getQueryContext()->getScalars() : Scalars{}; - Int64 shard_num = 0; /// shard_num is 1-based, so 0 - no shard specified + UInt64 shard_num = 0; /// shard_num is 1-based, so 0 - no shard specified auto it = scalars.find("_shard_num"); if (it != scalars.end()) { diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 576349844ec..e9cf80b0808 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -103,7 +104,8 @@ ReadFromRemote::ReadFromRemote( Tables external_tables_, Poco::Logger * log_, UInt32 shard_count_, - std::shared_ptr storage_limits_) + std::shared_ptr storage_limits_, + const String & cluster_name_) : ISourceStep(DataStream{.header = std::move(header_)}) , shards(std::move(shards_)) , stage(stage_) @@ -116,6 +118,7 @@ ReadFromRemote::ReadFromRemote( , storage_limits(std::move(storage_limits_)) , log(log_) , shard_count(shard_count_) + , cluster_name(cluster_name_) { } @@ -234,6 +237,16 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact scalars["_shard_num"] = Block{{DataTypeUInt32().createColumnConst(1, shard.shard_info.shard_num), std::make_shared(), "_shard_num"}}; + if (context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS) + { + String cluster_for_parallel_replicas = cluster_name; + LOG_DEBUG(&Poco::Logger::get(__FUNCTION__), "_cluster_for_parallel_replicas: {}", cluster_for_parallel_replicas); + scalars["_cluster_for_parallel_replicas"] = Block{ + {DataTypeString().createColumnConst(1, cluster_for_parallel_replicas), + std::make_shared(), + "_cluster_for_parallel_replicas"}}; + } + std::shared_ptr remote_query_executor; remote_query_executor = std::make_shared( @@ -242,6 +255,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact remote_query_executor->setLogger(log); if (context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS) + { // when doing parallel reading from replicas (ParallelReplicasMode::READ_TASKS) on a shard: // establish a connection to a replica on the shard, the replica will instantiate coordinator to manage parallel reading from replicas on the shard. // The coordinator will return query result from the shard. @@ -249,6 +263,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact // Using PoolMode::GET_MANY for this mode will(can) lead to instantiation of several coordinators (depends on max_parallel_replicas setting) // each will execute parallel reading from replicas, so the query result will be multiplied by the number of created coordinators remote_query_executor->setPoolMode(PoolMode::GET_ONE); + } else remote_query_executor->setPoolMode(PoolMode::GET_MANY); diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index ac869cd89f9..27e640970ce 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -35,7 +35,8 @@ public: Tables external_tables_, Poco::Logger * log_, UInt32 shard_count_, - std::shared_ptr storage_limits_); + std::shared_ptr storage_limits_, + const String & cluster_name_); String getName() const override { return "ReadFromRemote"; } @@ -55,8 +56,9 @@ private: Tables external_tables; std::shared_ptr storage_limits; Poco::Logger * log; - UInt32 shard_count; + String cluster_name; + void addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard); void addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard); }; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ad9013d9f13..25f64ec996d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -220,7 +220,18 @@ void StorageMergeTree::read( local_context, query_info.query, table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); - auto cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas); + auto scalars = local_context->hasQueryContext() ? local_context->getQueryContext()->getScalars() : Scalars{}; + String cluster_for_parallel_replicas = local_context->getSettingsRef().cluster_for_parallel_replicas; + { + auto it = scalars.find("_cluster_for_parallel_replicas"); + if (it != scalars.end()) + { + const Block & block = it->second; + cluster_for_parallel_replicas = block.getColumns()[0]->getDataAt(0).toString(); + } + } + LOG_DEBUG(&Poco::Logger::get("StorageMergeTree::read"), "_cluster_for_parallel_replicas: {}", cluster_for_parallel_replicas); + auto cluster = local_context->getCluster(cluster_for_parallel_replicas); Block header; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7fce373e26b..5513cc90fff 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5153,7 +5153,18 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl( { auto table_id = getStorageID(); - auto parallel_replicas_cluster = local_context->getCluster(local_context->getSettingsRef().cluster_for_parallel_replicas); + auto scalars = local_context->hasQueryContext() ? local_context->getQueryContext()->getScalars() : Scalars{}; + String cluster_for_parallel_replicas = local_context->getSettingsRef().cluster_for_parallel_replicas; + { + auto it = scalars.find("_cluster_for_parallel_replicas"); + if (it != scalars.end()) + { + const Block & block = it->second; + cluster_for_parallel_replicas = block.getColumns()[0]->getDataAt(0).toString(); + } + } + LOG_DEBUG(&Poco::Logger::get(__FUNCTION__), "_cluster_for_parallel_replicas: {}", cluster_for_parallel_replicas); + auto parallel_replicas_cluster = local_context->getCluster(cluster_for_parallel_replicas); ASTPtr modified_query_ast; Block header; diff --git a/tests/integration/test_parallel_replicas_over_distributed/test.py b/tests/integration/test_parallel_replicas_over_distributed/test.py index 5ad191af331..51ce86d828f 100644 --- a/tests/integration/test_parallel_replicas_over_distributed/test.py +++ b/tests/integration/test_parallel_replicas_over_distributed/test.py @@ -105,10 +105,10 @@ def test_parallel_replicas_over_distributed(start_cluster, cluster): expected_result = f"6001\t-1999\t1999\t0\n" # w/o parallel replicas - assert ( - node.query(f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d") - == expected_result - ) + # assert ( + # node.query(f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d") + # == expected_result + # ) # parallel replicas assert ( @@ -119,7 +119,7 @@ def test_parallel_replicas_over_distributed(start_cluster, cluster): "prefer_localhost_replica": 0, "max_parallel_replicas": 4, "use_hedged_requests": 0, - "cluster_for_parallel_replicas": cluster, + # "cluster_for_parallel_replicas": cluster, }, ) == expected_result diff --git a/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql index 4e7c72ebfd8..30dfb32678d 100644 --- a/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql +++ b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql @@ -14,13 +14,13 @@ insert into test select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; --, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; insert into test select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; --, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; -- 2 shards @@ -38,10 +38,10 @@ insert into test2 select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test2_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0, cluster_for_parallel_replicas = 'test_cluster_two_shard_three_replicas_localhost'; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; --, cluster_for_parallel_replicas = 'test_cluster_two_shard_three_replicas_localhost'; insert into test2 select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test2_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0, cluster_for_parallel_replicas = 'test_cluster_two_shard_three_replicas_localhost'; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; --, cluster_for_parallel_replicas = 'test_cluster_two_shard_three_replicas_localhost'; From c85986a5dae9b0ff8c05992970438b36da327bd6 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 14 Aug 2023 23:06:48 +0000 Subject: [PATCH 552/777] Update test --- .../test_parallel_replicas_over_distributed/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_parallel_replicas_over_distributed/test.py b/tests/integration/test_parallel_replicas_over_distributed/test.py index 51ce86d828f..ee52c8c040e 100644 --- a/tests/integration/test_parallel_replicas_over_distributed/test.py +++ b/tests/integration/test_parallel_replicas_over_distributed/test.py @@ -105,10 +105,10 @@ def test_parallel_replicas_over_distributed(start_cluster, cluster): expected_result = f"6001\t-1999\t1999\t0\n" # w/o parallel replicas - # assert ( - # node.query(f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d") - # == expected_result - # ) + assert ( + node.query(f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d") + == expected_result + ) # parallel replicas assert ( From e82b6b02cafa427547253105ebe3c58828738a99 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 14 Aug 2023 23:15:27 +0000 Subject: [PATCH 553/777] Fix style --- src/Processors/QueryPlan/ReadFromRemote.cpp | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index e9cf80b0808..425728a2e80 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -239,12 +239,9 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact if (context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS) { - String cluster_for_parallel_replicas = cluster_name; - LOG_DEBUG(&Poco::Logger::get(__FUNCTION__), "_cluster_for_parallel_replicas: {}", cluster_for_parallel_replicas); + LOG_DEBUG(&Poco::Logger::get(__FUNCTION__), "_cluster_for_parallel_replicas: {}", cluster_name); scalars["_cluster_for_parallel_replicas"] = Block{ - {DataTypeString().createColumnConst(1, cluster_for_parallel_replicas), - std::make_shared(), - "_cluster_for_parallel_replicas"}}; + {DataTypeString().createColumnConst(1, cluster_name), std::make_shared(), "_cluster_for_parallel_replicas"}}; } std::shared_ptr remote_query_executor; From 9f7a25eeadcd9c78eb61ec1c97cb46e022377600 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 14 Aug 2023 23:27:36 +0000 Subject: [PATCH 554/777] Fix style once more --- src/Processors/QueryPlan/ReadFromRemote.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 425728a2e80..cbcaca3e971 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -240,8 +240,8 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact if (context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS) { LOG_DEBUG(&Poco::Logger::get(__FUNCTION__), "_cluster_for_parallel_replicas: {}", cluster_name); - scalars["_cluster_for_parallel_replicas"] = Block{ - {DataTypeString().createColumnConst(1, cluster_name), std::make_shared(), "_cluster_for_parallel_replicas"}}; + scalars["_cluster_for_parallel_replicas"] = + Block{{DataTypeString().createColumnConst(1, cluster_name), std::make_shared(), "_cluster_for_parallel_replicas"}}; } std::shared_ptr remote_query_executor; From 5a1265d16d56b0bfae665bf2474c515de157f050 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 15 Aug 2023 00:23:08 +0000 Subject: [PATCH 555/777] add docs --- docs/en/operations/settings/settings.md | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index de3cc00e4c9..29434ef556f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -98,6 +98,18 @@ Default value: 0. ``` +## mutations_execute_nondeterministic_on_initiator {#mutations_execute_nondeterministic_on_initiator} + +If true constant nondeterministic functions (e.g. function `now()`) are executed on initiator and replaced to literals in `UPDATE` and `DELETE` queries. It helps to keep data in sync on replicas while executing mutations with constant nondeterministic functions. Default value: `false`. + +## mutations_execute_subqueries_on_initiator {#mutations_execute_subqueries_on_initiator} + +If true scalar subqueries are executed on initiator and replaced to literals in `UPDATE` and `DELETE` queries. Default value: `false`. + +## mutations_max_literal_size_to_replace {#mutations_max_literal_size_to_replace} + +The maximum size of serialized literal in bytes to replace in `UPDATE` and `DELETE` queries. Takes effect only if at least one the two settings above is enabled. Default value: 16384 (16 KiB). + ## distributed_product_mode {#distributed-product-mode} Changes the behaviour of [distributed subqueries](../../sql-reference/operators/in.md). @@ -4298,7 +4310,7 @@ Use this setting only for backward compatibility if your use cases depend on old ## session_timezone {#session_timezone} Sets the implicit time zone of the current session or query. -The implicit time zone is the time zone applied to values of type DateTime/DateTime64 which have no explicitly specified time zone. +The implicit time zone is the time zone applied to values of type DateTime/DateTime64 which have no explicitly specified time zone. The setting takes precedence over the globally configured (server-level) implicit time zone. A value of '' (empty string) means that the implicit time zone of the current session or query is equal to the [server time zone](../server-configuration-parameters/settings.md#server_configuration_parameters-timezone). @@ -4333,7 +4345,7 @@ SELECT toDateTime64(toDateTime64('1999-12-12 23:23:23.123', 3), 3, 'Europe/Zuric ``` :::warning -Not all functions that parse DateTime/DateTime64 respect `session_timezone`. This can lead to subtle errors. +Not all functions that parse DateTime/DateTime64 respect `session_timezone`. This can lead to subtle errors. See the following example and explanation. ::: From bfa9d361ccfde6c548cac3e2ffbd337ea2d0ee76 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 15 Aug 2023 12:16:22 +0800 Subject: [PATCH 556/777] fix tests --- src/IO/ReadBufferFromString.h | 5 ++- src/Processors/CMakeLists.txt | 4 +- .../Impl/NativeORCBlockInputFormat.cpp | 45 +++++++++++++------ .../Formats/Impl/NativeORCBlockInputFormat.h | 27 +++++++---- src/Processors/examples/native_orc.cpp | 30 +++++++++++-- 5 files changed, 82 insertions(+), 29 deletions(-) diff --git a/src/IO/ReadBufferFromString.h b/src/IO/ReadBufferFromString.h index 7ea6afc3543..f20e319b931 100644 --- a/src/IO/ReadBufferFromString.h +++ b/src/IO/ReadBufferFromString.h @@ -19,7 +19,10 @@ public: class ReadBufferFromOwnString : public String, public ReadBufferFromString { public: - explicit ReadBufferFromOwnString(const String & s_): String(s_), ReadBufferFromString(*this) {} + template + explicit ReadBufferFromOwnString(S && s_) : String(std::forward(s_)), ReadBufferFromString(*this) + { + } }; } diff --git a/src/Processors/CMakeLists.txt b/src/Processors/CMakeLists.txt index 7e965188b4c..7c9ad405432 100644 --- a/src/Processors/CMakeLists.txt +++ b/src/Processors/CMakeLists.txt @@ -1,4 +1,4 @@ -if (ENABLE_EXAMPLES) +#if (ENABLE_EXAMPLES) add_subdirectory(examples) -endif () +#endif () diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 98efd14f343..c259cd8907d 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -1,4 +1,5 @@ #include "NativeORCBlockInputFormat.h" +#include "ArrowBufferedStreams.h" #if USE_ORC # include @@ -39,17 +40,17 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int UNKNOWN_TYPE; extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; - extern const int BAD_ARGUMENTS; extern const int THERE_IS_NO_COLUMN; + extern const int INCORRECT_DATA; } -ORCInputStream::ORCInputStream(SeekableReadBuffer & in_) : in(in_) +ORCInputStream::ORCInputStream(SeekableReadBuffer & in_, size_t file_size_) : in(in_), file_size(file_size_) { } uint64_t ORCInputStream::getLength() const { - return getFileSizeFromReadBuffer(in); + return file_size; } uint64_t ORCInputStream::getNaturalReadSize() const @@ -65,17 +66,34 @@ void ORCInputStream::read(void * buf, uint64_t length, uint64_t offset) in.readStrict(reinterpret_cast(buf), length); } -std::unique_ptr asORCInputStream(ReadBuffer & in) +std::unique_ptr asORCInputStream(ReadBuffer & in, const FormatSettings & settings, std::atomic & is_cancelled) { bool has_file_size = isBufferWithFileSize(in); - if (!has_file_size) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "ORC format supports only input with known size"); - auto * seekable_in = dynamic_cast(&in); - if (!seekable_in) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "ORC format supports only seekable input"); - return std::make_unique(*seekable_in); + if (has_file_size && seekable_in && settings.seekable_read) + return std::make_unique(*seekable_in, getFileSizeFromReadBuffer(in)); + + // fallback to loading the entire file in memory + return asORCInputStreamLoadIntoMemory(in, is_cancelled); +} + +std::unique_ptr asORCInputStreamLoadIntoMemory(ReadBuffer & in, std::atomic & is_cancelled) +{ + size_t magic_size = strlen(ORC_MAGIC_BYTES); + std::string file_data(magic_size, '\0'); + + /// Avoid loading the whole file if it doesn't seem to even be in the correct format. + size_t bytes_read = in.read(file_data.data(), magic_size); + if (bytes_read < magic_size || file_data != ORC_MAGIC_BYTES) + throw Exception(ErrorCodes::INCORRECT_DATA, "Not an ORC file"); + + WriteBufferFromString file_buffer(file_data, AppendModeTag{}); + copyData(in, file_buffer, is_cancelled); + file_buffer.finalize(); + + size_t file_size = file_data.size(); + return std::make_unique(std::move(file_data), file_size); } static DataTypePtr parseORCType(const orc::Type * orc_type) @@ -158,14 +176,14 @@ static void getFileReaderAndSchema( ReadBuffer & in, std::unique_ptr & file_reader, Block & header, - const FormatSettings & /*format_settings*/, + const FormatSettings & format_settings, std::atomic & is_stopped) { if (is_stopped) return; orc::ReaderOptions options; - auto input_stream = asORCInputStream(in); + auto input_stream = asORCInputStream(in, format_settings, is_stopped); file_reader = orc::createReader(std::move(input_stream), options); const auto & schema = file_reader->getType(); @@ -1050,8 +1068,7 @@ void registerInputFormatORC(FormatFactory & factory) void registerORCSchemaReader(FormatFactory & factory) { factory.registerSchemaReader( - "ORC", - [](ReadBuffer & buf, const FormatSettings & settings) { return std::make_shared(buf, settings); }); + "ORC", [](ReadBuffer & buf, const FormatSettings & settings) { return std::make_shared(buf, settings); }); factory.registerAdditionalInfoForSchemaCacheGetter( "ORC", diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h index 5fbc901b283..1535d81e815 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h @@ -1,4 +1,5 @@ #pragma once +#include "IO/ReadBufferFromString.h" #include "config.h" #if USE_ORC @@ -13,19 +14,33 @@ namespace DB class ORCInputStream : public orc::InputStream { public: - explicit ORCInputStream(SeekableReadBuffer & in_); + ORCInputStream(SeekableReadBuffer & in_, size_t file_size_); uint64_t getLength() const override; uint64_t getNaturalReadSize() const override; void read(void * buf, uint64_t length, uint64_t offset) override; const std::string & getName() const override { return name; } -private: +protected: SeekableReadBuffer & in; + size_t file_size; std::string name = "ORCInputStream"; }; -std::unique_ptr asORCInputStream(ReadBuffer & in); +class ORCInputStreamFromString : public ReadBufferFromOwnString, public ORCInputStream +{ +public: + template + ORCInputStreamFromString(S && s_, size_t file_size_) + : ReadBufferFromOwnString(std::forward(s_)), ORCInputStream(dynamic_cast(*this), file_size_) + { + } +}; + +std::unique_ptr asORCInputStream(ReadBuffer & in, const FormatSettings & settings, std::atomic & is_cancelled); + +// Reads the whole file into a memory buffer, owned by the returned RandomAccessFile. +std::unique_ptr asORCInputStreamLoadIntoMemory(ReadBuffer & in, std::atomic & is_cancelled); class ORCColumnToCHColumn; @@ -91,11 +106,7 @@ public: using ORCColumnWithType = std::pair; using NameToColumnPtr = std::unordered_map; - ORCColumnToCHColumn( - const Block & header_, - bool allow_missing_columns_, - bool null_as_default_, - bool case_insensitive_matching_ = false); + ORCColumnToCHColumn(const Block & header_, bool allow_missing_columns_, bool null_as_default_, bool case_insensitive_matching_ = false); void orcTableToCHChunk( Chunk & res, diff --git a/src/Processors/examples/native_orc.cpp b/src/Processors/examples/native_orc.cpp index 2c1715d3ce4..673dcd4c4e2 100644 --- a/src/Processors/examples/native_orc.cpp +++ b/src/Processors/examples/native_orc.cpp @@ -1,14 +1,36 @@ #include #include #include +#include using namespace DB; int main() { - ReadBufferFromFile in("/data1/clickhouse_official/data/user_files/bigolive_audience_stats_orc.orc"); - ORCSchemaReader schema_reader(in, {}); - auto schema = schema_reader.readSchema(); - std::cout << "schema:" << schema.toString() << std::endl; + /// Read schema from orc file + // String path = "/path/to/orc/file"; + String path = "/data1/clickhouse_official/data/user_files/bigolive_audience_stats_orc.orc"; + { + ReadBufferFromFile in(path); + ORCSchemaReader schema_reader(in, {}); + auto schema = schema_reader.readSchema(); + std::cout << "schema:" << schema.toString() << std::endl; + } + + /// Read schema from string with orc data + { + ReadBufferFromFile in(path); + + String content; + WriteBufferFromString out(content); + + copyData(in, out); + + content.resize(out.count()); + ReadBufferFromString in2(content); + ORCSchemaReader schema_reader(in2, {}); + auto schema = schema_reader.readSchema(); + std::cout << "schema:" << schema.toString() << std::endl; + } return 0; } From 724af579d7b78c823eeea6da372260c979d86908 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 15 Aug 2023 12:17:49 +0800 Subject: [PATCH 557/777] revert file --- src/Processors/CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/CMakeLists.txt b/src/Processors/CMakeLists.txt index 7c9ad405432..7e965188b4c 100644 --- a/src/Processors/CMakeLists.txt +++ b/src/Processors/CMakeLists.txt @@ -1,4 +1,4 @@ -#if (ENABLE_EXAMPLES) +if (ENABLE_EXAMPLES) add_subdirectory(examples) -#endif () +endif () From 8f01c026a668ff22f6446ca5aa230a9a850e9aa1 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Tue, 15 Aug 2023 04:29:34 +0000 Subject: [PATCH 558/777] fix timeout_overflow_mode when having subquery in the rhs of IN Signed-off-by: Duc Canh Le --- src/Interpreters/PreparedSets.cpp | 6 +++++- .../02844_subquery_timeout_with_break.reference | 0 .../0_stateless/02844_subquery_timeout_with_break.sql | 7 +++++++ 3 files changed, 12 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02844_subquery_timeout_with_break.reference create mode 100644 tests/queries/0_stateless/02844_subquery_timeout_with_break.sql diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 67822ecf440..87855a80920 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -198,7 +198,11 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) CompletedPipelineExecutor executor(pipeline); executor.execute(); - set_and_key->set->checkIsCreated(); + /// SET may not be created successfully at this step because the sub-query timeout, but we set + /// timeout_overflow_mode to `break` so no exception is throw and the executor just stops executing + /// the pipeline without setting `set_and_key->set->is_created` to true. + if (!set_and_key->set->isCreated()) + return nullptr; return set_and_key->set; } diff --git a/tests/queries/0_stateless/02844_subquery_timeout_with_break.reference b/tests/queries/0_stateless/02844_subquery_timeout_with_break.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02844_subquery_timeout_with_break.sql b/tests/queries/0_stateless/02844_subquery_timeout_with_break.sql new file mode 100644 index 00000000000..28d07003f32 --- /dev/null +++ b/tests/queries/0_stateless/02844_subquery_timeout_with_break.sql @@ -0,0 +1,7 @@ +CREATE TABLE t (key UInt64, value UInt64, INDEX value_idx value TYPE bloom_filter GRANULARITY 1) ENGINE=MergeTree() ORDER BY key; + +INSERT INTO t SELECT number, rand()%1000 FROM numbers(10000); + +SET timeout_overflow_mode='break'; +SET max_execution_time=0.1; +SELECT * FROM t WHERE value IN (SELECT number FROM numbers(1000000000)); From 357903973357becef3fed58e86ad32fd4de36f9f Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Tue, 15 Aug 2023 14:02:36 +0800 Subject: [PATCH 559/777] Fix an unexpected behavior in #53152 The problematic case happens when an array column of date/datetime is simultaneously specified in an Array Join clause without aliases and in a time converter, such as toYYYYMM. After applying Array Join without aliases, the column's name refers to the flattened array items, however, its data type is recognized as a ColumnArray still, which leads to the unexpected exception throws when building the preimage for the time converters. As a quick fix, we more strictly check the data types of the time converters and quit the preimage optimization pass early. --- ...ptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp | 7 +++++-- .../02843_date_predicate_optimizations_bugs.reference | 2 ++ .../02843_date_predicate_optimizations_bugs.sql | 9 +++++++++ 3 files changed, 16 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.reference create mode 100644 tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.sql diff --git a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp index 6a9251cec49..17fad4ba6e4 100644 --- a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp +++ b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp @@ -160,15 +160,18 @@ void OptimizeDateOrDateTimeConverterWithPreimageMatcher::visit(const ASTFunction auto data_type_and_name = data.tables[*pos].columns.tryGetByName(column_id->shortName()); if (!data_type_and_name) return; + const auto column_type = data_type_and_name->type; + if (!column_type || (!isDateOrDate32(*column_type) && !isDateTime(*column_type) && !isDateTime64(*column_type))) return; + const auto & converter = FunctionFactory::instance().tryGet(ast_func->name, data.context); if (!converter) return; ColumnsWithTypeAndName args; - args.emplace_back(data_type_and_name->type, "tmp"); + args.emplace_back(column_type, "tmp"); auto converter_base = converter->build(args); if (!converter_base || !converter_base->hasInformationAboutPreimage()) return; - auto preimage_range = converter_base->getPreimage(*(data_type_and_name->type), literal->value); + auto preimage_range = converter_base->getPreimage(*column_type, literal->value); if (!preimage_range) return; const auto new_ast = generateOptimizedDateFilterAST(comparator, *data_type_and_name, *preimage_range); diff --git a/tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.reference b/tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.reference new file mode 100644 index 00000000000..d641328e9a5 --- /dev/null +++ b/tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.reference @@ -0,0 +1,2 @@ +202308 1 +202308 2 diff --git a/tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.sql b/tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.sql new file mode 100644 index 00000000000..6e26a5166cb --- /dev/null +++ b/tests/queries/0_stateless/02843_date_predicate_optimizations_bugs.sql @@ -0,0 +1,9 @@ +select + toYYYYMM(date) as date_, + n +from (select + [toDate('20230815'), toDate('20230816')] as date, + [1, 2] as n +) as data +array join date, n +where date_ >= 202303; From bfab73576789fb868ae3f1accb5f3e0ce8ebfcdf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 15 Aug 2023 09:42:36 +0300 Subject: [PATCH 560/777] Update 02844_subquery_timeout_with_break.sql --- .../queries/0_stateless/02844_subquery_timeout_with_break.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02844_subquery_timeout_with_break.sql b/tests/queries/0_stateless/02844_subquery_timeout_with_break.sql index 28d07003f32..511ed0c59de 100644 --- a/tests/queries/0_stateless/02844_subquery_timeout_with_break.sql +++ b/tests/queries/0_stateless/02844_subquery_timeout_with_break.sql @@ -1,3 +1,4 @@ +DROP TABLE IF EXISTS t; CREATE TABLE t (key UInt64, value UInt64, INDEX value_idx value TYPE bloom_filter GRANULARITY 1) ENGINE=MergeTree() ORDER BY key; INSERT INTO t SELECT number, rand()%1000 FROM numbers(10000); @@ -5,3 +6,5 @@ INSERT INTO t SELECT number, rand()%1000 FROM numbers(10000); SET timeout_overflow_mode='break'; SET max_execution_time=0.1; SELECT * FROM t WHERE value IN (SELECT number FROM numbers(1000000000)); + +DROP TABLE t; From edee00c5b4d4634e75c19be464cb533aa253412b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 15 Aug 2023 09:44:13 +0300 Subject: [PATCH 561/777] Update PreparedSets.cpp --- src/Interpreters/PreparedSets.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 87855a80920..f55a8a9d5cb 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -198,8 +198,8 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) CompletedPipelineExecutor executor(pipeline); executor.execute(); - /// SET may not be created successfully at this step because the sub-query timeout, but we set - /// timeout_overflow_mode to `break` so no exception is throw and the executor just stops executing + /// SET may not be created successfully at this step because of the sub-query timeout, but if we have + /// timeout_overflow_mode set to `break`, no exception is thrown, and the executor just stops executing /// the pipeline without setting `set_and_key->set->is_created` to true. if (!set_and_key->set->isCreated()) return nullptr; From b9fa29c4c8004f1b1a483cfa2ce21dd350163c05 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 15 Aug 2023 09:46:40 +0300 Subject: [PATCH 562/777] Update OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp --- ...OrDateTimeConverterWithPreimageVisitor.cpp | 40 +++++++++++-------- 1 file changed, 24 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp index 17fad4ba6e4..dd205ae6508 100644 --- a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp +++ b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp @@ -118,35 +118,37 @@ void OptimizeDateOrDateTimeConverterWithPreimageMatcher::visit(const ASTFunction {"greaterOrEquals", "lessOrEquals"}, }; - if (!swap_relations.contains(function.name)) return; + if (!swap_relations.contains(function.name)) + return; - if (!function.arguments || function.arguments->children.size() != 2) return; + if (!function.arguments || function.arguments->children.size() != 2) + return; size_t func_id = function.arguments->children.size(); for (size_t i = 0; i < function.arguments->children.size(); i++) - { if (const auto * func = function.arguments->children[i]->as()) - { func_id = i; - } - } - if (func_id == function.arguments->children.size()) return; + if (func_id == function.arguments->children.size()) + return; size_t literal_id = 1 - func_id; const auto * literal = function.arguments->children[literal_id]->as(); - if (!literal || literal->value.getType() != Field::Types::UInt64) return; + if (!literal || literal->value.getType() != Field::Types::UInt64) + return; String comparator = literal_id > func_id ? function.name : swap_relations.at(function.name); const auto * ast_func = function.arguments->children[func_id]->as(); /// Currently we only handle single-argument functions. - if (!ast_func || !ast_func->arguments || ast_func->arguments->children.size() != 1) return; + if (!ast_func || !ast_func->arguments || ast_func->arguments->children.size() != 1) + return; const auto * column_id = ast_func->arguments->children.at(0)->as(); - if (!column_id) return; + if (!column_id) + return; auto pos = IdentifierSemantic::getMembership(*column_id); if (!pos) @@ -158,24 +160,30 @@ void OptimizeDateOrDateTimeConverterWithPreimageMatcher::visit(const ASTFunction return; auto data_type_and_name = data.tables[*pos].columns.tryGetByName(column_id->shortName()); - if (!data_type_and_name) return; + if (!data_type_and_name) + return; const auto column_type = data_type_and_name->type; - if (!column_type || (!isDateOrDate32(*column_type) && !isDateTime(*column_type) && !isDateTime64(*column_type))) return; + if (!column_type || (!isDateOrDate32(*column_type) && !isDateTime(*column_type) && !isDateTime64(*column_type))) + return; const auto & converter = FunctionFactory::instance().tryGet(ast_func->name, data.context); - if (!converter) return; + if (!converter) + return; ColumnsWithTypeAndName args; args.emplace_back(column_type, "tmp"); auto converter_base = converter->build(args); - if (!converter_base || !converter_base->hasInformationAboutPreimage()) return; + if (!converter_base || !converter_base->hasInformationAboutPreimage()) + return; auto preimage_range = converter_base->getPreimage(*column_type, literal->value); - if (!preimage_range) return; + if (!preimage_range) + return; const auto new_ast = generateOptimizedDateFilterAST(comparator, *data_type_and_name, *preimage_range); - if (!new_ast) return; + if (!new_ast) + return; ast = new_ast; } From c2017e0ea352cc2343666e25370a447afab4a2f9 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 15 Aug 2023 14:49:45 +0800 Subject: [PATCH 563/777] update orc version --- contrib/orc | 2 +- src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/orc b/contrib/orc index 568d1d60c25..a20d1d9d7ad 160000 --- a/contrib/orc +++ b/contrib/orc @@ -1 +1 @@ -Subproject commit 568d1d60c250af1890f226c182bc15bd8cc94cf1 +Subproject commit a20d1d9d7ad4a4be7b7ba97588e16ca8b9abb2b6 diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index c259cd8907d..45183a91f43 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -502,7 +502,7 @@ static ColumnWithTypeAndName readColumnWithDecimalDataCast( auto & column_data = static_cast(*internal_column).getData(); column_data.reserve(orc_column->numElements); - auto * orc_decimal_column = const_cast(dynamic_cast(orc_column)); + const auto * orc_decimal_column = dynamic_cast(orc_column); for (size_t i = 0; i < orc_decimal_column->numElements; ++i) { DecimalType decimal_value; From df02512ebfa8efc455519c5e5edd7492e5ad0c16 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 15 Aug 2023 08:53:08 +0200 Subject: [PATCH 564/777] Do not send logs to CI if the credentials are not set --- tests/ci/ast_fuzzer_check.py | 10 ++++++---- tests/ci/functional_test_check.py | 9 +++++---- tests/ci/stress_check.py | 9 +++++---- 3 files changed, 16 insertions(+), 12 deletions(-) diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 56b356f5449..82b2732c2b2 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -145,10 +145,12 @@ def main(): ci_logs_password = os.getenv( "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - subprocess.check_call( - f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}' '{main_log_path}'", - shell=True, - ) + + if ci_logs_host != 'CLICKHOUSE_CI_LOGS_HOST': + subprocess.check_call( + f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}' '{main_log_path}'", + shell=True, + ) check_name_lower = ( check_name.lower().replace("(", "").replace(")", "").replace(" ", "") diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index d06da94d0f0..2d9ab77c9cf 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -394,10 +394,11 @@ def main(): ci_logs_password = os.getenv( "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - subprocess.check_call( - f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", - shell=True, - ) + if ci_logs_host != 'CLICKHOUSE_CI_LOGS_HOST': + subprocess.check_call( + f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", + shell=True, + ) report_url = upload_results( s3_helper, diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 42d372efb5d..b9af5fd5e83 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -209,10 +209,11 @@ def run_stress_test(docker_image_name): ci_logs_password = os.getenv( "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - subprocess.check_call( - f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", - shell=True, - ) + if ci_logs_host != 'CLICKHOUSE_CI_LOGS_HOST': + subprocess.check_call( + f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", + shell=True, + ) report_url = upload_results( s3_helper, From a92fe25ff9968a2edd51f918802c4485957f989a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 15 Aug 2023 07:15:58 +0000 Subject: [PATCH 565/777] Automatic style fix --- tests/ci/ast_fuzzer_check.py | 2 +- tests/ci/functional_test_check.py | 2 +- tests/ci/stress_check.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 82b2732c2b2..1a75d02bef4 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -146,7 +146,7 @@ def main(): "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - if ci_logs_host != 'CLICKHOUSE_CI_LOGS_HOST': + if ci_logs_host != "CLICKHOUSE_CI_LOGS_HOST": subprocess.check_call( f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}' '{main_log_path}'", shell=True, diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 2d9ab77c9cf..22210390b09 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -394,7 +394,7 @@ def main(): ci_logs_password = os.getenv( "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - if ci_logs_host != 'CLICKHOUSE_CI_LOGS_HOST': + if ci_logs_host != "CLICKHOUSE_CI_LOGS_HOST": subprocess.check_call( f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", shell=True, diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index b9af5fd5e83..9c18bcbfe40 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -209,7 +209,7 @@ def run_stress_test(docker_image_name): ci_logs_password = os.getenv( "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - if ci_logs_host != 'CLICKHOUSE_CI_LOGS_HOST': + if ci_logs_host != "CLICKHOUSE_CI_LOGS_HOST": subprocess.check_call( f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", shell=True, From 8992cc5af4653365351a98f2d00c7e4416c4965f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Jul 2023 10:48:00 +0000 Subject: [PATCH 566/777] Factorize constants --- .../MergeTree/MergeTreeIndexAnnoy.cpp | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index f77cfe4fed0..1c92645dbfa 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -25,6 +25,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +static constexpr auto DISTANCE_FUNCTION_L2 = "L2Distance"; +static constexpr auto DISTANCE_FUNCTION_COSINE = "cosineDistance"; + +static constexpr auto DEFAULT_TREES = 100uz; +static constexpr auto DEFAULT_DISTANCE_FUNCTION = DISTANCE_FUNCTION_L2; template AnnoyIndexWithSerialization::AnnoyIndexWithSerialization(size_t dimensions) @@ -224,9 +229,9 @@ bool MergeTreeIndexConditionAnnoy::alwaysUnknownOrTrue() const std::vector MergeTreeIndexConditionAnnoy::getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const { - if (distance_function == "L2Distance") + if (distance_function == DISTANCE_FUNCTION_L2) return getUsefulRangesImpl(idx_granule); - else if (distance_function == "cosineDistance") + else if (distance_function == DISTANCE_FUNCTION_COSINE) return getUsefulRangesImpl(idx_granule); std::unreachable(); } @@ -289,9 +294,9 @@ MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, UInt64 MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const { - if (distance_function == "L2Distance") + if (distance_function == DISTANCE_FUNCTION_L2) return std::make_shared>(index.name, index.sample_block); - else if (distance_function == "cosineDistance") + else if (distance_function == DISTANCE_FUNCTION_COSINE) return std::make_shared>(index.name, index.sample_block); std::unreachable(); } @@ -299,9 +304,9 @@ MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const { /// TODO: Support more metrics. Available metrics: https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 - if (distance_function == "L2Distance") + if (distance_function == DISTANCE_FUNCTION_L2) return std::make_shared>(index.name, index.sample_block, trees); - else if (distance_function == "cosineDistance") + else if (distance_function == DISTANCE_FUNCTION_COSINE) return std::make_shared>(index.name, index.sample_block, trees); std::unreachable(); } @@ -313,14 +318,11 @@ MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const Selec MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) { - static constexpr auto default_trees = 100uz; - static constexpr auto default_distance_function = "L2Distance"; - - String distance_function = default_distance_function; + String distance_function = DEFAULT_DISTANCE_FUNCTION; if (!index.arguments.empty()) distance_function = index.arguments[0].get(); - UInt64 trees = default_trees; + UInt64 trees = DEFAULT_TREES; if (index.arguments.size() > 1) trees = index.arguments[1].get(); @@ -350,8 +352,8 @@ void annoyIndexValidator(const IndexDescription & index, bool /* attach */) if (!index.arguments.empty()) { String distance_name = index.arguments[0].get(); - if (distance_name != "L2Distance" && distance_name != "cosineDistance") - throw Exception(ErrorCodes::INCORRECT_DATA, "Annoy index only supports distance functions 'L2Distance' and 'cosineDistance'"); + if (distance_name != DISTANCE_FUNCTION_L2 && distance_name != DISTANCE_FUNCTION_COSINE) + throw Exception(ErrorCodes::INCORRECT_DATA, "Annoy index only supports distance functions '{}' and '{}'", DISTANCE_FUNCTION_L2, DISTANCE_FUNCTION_COSINE); } /// Check data type of indexed column: From 33948a150fefe36ebf82bb8196b52215e577270b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 15 Aug 2023 11:50:11 +0200 Subject: [PATCH 567/777] Restart killed PublishedReleaseCI workflows --- tests/ci/workflow_approve_rerun_lambda/app.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 5e2331ece3c..e511d773577 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -64,6 +64,7 @@ NEED_RERUN_WORKFLOWS = { "DocsCheck", "MasterCI", "NightlyBuilds", + "PublishedReleaseCI", "PullRequestCI", "ReleaseBranchCI", } From 48c62fd75e2977ee9b23ceea070782188536c2ba Mon Sep 17 00:00:00 2001 From: Davit Vardanyan <78792753+davvard@users.noreply.github.com> Date: Tue, 15 Aug 2023 16:00:27 +0400 Subject: [PATCH 568/777] Add: USearch --- .gitmodules | 3 + contrib/CMakeLists.txt | 1 + contrib/usearch | 1 + contrib/usearch-cmake/CMakeLists.txt | 15 + .../mergetree-family/annindexes.md | 41 ++ src/CMakeLists.txt | 4 + src/Core/Settings.h | 1 + src/Storages/MergeTree/MergeTreeIndexHnsw.cpp | 351 ++++++++++++++++++ src/Storages/MergeTree/MergeTreeIndexHnsw.h | 103 +++++ src/Storages/MergeTree/MergeTreeIndices.cpp | 5 + src/Storages/MergeTree/MergeTreeIndices.h | 5 + .../0_stateless/02354_hnsw_index.reference | 143 +++++++ .../queries/0_stateless/02354_hnsw_index.sql | 229 ++++++++++++ 13 files changed, 902 insertions(+) create mode 160000 contrib/usearch create mode 100644 contrib/usearch-cmake/CMakeLists.txt create mode 100644 src/Storages/MergeTree/MergeTreeIndexHnsw.cpp create mode 100644 src/Storages/MergeTree/MergeTreeIndexHnsw.h create mode 100644 tests/queries/0_stateless/02354_hnsw_index.reference create mode 100644 tests/queries/0_stateless/02354_hnsw_index.sql diff --git a/.gitmodules b/.gitmodules index 86fd7832dd9..36721723371 100644 --- a/.gitmodules +++ b/.gitmodules @@ -347,3 +347,6 @@ [submodule "contrib/incbin"] path = contrib/incbin url = https://github.com/graphitemaster/incbin.git +[submodule "contrib/usearch"] + path = contrib/usearch + url = https://github.com/unum-cloud/usearch.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 16135351cce..abecfcb30c8 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -196,6 +196,7 @@ if (ARCH_S390X) add_contrib(crc32-s390x-cmake crc32-s390x) endif() add_contrib (annoy-cmake annoy) +add_contrib (usearch-cmake usearch) add_contrib (xxHash-cmake xxHash) add_contrib (libbcrypt-cmake libbcrypt) diff --git a/contrib/usearch b/contrib/usearch new file mode 160000 index 00000000000..387b78b28b1 --- /dev/null +++ b/contrib/usearch @@ -0,0 +1 @@ +Subproject commit 387b78b28b17b8954024ffc81e97cbcfa10d1f30 diff --git a/contrib/usearch-cmake/CMakeLists.txt b/contrib/usearch-cmake/CMakeLists.txt new file mode 100644 index 00000000000..f4ed6a9adca --- /dev/null +++ b/contrib/usearch-cmake/CMakeLists.txt @@ -0,0 +1,15 @@ +option(ENABLE_USEARCH "Enable USearch (Approximate Neighborhood Search, HNSW) support" ${ENABLE_LIBRARIES}) + +if (NOT ENABLE_USEARCH) + message (STATUS "Not using usearch") + return() +endif() + +set(USEARCH_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/usearch") +set(USEARCH_SOURCE_DIR "${USEARCH_PROJECT_DIR}/include") + +add_library(_usearch INTERFACE) +target_include_directories(_usearch SYSTEM INTERFACE ${USEARCH_PROJECT_DIR}/fp16/include ${USEARCH_PROJECT_DIR}/robin-map/include ${USEARCH_PROJECT_DIR}/simsimd/include ${USEARCH_SOURCE_DIR}) + +add_library(ch_contrib::usearch ALIAS _usearch) +target_compile_definitions(_usearch INTERFACE ENABLE_USEARCH) \ No newline at end of file diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 81c69215472..4fd1d0d17db 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -142,6 +142,8 @@ was specified for ANN indexes, the default value is 100 million. - [Annoy](/docs/en/engines/table-engines/mergetree-family/annindexes.md#annoy-annoy) +- [USearch](/docs/en/engines/table-engines/mergetree-family/annindexes.md#usearch-usearch) + ## Annoy {#annoy} Annoy indexes are currently experimental, to use them you first need to `SET allow_experimental_annoy_index = 1`. They are also currently @@ -216,3 +218,42 @@ ORDER BY L2Distance(vectors, Point) LIMIT N SETTINGS annoy_index_search_k_nodes=100; ``` + + +## USearch {#usearch} + +USearch indexes are currently experimental, to use them you first need to `SET allow_experimental_usearch_index = 1`. + +This type of ANN index implements [the HNSW algorithm](https://github.com/unum-cloud/usearch). + +Syntax to create an USearch index over an [Array](../../../sql-reference/data-types/array.md) column: + +```sql +CREATE TABLE table_with_usearch_index +( + id Int64, + vectors Array(Float32), + INDEX [ann_index_name] vectors TYPE usearch([Distance]) [GRANULARITY N] +) +ENGINE = MergeTree +ORDER BY id; +``` + +Syntax to create an ANN index over a [Tuple](../../../sql-reference/data-types/tuple.md) column: + +```sql +CREATE TABLE table_with_usearch_index +( + id Int64, + vectors Tuple(Float32[, Float32[, ...]]), + INDEX [ann_index_name] vectors TYPE usearch([Distance]) [GRANULARITY N] +) +ENGINE = MergeTree +ORDER BY id; +``` + +USearch currently supports two distance functions: +- `L2Distance`, also called Euclidean distance, is the length of a line segment between two points in Euclidean space + ([Wikipedia](https://en.wikipedia.org/wiki/Euclidean_distance)). +- `cosineDistance`, also called cosine similarity, is the cosine of the angle between two (non-zero) vectors + ([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)). diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index ddb6fcebd23..e4e9fd19cff 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -599,6 +599,10 @@ if (TARGET ch_contrib::annoy) dbms_target_link_libraries(PUBLIC ch_contrib::annoy) endif() +if (TARGET ch_contrib::usearch) + dbms_target_link_libraries(PUBLIC ch_contrib::usearch) +endif() + if (TARGET ch_rust::skim) dbms_target_include_directories(PRIVATE $) dbms_target_link_libraries(PUBLIC ch_rust::skim) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 714ac17a15d..14714981c00 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -772,6 +772,7 @@ class IColumn; M(Bool, allow_experimental_hash_functions, false, "Enable experimental hash functions", 0) \ M(Bool, allow_experimental_object_type, false, "Allow Object and JSON data types", 0) \ M(Bool, allow_experimental_annoy_index, false, "Allows to use Annoy index. Disabled by default because this feature is experimental", 0) \ + M(Bool, allow_experimental_usearch_index, false, "Allows to use USearch index. Disabled by default because this feature is experimental", 0) \ M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \ M(Int64, annoy_index_search_k_nodes, -1, "SELECT queries search up to this many nodes in Annoy indexes.", 0) \ M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \ diff --git a/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp b/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp new file mode 100644 index 00000000000..67726f41e8c --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp @@ -0,0 +1,351 @@ +#ifdef ENABLE_USEARCH + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int INCORRECT_DATA; + extern const int INCORRECT_NUMBER_OF_COLUMNS; + extern const int INCORRECT_QUERY; + extern const int LOGICAL_ERROR; +} + + +template +USearchIndexWithSerialization::USearchIndexWithSerialization(size_t dimensions) + : Base(Base::make(unum::usearch::metric_punned_t(dimensions, Metric))) +{ +} + +template +void USearchIndexWithSerialization::serialize([[maybe_unused]] WriteBuffer & ostr) const +{ + auto callback = [&ostr](void * from, size_t n) + { + ostr.write(reinterpret_cast(from), n); + return true; + }; + + Base::stream(callback); +} + +template +void USearchIndexWithSerialization::deserialize([[maybe_unused]] ReadBuffer & istr) +{ + BufferBase::Position & pos = istr.position(); + unum::usearch::memory_mapped_file_t memory_map(pos, istr.buffer().size() - istr.count()); + Base::view(std::move(memory_map)); + pos += Base::stream_length(); + + auto copy = Base::copy(); + if (!copy) + throw std::runtime_error("Can't copy index"); + Base::swap(copy.index); +} + +template +size_t USearchIndexWithSerialization::getDimensions() const +{ + return Base::dimensions(); +} + + +template +MergeTreeIndexGranuleUSearch::MergeTreeIndexGranuleUSearch(const String & index_name_, const Block & index_sample_block_) + : index_name(index_name_), index_sample_block(index_sample_block_), index(nullptr) +{ +} + +template +MergeTreeIndexGranuleUSearch::MergeTreeIndexGranuleUSearch( + const String & index_name_, const Block & index_sample_block_, USearchIndexWithSerializationPtr index_) + : index_name(index_name_), index_sample_block(index_sample_block_), index(std::move(index_)) +{ +} + +template +void MergeTreeIndexGranuleUSearch::serializeBinary(WriteBuffer & ostr) const +{ + /// Number of dimensions is required in the index constructor, + /// so it must be written and read separately from the other part + writeIntBinary(static_cast(index->getDimensions()), ostr); // write dimension + index->serialize(ostr); +} + +template +void MergeTreeIndexGranuleUSearch::deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion /*version*/) +{ + UInt64 dimension; + readIntBinary(dimension, istr); + index = std::make_shared>(dimension); + index->deserialize(istr); +} + +template +MergeTreeIndexAggregatorUSearch::MergeTreeIndexAggregatorUSearch(const String & index_name_, const Block & index_sample_block_) + : index_name(index_name_), index_sample_block(index_sample_block_) +{ +} + +template +MergeTreeIndexGranulePtr MergeTreeIndexAggregatorUSearch::getGranuleAndReset() +{ + auto granule = std::make_shared>(index_name, index_sample_block, index); + index = nullptr; + return granule; +} + +template +void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t * pos, size_t limit) +{ + if (*pos >= block.rows()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "The provided position is not less than the number of block rows. Position: {}, Block rows: {}.", + *pos, + block.rows()); + + size_t rows_read = std::min(limit, block.rows() - *pos); + if (rows_read == 0) + return; + + if (index_sample_block.columns() > 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected block with single column"); + + const String & index_column_name = index_sample_block.getByPosition(0).name; + ColumnPtr column_cut = block.getByName(index_column_name).column->cut(*pos, rows_read); + + if (const auto & column_array = typeid_cast(column_cut.get())) + { + const auto & data = column_array->getData(); + const auto & array = typeid_cast(data).getData(); + + if (array.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Array has 0 rows, {} rows expected", rows_read); + + const auto & offsets = column_array->getOffsets(); + const size_t num_rows = offsets.size(); + + + /// Check all sizes are the same + size_t size = offsets[0]; + for (size_t i = 0; i < num_rows - 1; ++i) + if (offsets[i + 1] - offsets[i] != size) + throw Exception(ErrorCodes::INCORRECT_DATA, "All arrays in column {} must have equal length", index_column_name); + + + index = std::make_shared>(size); + + /// Add all rows of block + index->reserve(unum::usearch::ceil2(index->size() + num_rows + 1)); + index->add(index->size(), array.data()); + for (size_t current_row = 1; current_row < num_rows; ++current_row) + index->add(index->size(), &array[offsets[current_row - 1]]); + } + else if (const auto & column_tuple = typeid_cast(column_cut.get())) + { + const auto & columns = column_tuple->getColumns(); + std::vector> data{column_tuple->size(), std::vector()}; + for (const auto & column : columns) + { + const auto & pod_array = typeid_cast(column.get())->getData(); + for (size_t i = 0; i < pod_array.size(); ++i) + data[i].push_back(pod_array[i]); + } + + if (data.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Tuple has 0 rows, {} rows expected", rows_read); + + index = std::make_shared>(data[0].size()); + index->reserve(unum::usearch::ceil2(index->size() + data.size())); + for (const auto & item : data) + index->add(index->size(), item.data()); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array or Tuple column"); + + *pos += rows_read; +} + + +MergeTreeIndexConditionUSearch::MergeTreeIndexConditionUSearch( + const IndexDescription & /*index_description*/, const SelectQueryInfo & query, const String & distance_function_, ContextPtr context) + : ann_condition(query, context), distance_function(distance_function_) +{ +} + +bool MergeTreeIndexConditionUSearch::mayBeTrueOnGranule(MergeTreeIndexGranulePtr /*idx_granule*/) const +{ + throw Exception(ErrorCodes::LOGICAL_ERROR, "mayBeTrueOnGranule is not supported for ANN skip indexes"); +} + +bool MergeTreeIndexConditionUSearch::alwaysUnknownOrTrue() const +{ + return ann_condition.alwaysUnknownOrTrue(distance_function); +} + +std::vector MergeTreeIndexConditionUSearch::getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const +{ + if (distance_function == "L2Distance") + return getUsefulRangesImpl(idx_granule); + else if (distance_function == "cosineDistance") + return getUsefulRangesImpl(idx_granule); + std::unreachable(); +} + +template +std::vector MergeTreeIndexConditionUSearch::getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const +{ + const UInt64 limit = ann_condition.getLimit(); + const UInt64 index_granularity = ann_condition.getIndexGranularity(); + const std::optional comparison_distance = ann_condition.getQueryType() == ApproximateNearestNeighborInformation::Type::Where + ? std::optional(ann_condition.getComparisonDistanceForWhereQuery()) + : std::nullopt; + + if (comparison_distance && comparison_distance.value() < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to optimize query with where without distance"); + + const std::vector reference_vector = ann_condition.getReferenceVector(); + const auto granule = std::dynamic_pointer_cast>(idx_granule); + if (granule == nullptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Granule has the wrong type"); + + const USearchIndexWithSerializationPtr index = granule->index; + if (ann_condition.getDimensions() != index->dimensions()) + throw Exception( + ErrorCodes::INCORRECT_QUERY, + "The dimension of the space in the request ({}) " + "does not match the dimension in the index ({})", + ann_condition.getDimensions(), + index->dimensions()); + + auto result = index->search(reference_vector.data(), limit); + std::vector neighbors(result.size()); /// indexes of dots which were closest to the reference vector + std::vector distances(result.size()); + result.dump_to(neighbors.data(), distances.data()); + + std::vector granule_numbers; + granule_numbers.reserve(neighbors.size()); + for (size_t i = 0; i < neighbors.size(); ++i) + { + if (comparison_distance && distances[i] > comparison_distance) + continue; + granule_numbers.push_back(neighbors[i] / index_granularity); + } + + /// make unique + std::sort(granule_numbers.begin(), granule_numbers.end()); + granule_numbers.erase(std::unique(granule_numbers.begin(), granule_numbers.end()), granule_numbers.end()); + + return granule_numbers; +} + +MergeTreeIndexUSearch::MergeTreeIndexUSearch(const IndexDescription & index_, const String & distance_function_) + : IMergeTreeIndex(index_), distance_function(distance_function_) +{ +} + + +MergeTreeIndexGranulePtr MergeTreeIndexUSearch::createIndexGranule() const +{ + if (distance_function == "L2Distance") + return std::make_shared>(index.name, index.sample_block); + else if (distance_function == "cosineDistance") + return std::make_shared>(index.name, index.sample_block); + std::unreachable(); +} + +MergeTreeIndexAggregatorPtr MergeTreeIndexUSearch::createIndexAggregator() const +{ + if (distance_function == "L2Distance") + return std::make_shared>(index.name, index.sample_block); + else if (distance_function == "cosineDistance") + return std::make_shared>(index.name, index.sample_block); + std::unreachable(); +} + +MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const +{ + return std::make_shared(index, query, distance_function, context); +}; + +MergeTreeIndexPtr usearchIndexCreator(const IndexDescription & index) +{ + static constexpr auto default_distance_function = "L2Distance"; + String distance_function = default_distance_function; + if (!index.arguments.empty()) + distance_function = index.arguments[0].get(); + + return std::make_shared(index, distance_function); +} + +void usearchIndexValidator(const IndexDescription & index, bool /* attach */) +{ + /// Check number and type of USearch index arguments: + + if (index.arguments.size() > 1) + throw Exception(ErrorCodes::INCORRECT_QUERY, "USearch index must not have more than one parameters"); + + if (!index.arguments.empty() && index.arguments[0].getType() != Field::Types::String) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Distance function argument of USearch index must be of type String"); + + /// Check that the index is created on a single column + + if (index.column_names.size() != 1 || index.data_types.size() != 1) + throw Exception(ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS, "USearch indexes must be created on a single column"); + + /// Check that a supported metric was passed as first argument + + if (!index.arguments.empty()) + { + String distance_name = index.arguments[0].get(); + if (distance_name != "L2Distance" && distance_name != "cosineDistance") + throw Exception(ErrorCodes::INCORRECT_DATA, "USearch index only supports distance functions 'L2Distance' and 'cosineDistance'"); + } + + /// Check data type of indexed column: + + auto throw_unsupported_underlying_column_exception = []() + { + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, "USearch indexes can only be created on columns of type Array(Float32) and Tuple(Float32)"); + }; + + DataTypePtr data_type = index.sample_block.getDataTypes()[0]; + + if (const auto * data_type_array = typeid_cast(data_type.get())) + { + TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); + if (!WhichDataType(nested_type_index).isFloat32()) + throw_unsupported_underlying_column_exception(); + } + else if (const auto * data_type_tuple = typeid_cast(data_type.get())) + { + const DataTypes & inner_types = data_type_tuple->getElements(); + for (const auto & inner_type : inner_types) + { + TypeIndex nested_type_index = inner_type->getTypeId(); + if (!WhichDataType(nested_type_index).isFloat32()) + throw_unsupported_underlying_column_exception(); + } + } + else + throw_unsupported_underlying_column_exception(); +} +} + +#endif diff --git a/src/Storages/MergeTree/MergeTreeIndexHnsw.h b/src/Storages/MergeTree/MergeTreeIndexHnsw.h new file mode 100644 index 00000000000..6be4a649ebe --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeIndexHnsw.h @@ -0,0 +1,103 @@ +#pragma once + +#ifdef ENABLE_USEARCH + +#include + +#include + +namespace DB +{ + +template +class USearchIndexWithSerialization : public unum::usearch::index_dense_t +{ + using Base = unum::usearch::index_dense_t; + +public: + explicit USearchIndexWithSerialization(size_t dimensions); + void serialize(WriteBuffer & ostr) const; + void deserialize(ReadBuffer & istr); + size_t getDimensions() const; +}; + +template +using USearchIndexWithSerializationPtr = std::shared_ptr>; + +template +struct MergeTreeIndexGranuleUSearch final : public IMergeTreeIndexGranule +{ + MergeTreeIndexGranuleUSearch(const String & index_name_, const Block & index_sample_block_); + MergeTreeIndexGranuleUSearch( + const String & index_name_, const Block & index_sample_block_, USearchIndexWithSerializationPtr index_); + + ~MergeTreeIndexGranuleUSearch() override = default; + + void serializeBinary(WriteBuffer & ostr) const override; + void deserializeBinary(ReadBuffer & istr, MergeTreeIndexVersion version) override; + + bool empty() const override { return !index.get(); } + + const String index_name; + const Block index_sample_block; + USearchIndexWithSerializationPtr index; +}; + +template +struct MergeTreeIndexAggregatorUSearch final : IMergeTreeIndexAggregator +{ + MergeTreeIndexAggregatorUSearch(const String & index_name_, const Block & index_sample_block); + ~MergeTreeIndexAggregatorUSearch() override = default; + + bool empty() const override { return !index || index->size() == 0; } + MergeTreeIndexGranulePtr getGranuleAndReset() override; + void update(const Block & block, size_t * pos, size_t limit) override; + + const String index_name; + const Block index_sample_block; + USearchIndexWithSerializationPtr index; +}; + + +class MergeTreeIndexConditionUSearch final : public IMergeTreeIndexConditionApproximateNearestNeighbor +{ +public: + MergeTreeIndexConditionUSearch( + const IndexDescription & index_description, const SelectQueryInfo & query, const String & distance_function, ContextPtr context); + + ~MergeTreeIndexConditionUSearch() override = default; + + bool alwaysUnknownOrTrue() const override; + bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const override; + std::vector getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const override; + +private: + template + std::vector getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const; + + const ApproximateNearestNeighborCondition ann_condition; + const String distance_function; +}; + + +class MergeTreeIndexUSearch : public IMergeTreeIndex +{ +public: + MergeTreeIndexUSearch(const IndexDescription & index_, const String & distance_function_); + + ~MergeTreeIndexUSearch() override = default; + + MergeTreeIndexGranulePtr createIndexGranule() const override; + MergeTreeIndexAggregatorPtr createIndexAggregator() const override; + MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override; + + bool mayBenefitFromIndexForIn(const ASTPtr & /*node*/) const override { return false; } + +private: + const String distance_function; +}; + + +} + +#endif diff --git a/src/Storages/MergeTree/MergeTreeIndices.cpp b/src/Storages/MergeTree/MergeTreeIndices.cpp index 6ae96d00171..322cdd35afe 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.cpp +++ b/src/Storages/MergeTree/MergeTreeIndices.cpp @@ -132,6 +132,11 @@ MergeTreeIndexFactory::MergeTreeIndexFactory() registerValidator("annoy", annoyIndexValidator); #endif +#ifdef ENABLE_USEARCH + registerCreator("usearch", usearchIndexCreator); + registerValidator("usearch", usearchIndexValidator); +#endif + registerCreator("inverted", invertedIndexCreator); registerValidator("inverted", invertedIndexValidator); diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index 1ad6b082223..40128bab9d0 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -238,6 +238,11 @@ MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index); void annoyIndexValidator(const IndexDescription & index, bool attach); #endif +#ifdef ENABLE_USEARCH +MergeTreeIndexPtr usearchIndexCreator(const IndexDescription& index); +void usearchIndexValidator(const IndexDescription& index, bool attach); +#endif + MergeTreeIndexPtr invertedIndexCreator(const IndexDescription& index); void invertedIndexValidator(const IndexDescription& index, bool attach); diff --git a/tests/queries/0_stateless/02354_hnsw_index.reference b/tests/queries/0_stateless/02354_hnsw_index.reference new file mode 100644 index 00000000000..102edf6d026 --- /dev/null +++ b/tests/queries/0_stateless/02354_hnsw_index.reference @@ -0,0 +1,143 @@ +--- Negative tests --- +--- Test default GRANULARITY (should be 100 mio. for usearch)--- +CREATE TABLE default.tab\n(\n `id` Int32,\n `vector` Array(Float32),\n INDEX usearch_index vector TYPE usearch GRANULARITY 1\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `id` Int32,\n `vector` Array(Float32),\n INDEX usearch_index vector TYPE usearch GRANULARITY 1\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 +--- Test with Array, GRANULARITY = 1, index_granularity = 5 --- +WHERE type, L2Distance, check that index is used +Expression ((Projection + Before ORDER BY)) + Limit (preliminary LIMIT (without OFFSET)) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 + Skip + Name: usearch_index + Description: usearch GRANULARITY 1 + Parts: 1/1 + Granules: 1/3 +ORDER BY type, L2Distance, check that index is used +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 + Skip + Name: usearch_index + Description: usearch GRANULARITY 1 + Parts: 1/1 + Granules: 3/3 +Reference ARRAYs with non-matching dimension are rejected +Special case: MaximumDistance is negative +WHERE type, L2Distance +Special case: setting max_limit_for_ann_queries +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 +--- Test with Tuple, GRANULARITY = 1, index_granularity = 5 --- +WHERE type, L2Distance, check that index is used +Expression ((Projection + Before ORDER BY)) + Limit (preliminary LIMIT (without OFFSET)) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 + Skip + Name: usearch_index + Description: usearch GRANULARITY 1 + Parts: 1/1 + Granules: 1/3 +ORDER BY type, L2Distance, check that index is used +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 3/3 + Skip + Name: usearch_index + Description: usearch GRANULARITY 1 + Parts: 1/1 + Granules: 3/3 +--- Test non-default metric (cosine distance) --- +--- Test with Array, GRANULARITY = 2, index_granularity = 4 --- +WHERE type, L2Distance, check that index is used +Expression ((Projection + Before ORDER BY)) + Limit (preliminary LIMIT (without OFFSET)) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: usearch_index + Description: usearch GRANULARITY 2 + Parts: 0/1 + Granules: 2/4 +ORDER BY type, L2Distance, check that index is used +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: usearch_index + Description: usearch GRANULARITY 2 + Parts: 1/1 + Granules: 4/4 +--- Test with Array, GRANULARITY = 4, index_granularity = 4 --- +WHERE type, L2Distance, check that index is used +Expression ((Projection + Before ORDER BY)) + Limit (preliminary LIMIT (without OFFSET)) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: usearch_index + Description: usearch GRANULARITY 4 + Parts: 0/1 + Granules: 3/4 +ORDER BY type, L2Distance, check that index is used +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + ReadFromMergeTree (default.tab) + Indexes: + PrimaryKey + Condition: true + Parts: 1/1 + Granules: 4/4 + Skip + Name: usearch_index + Description: usearch GRANULARITY 4 + Parts: 1/1 + Granules: 4/4 diff --git a/tests/queries/0_stateless/02354_hnsw_index.sql b/tests/queries/0_stateless/02354_hnsw_index.sql new file mode 100644 index 00000000000..059705254d8 --- /dev/null +++ b/tests/queries/0_stateless/02354_hnsw_index.sql @@ -0,0 +1,229 @@ +-- Tags: no-fasttest, no-ubsan, no-cpu-aarch64, no-upgrade-check + +SET allow_experimental_usearch_index = 1; +SET allow_experimental_analyzer = 0; + +SELECT '--- Negative tests ---'; + +DROP TABLE IF EXISTS tab; + +-- must have at most 2 arguments +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch('too', 'many', 'arguments')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } + +-- first argument (distance_function) must be String +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } + +-- must be created on single column +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index (vector, id) TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_NUMBER_OF_COLUMNS } + +-- reject unsupported distance functions +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch('wormholeDistance')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_DATA } + +-- must be created on Array/Tuple(Float32) columns +SET allow_suspicious_low_cardinality_types = 1; +CREATE TABLE tab(id Int32, vector Float32, INDEX usearch_index vector TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vector Array(Float64), INDEX usearch_index vector TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vector Tuple(Float64), INDEX usearch_index vector TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vector LowCardinality(Float32), INDEX usearch_index vector TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } +CREATE TABLE tab(id Int32, vector Nullable(Float32), INDEX usearch_index vector TYPE usearch()) ENGINE = MergeTree ORDER BY id; -- { serverError ILLEGAL_COLUMN } + +SELECT '--- Test default GRANULARITY (should be 100 mio. for usearch)---'; + +CREATE TABLE tab (id Int32, vector Array(Float32), INDEX usearch_index(vector) TYPE usearch) ENGINE=MergeTree ORDER BY id; +SHOW CREATE TABLE tab; +DROP TABLE tab; + +CREATE TABLE tab (id Int32, vector Array(Float32)) ENGINE=MergeTree ORDER BY id; +ALTER TABLE tab ADD INDEX usearch_index(vector) TYPE usearch; +SHOW CREATE TABLE tab; + +DROP TABLE tab; + + +SELECT '--- Test with Array, GRANULARITY = 1, index_granularity = 5 ---'; + +DROP TABLE IF EXISTS tab; +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch() GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 5; +INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); + +-- rows = 15, index_granularity = 5, GRANULARITY = 1 gives 3 usearch-indexed blocks (each comprising a single granule) +-- condition 'L2Distance(vector, reference_vector) < 1.0' ensures that only one usearch-indexed block produces results --> "Granules: 1/3" + +-- SELECT 'WHERE type, L2Distance'; +-- SELECT * +-- FROM tab +-- WHERE L2Distance(vector, [0.0, 0.0, 10.0]) < 1.0 +-- LIMIT 3; + +SELECT 'WHERE type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +WHERE L2Distance(vector, [0.0, 0.0, 10.0]) < 1.0 +LIMIT 3; + +-- SELECT 'ORDER BY type, L2Distance'; +-- SELECT * +-- FROM tab +-- ORDER BY L2Distance(vector, [0.0, 0.0, 10.0]) +-- LIMIT 3; + +SELECT 'ORDER BY type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(vector, [0.0, 0.0, 10.0]) +LIMIT 3; + +-- Test special cases. Corresponding special case tests are omitted from later tests. + +SELECT 'Reference ARRAYs with non-matching dimension are rejected'; +SELECT * +FROM tab +ORDER BY L2Distance(vector, [0.0, 0.0]) +LIMIT 3; -- { serverError INCORRECT_QUERY } + +SELECT 'Special case: MaximumDistance is negative'; +SELECT 'WHERE type, L2Distance'; +SELECT * +FROM tab +WHERE L2Distance(vector, [0.0, 0.0, 10.0]) < -1.0 +LIMIT 3; -- { serverError INCORRECT_QUERY } + +SELECT 'Special case: setting max_limit_for_ann_queries'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(vector, [5.3, 7.3, 2.1]) +LIMIT 3 +SETTINGS max_limit_for_ann_queries=2; -- doesn't use the ann index + +DROP TABLE tab; + +-- Test Tuple embeddings. Triggers different logic than Array inside MergeTreeIndexUSearch but the same logic as Array above MergeTreeIndexUSearch. +-- Therefore test Tuple case just once. + +SELECT '--- Test with Tuple, GRANULARITY = 1, index_granularity = 5 ---'; + +CREATE TABLE tab(id Int32, vector Tuple(Float32, Float32, Float32), INDEX usearch_index vector TYPE usearch() GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 5; +INSERT INTO tab VALUES (1, (0.0, 0.0, 10.0)), (2, (0.0, 0.0, 10.5)), (3, (0.0, 0.0, 9.5)), (4, (0.0, 0.0, 9.7)), (5, (0.0, 0.0, 10.2)), (6, (10.0, 0.0, 0.0)), (7, (9.5, 0.0, 0.0)), (8, (9.7, 0.0, 0.0)), (9, (10.2, 0.0, 0.0)), (10, (10.5, 0.0, 0.0)), (11, (0.0, 10.0, 0.0)), (12, (0.0, 9.5, 0.0)), (13, (0.0, 9.7, 0.0)), (14, (0.0, 10.2, 0.0)), (15, (0.0, 10.5, 0.0)); + +-- SELECT 'WHERE type, L2Distance'; +-- SELECT * +-- FROM tab +-- WHERE L2Distance(vector, (0.0, 0.0, 10.0)) < 1.0 +-- LIMIT 3; + +SELECT 'WHERE type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +WHERE L2Distance(vector, (0.0, 0.0, 10.0)) < 1.0 +LIMIT 3; + +-- SELECT 'ORDER BY type, L2Distance'; +-- SELECT * +-- FROM tab +-- ORDER BY L2Distance(vector, (0.0, 0.0, 10.0)) +-- LIMIT 3; + +SELECT 'ORDER BY type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(vector, (0.0, 0.0, 10.0)) +LIMIT 3; + +DROP TABLE tab; + +-- Not a systematic test, just to make sure no bad things happen +SELECT '--- Test non-default metric (cosine distance) ---'; + +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch('cosineDistance') GRANULARITY 1) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 5; +INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0]), (2, [0.0, 0.0, 10.5]), (3, [0.0, 0.0, 9.5]), (4, [0.0, 0.0, 9.7]), (5, [0.0, 0.0, 10.2]), (6, [10.0, 0.0, 0.0]), (7, [9.5, 0.0, 0.0]), (8, [9.7, 0.0, 0.0]), (9, [10.2, 0.0, 0.0]), (10, [10.5, 0.0, 0.0]), (11, [0.0, 10.0, 0.0]), (12, [0.0, 9.5, 0.0]), (13, [0.0, 9.7, 0.0]), (14, [0.0, 10.2, 0.0]), (15, [0.0, 10.5, 0.0]); + +-- SELECT 'WHERE type, L2Distance'; +-- SELECT * +-- FROM tab +-- WHERE L2Distance(vector, [0.0, 0.0, 10.0]) < 1.0 +-- LIMIT 3; + +-- SELECT 'ORDER BY type, L2Distance'; +-- SELECT * +-- FROM tab +-- ORDER BY L2Distance(vector, [0.0, 0.0, 10.0]) +-- LIMIT 3; + +DROP TABLE tab; + +SELECT '--- Test with Array, GRANULARITY = 2, index_granularity = 4 ---'; + +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch() GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 4; +INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0, 0.0]), (2, [0.0, 0.0, 10.5, 0.0]), (3, [0.0, 0.0, 9.5, 0.0]), (4, [0.0, 0.0, 9.7, 0.0]), (5, [10.0, 0.0, 0.0, 0.0]), (6, [9.5, 0.0, 0.0, 0.0]), (7, [9.7, 0.0, 0.0, 0.0]), (8, [10.2, 0.0, 0.0, 0.0]), (9, [0.0, 10.0, 0.0, 0.0]), (10, [0.0, 9.5, 0.0, 0.0]), (11, [0.0, 9.7, 0.0, 0.0]), (12, [0.0, 9.7, 0.0, 0.0]), (13, [0.0, 0.0, 0.0, 10.3]), (14, [0.0, 0.0, 0.0, 9.5]), (15, [0.0, 0.0, 0.0, 10.0]), (16, [0.0, 0.0, 0.0, 10.5]); + +-- rows = 16, index_granularity = 4, GRANULARITY = 2 gives 2 usearch-indexed blocks (each comprising two granules) +-- condition 'L2Distance(vector, reference_vector) < 1.0' ensures that only one usearch-indexed block produces results --> "Granules: 2/4" + +-- SELECT 'WHERE type, L2Distance'; +-- SELECT * +-- FROM tab +-- WHERE L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) < 5.0 +-- LIMIT 3; + +SELECT 'WHERE type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +WHERE L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) < 5.0 +LIMIT 3; + +-- SELECT 'ORDER BY type, L2Distance'; +-- SELECT * +-- FROM tab +-- ORDER BY L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) +-- LIMIT 3; + +SELECT 'ORDER BY type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) +LIMIT 3; + +DROP TABLE tab; + +SELECT '--- Test with Array, GRANULARITY = 4, index_granularity = 4 ---'; + +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch() GRANULARITY 4) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 4; +INSERT INTO tab VALUES (1, [0.0, 0.0, 10.0, 0.0]), (2, [0.0, 0.0, 10.5, 0.0]), (3, [0.0, 0.0, 9.5, 0.0]), (4, [0.0, 0.0, 9.7, 0.0]), (5, [10.0, 0.0, 0.0, 0.0]), (6, [9.5, 0.0, 0.0, 0.0]), (7, [9.7, 0.0, 0.0, 0.0]), (8, [10.2, 0.0, 0.0, 0.0]), (9, [0.0, 10.0, 0.0, 0.0]), (10, [0.0, 9.5, 0.0, 0.0]), (11, [0.0, 9.7, 0.0, 0.0]), (12, [0.0, 9.7, 0.0, 0.0]), (13, [0.0, 0.0, 0.0, 10.3]), (14, [0.0, 0.0, 0.0, 9.5]), (15, [0.0, 0.0, 0.0, 10.0]), (16, [0.0, 0.0, 0.0, 10.5]); + +-- rows = 16, index_granularity = 4, GRANULARITY = 4 gives a single usearch-indexed block (comprising all granules) +-- no two matches happen to be located in the same granule, so with LIMIT = 3, we'll get "Granules: 2/4" + +-- SELECT 'WHERE type, L2Distance'; +-- SELECT * +-- FROM tab +-- WHERE L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) < 5.0 +-- LIMIT 3; + +SELECT 'WHERE type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +WHERE L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) < 5.0 +LIMIT 3; + +-- SELECT 'ORDER BY type, L2Distance'; +-- SELECT * +-- FROM tab +-- ORDER BY L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) +-- LIMIT 3; + +SELECT 'ORDER BY type, L2Distance, check that index is used'; +EXPLAIN indexes=1 +SELECT * +FROM tab +ORDER BY L2Distance(vector, [10.0, 0.0, 10.0, 0.0]) +LIMIT 3; + +DROP TABLE tab; From bf40767f10e16d9fd6c5b29a8af1ae81c93694fc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 15 Aug 2023 14:27:49 +0200 Subject: [PATCH 569/777] fix another race --- src/Storages/MergeTree/MergeTreeData.cpp | 25 +++++++++++++------ src/Storages/StorageMergeTree.cpp | 12 ++++++--- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++- ..._replace_partition_from_table_zookeeper.sh | 20 --------------- .../00933_ttl_replicated_zookeeper.sh | 16 ------------ ...034_move_partition_from_table_zookeeper.sh | 17 ------------- .../02443_detach_attach_partition.sh | 2 +- .../0_stateless/02482_load_parts_refcounts.sh | 17 ------------- tests/queries/shell_config.sh | 20 +++++++++++++++ 9 files changed, 51 insertions(+), 82 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 561eef28c78..4026be31286 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5832,18 +5832,21 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const { const String source_dir = "detached/"; - std::map name_to_disk; - /// Let's compose a list of parts that should be added. if (attach_part) { const String part_id = partition->as().value.safeGet(); validateDetachedPartName(part_id); - auto disk = getDiskForDetachedPart(part_id); - renamed_parts.addPart(part_id, "attaching_" + part_id, disk); - - if (MergeTreePartInfo::tryParsePartName(part_id, format_version)) - name_to_disk[part_id] = getDiskForDetachedPart(part_id); + if (temporary_parts.contains(String(DETACHED_DIR_NAME) + "/" + part_id)) + { + LOG_WARNING(log, "Will not try to attach part {} because its directory is temporary, " + "probably it's being detached right now", part_id); + } + else + { + auto disk = getDiskForDetachedPart(part_id); + renamed_parts.addPart(part_id, "attaching_" + part_id, disk); + } } else { @@ -5860,6 +5863,12 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const for (const auto & part_info : detached_parts) { + if (temporary_parts.contains(String(DETACHED_DIR_NAME) + "/" + part_info.dir_name)) + { + LOG_WARNING(log, "Will not try to attach part {} because its directory is temporary, " + "probably it's being detached right now", part_info.dir_name); + continue; + } LOG_DEBUG(log, "Found part {}", part_info.dir_name); active_parts.add(part_info.dir_name); } @@ -5870,6 +5879,8 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const for (const auto & part_info : detached_parts) { const String containing_part = active_parts.getContainingPart(part_info.dir_name); + if (containing_part.empty()) + continue; LOG_DEBUG(log, "Found containing part {} for part {}", containing_part, part_info.dir_name); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9506d6f1075..03bb1b554eb 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1816,7 +1816,9 @@ void StorageMergeTree::dropPart(const String & part_name, bool detach, ContextPt if (detach) { auto metadata_snapshot = getInMemoryMetadataPtr(); - LOG_INFO(log, "Detaching {}", part->getDataPartStorage().getPartDirectory()); + String part_dir = part->getDataPartStorage().getPartDirectory(); + LOG_INFO(log, "Detaching {}", part_dir); + auto holder = getTemporaryPartDirectoryHolder(String(DETACHED_DIR_NAME) + "/" + part_dir); part->makeCloneInDetached("", metadata_snapshot, /*disk_transaction*/ {}); } @@ -1901,7 +1903,9 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont for (const auto & part : parts) { auto metadata_snapshot = getInMemoryMetadataPtr(); - LOG_INFO(log, "Detaching {}", part->getDataPartStorage().getPartDirectory()); + String part_dir = part->getDataPartStorage().getPartDirectory(); + LOG_INFO(log, "Detaching {}", part_dir); + auto holder = getTemporaryPartDirectoryHolder(String(DETACHED_DIR_NAME) + "/" + part_dir); part->makeCloneInDetached("", metadata_snapshot, /*disk_transaction*/ {}); } } @@ -1943,7 +1947,9 @@ void StorageMergeTree::dropPartsImpl(DataPartsVector && parts_to_remove, bool de /// NOTE: no race with background cleanup until we hold pointers to parts for (const auto & part : parts_to_remove) { - LOG_INFO(log, "Detaching {}", part->getDataPartStorage().getPartDirectory()); + String part_dir = part->getDataPartStorage().getPartDirectory(); + LOG_INFO(log, "Detaching {}", part_dir); + auto holder = getTemporaryPartDirectoryHolder(String(DETACHED_DIR_NAME) + "/" + part_dir); part->makeCloneInDetached("", metadata_snapshot, /*disk_transaction*/ {}); } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index bc2cff80c59..6b4ee3334c7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2097,7 +2097,9 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) { if (auto part_to_detach = part.getPartIfItWasActive()) { - LOG_INFO(log, "Detaching {}", part_to_detach->getDataPartStorage().getPartDirectory()); + String part_dir = part_to_detach->getDataPartStorage().getPartDirectory(); + LOG_INFO(log, "Detaching {}", part_dir); + auto holder = getTemporaryPartDirectoryHolder(String(DETACHED_DIR_NAME) + "/" + part_dir); part_to_detach->makeCloneInDetached("", metadata_snapshot, /*disk_transaction*/ {}); } } diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index c32b6d04a42..334025cba28 100755 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -11,26 +11,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -function query_with_retry -{ - local query="$1" && shift - - local retry=0 - until [ $retry -ge 5 ] - do - local result - result="$($CLICKHOUSE_CLIENT "$@" --query="$query" 2>&1)" - if [ "$?" == 0 ]; then - echo -n "$result" - return - else - retry=$((retry + 1)) - sleep 3 - fi - done - echo "Query '$query' failed with '$result'" -} - $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS src;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r1;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r2;" diff --git a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sh b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sh index 22d9e0690b3..d06037fb836 100755 --- a/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/00933_ttl_replicated_zookeeper.sh @@ -5,22 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -function query_with_retry -{ - retry=0 - until [ $retry -ge 5 ] - do - result=$($CLICKHOUSE_CLIENT $2 --query="$1" 2>&1) - if [ "$?" == 0 ]; then - echo -n "$result" - return - else - retry=$(($retry + 1)) - sleep 3 - fi - done - echo "Query '$1' failed with '$result'" -} $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS ttl_repl1" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS ttl_repl2" diff --git a/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh index e0a84323dbd..39c5742e7a7 100755 --- a/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/01034_move_partition_from_table_zookeeper.sh @@ -7,23 +7,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -function query_with_retry -{ - retry=0 - until [ $retry -ge 5 ] - do - result=$($CLICKHOUSE_CLIENT $2 --query="$1" 2>&1) - if [ "$?" == 0 ]; then - echo -n "$result" - return - else - retry=$(($retry + 1)) - sleep 3 - fi - done - echo "Query '$1' failed with '$result'" -} - $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS src;" $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst;" diff --git a/tests/queries/0_stateless/02443_detach_attach_partition.sh b/tests/queries/0_stateless/02443_detach_attach_partition.sh index 13ea966dbf5..5a3f1b64065 100755 --- a/tests/queries/0_stateless/02443_detach_attach_partition.sh +++ b/tests/queries/0_stateless/02443_detach_attach_partition.sh @@ -55,7 +55,7 @@ wait $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table0" $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table1" -while ! $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table0 ATTACH PARTITION ID 'all'" 2>/dev/null; do sleep 0.5; done +query_with_retry "ALTER TABLE alter_table0 ATTACH PARTITION ID 'all'" 2>/dev/null; $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table1 ATTACH PARTITION ID 'all'" 2>/dev/null $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA alter_table1" $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table1 ATTACH PARTITION ID 'all'" diff --git a/tests/queries/0_stateless/02482_load_parts_refcounts.sh b/tests/queries/0_stateless/02482_load_parts_refcounts.sh index 4d588dabeb9..fe3cee1359e 100755 --- a/tests/queries/0_stateless/02482_load_parts_refcounts.sh +++ b/tests/queries/0_stateless/02482_load_parts_refcounts.sh @@ -5,23 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -function query_with_retry -{ - retry=0 - until [ $retry -ge 5 ] - do - result=$($CLICKHOUSE_CLIENT $2 --query="$1" 2>&1) - if [ "$?" == 0 ]; then - echo -n "$result" - return - else - retry=$(($retry + 1)) - sleep 3 - fi - done - echo "Query '$1' failed with '$result'" -} - $CLICKHOUSE_CLIENT -n --query " DROP TABLE IF EXISTS load_parts_refcounts SYNC; diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index ef70c82aefc..12bc0002191 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -155,3 +155,23 @@ function random_str() local n=$1 && shift tr -cd '[:lower:]' < /dev/urandom | head -c"$n" } + +function query_with_retry +{ + local query="$1" && shift + + local retry=0 + until [ $retry -ge 5 ] + do + local result + result="$($CLICKHOUSE_CLIENT "$@" --query="$query" 2>&1)" + if [ "$?" == 0 ]; then + echo -n "$result" + return + else + retry=$((retry + 1)) + sleep 3 + fi + done + echo "Query '$query' failed with '$result'" +} From 2aa211acc2af778728f87a0cf36be8efb68243b3 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 15 Aug 2023 13:26:39 +0000 Subject: [PATCH 570/777] Added integration test for session log --- .../test.py | 5 +- tests/integration/test_session_log/.gitignore | 1 + .../integration/test_session_log/__init__.py | 0 .../test_session_log/configs/log.xml | 9 + .../test_session_log/configs/ports.xml | 9 + .../test_session_log/configs/session_log.xml | 9 + .../test_session_log/configs/users.xml | 23 ++ .../protos/clickhouse_grpc.proto | 1 + tests/integration/test_session_log/test.py | 292 ++++++++++++++++++ 9 files changed, 345 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_session_log/.gitignore create mode 100644 tests/integration/test_session_log/__init__.py create mode 100644 tests/integration/test_session_log/configs/log.xml create mode 100644 tests/integration/test_session_log/configs/ports.xml create mode 100644 tests/integration/test_session_log/configs/session_log.xml create mode 100644 tests/integration/test_session_log/configs/users.xml create mode 120000 tests/integration/test_session_log/protos/clickhouse_grpc.proto create mode 100644 tests/integration/test_session_log/test.py diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index c5c33b1cddb..5eaef09bf6d 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -28,10 +28,7 @@ proto_dir = os.path.join(SCRIPT_DIR, "./protos") gen_dir = os.path.join(SCRIPT_DIR, "./_gen") os.makedirs(gen_dir, exist_ok=True) run_and_check( - "python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \ - {proto_dir}/clickhouse_grpc.proto".format( - proto_dir=proto_dir, gen_dir=gen_dir - ), + f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", shell=True, ) diff --git a/tests/integration/test_session_log/.gitignore b/tests/integration/test_session_log/.gitignore new file mode 100644 index 00000000000..edf565ec632 --- /dev/null +++ b/tests/integration/test_session_log/.gitignore @@ -0,0 +1 @@ +_gen diff --git a/tests/integration/test_session_log/__init__.py b/tests/integration/test_session_log/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_session_log/configs/log.xml b/tests/integration/test_session_log/configs/log.xml new file mode 100644 index 00000000000..7a079b81e69 --- /dev/null +++ b/tests/integration/test_session_log/configs/log.xml @@ -0,0 +1,9 @@ + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/ports.xml b/tests/integration/test_session_log/configs/ports.xml new file mode 100644 index 00000000000..fbaefc16b3a --- /dev/null +++ b/tests/integration/test_session_log/configs/ports.xml @@ -0,0 +1,9 @@ + + 5433 + 9001 + 9100 + + + false + + \ No newline at end of file diff --git a/tests/integration/test_session_log/configs/session_log.xml b/tests/integration/test_session_log/configs/session_log.xml new file mode 100644 index 00000000000..a0e4e3e2216 --- /dev/null +++ b/tests/integration/test_session_log/configs/session_log.xml @@ -0,0 +1,9 @@ + + + system + session_log
+ + toYYYYMM(event_date) + 7500 +
+
diff --git a/tests/integration/test_session_log/configs/users.xml b/tests/integration/test_session_log/configs/users.xml new file mode 100644 index 00000000000..0416dfadc8a --- /dev/null +++ b/tests/integration/test_session_log/configs/users.xml @@ -0,0 +1,23 @@ + + + + 0 + + + + + + + pass + + + pass + + + pass + + + pass + + + \ No newline at end of file diff --git a/tests/integration/test_session_log/protos/clickhouse_grpc.proto b/tests/integration/test_session_log/protos/clickhouse_grpc.proto new file mode 120000 index 00000000000..25d15f11e3b --- /dev/null +++ b/tests/integration/test_session_log/protos/clickhouse_grpc.proto @@ -0,0 +1 @@ +../../../../src/Server/grpc_protos/clickhouse_grpc.proto \ No newline at end of file diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py new file mode 100644 index 00000000000..b860cde1df6 --- /dev/null +++ b/tests/integration/test_session_log/test.py @@ -0,0 +1,292 @@ +import os + +import grpc +import pymysql.connections +import psycopg2 as py_psql +import pytest +import random +import logging +import sys +import threading + +from helpers.cluster import ClickHouseCluster, run_and_check + +POSTGRES_SERVER_PORT = 5433 +MYSQL_SERVER_PORT = 9001 +GRPC_PORT = 9100 +SESSION_LOG_MATCHING_FIELDS = "auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DEFAULT_ENCODING = "utf-8" + +# Use grpcio-tools to generate *pb2.py files from *.proto. +proto_dir = os.path.join(SCRIPT_DIR, "./protos") +gen_dir = os.path.join(SCRIPT_DIR, "./_gen") +os.makedirs(gen_dir, exist_ok=True) +run_and_check( + f"python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} {proto_dir}/clickhouse_grpc.proto", + shell=True, +) + +sys.path.append(gen_dir) + +import clickhouse_grpc_pb2 +import clickhouse_grpc_pb2_grpc + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "node", + main_configs=[ + "configs/ports.xml", + "configs/log.xml", + "configs/session_log.xml", + ], + user_configs=["configs/users.xml"], + # Bug in TSAN reproduces in this test https://github.com/grpc/grpc/issues/29550#issuecomment-1188085387 + env_variables={ + "TSAN_OPTIONS": "report_atomic_races=0 " + os.getenv("TSAN_OPTIONS", default="") + }, + with_postgres=True, +) + + +def grpc_get_url(): + return f"{instance.ip_address}:{GRPC_PORT}" + + +def grpc_create_insecure_channel(): + channel = grpc.insecure_channel(grpc_get_url()) + grpc.channel_ready_future(channel).result(timeout=2) + return channel + + +session_id_counter = 0 + + +def next_session_id(): + global session_id_counter + session_id = session_id_counter + session_id_counter += 1 + return str(session_id) + + +def grpc_query(query, user_, pass_, raise_exception): + try: + query_info = clickhouse_grpc_pb2.QueryInfo( + query=query, + session_id=next_session_id(), + user_name=user_, + password=pass_, + ) + channel = grpc_create_insecure_channel() + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel) + result = stub.ExecuteQuery(query_info) + if result and result.HasField("exception"): + raise Exception(result.exception.display_text) + + return result.output.decode(DEFAULT_ENCODING) + except Exception: + assert raise_exception + + +def postgres_query(query, user_, pass_, raise_exception): + try: + connection_string = f"host={instance.hostname} port={POSTGRES_SERVER_PORT} dbname=default user={user_} password={pass_}" + cluster.exec_in_container(cluster.postgres_id, + [ + "/usr/bin/psql", + connection_string, + "--no-align", + "--field-separator=' '", + "-c", + query + ], + shell=True + ) + except Exception: + assert raise_exception + + +def mysql_query(query, user_, pass_, raise_exception): + try: + client = pymysql.connections.Connection( + host=instance.ip_address, + user=user_, + password=pass_, + database="default", + port=MYSQL_SERVER_PORT, + ) + cursor = client.cursor(pymysql.cursors.DictCursor) + if raise_exception: + with pytest.raises(Exception): + cursor.execute(query) + else: + cursor.execute(query) + cursor.fetchall() + except Exception: + assert raise_exception + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_grpc_session(started_cluster): + grpc_query("SELECT 1", "grpc_user", "pass", False) + grpc_query("SELECT 2", "grpc_user", "wrong_pass", True) + grpc_query("SELECT 3", "wrong_grpc_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "grpc_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'Logout'" + ) + assert logout_records == "grpc_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='grpc_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "grpc_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'grpc_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_mysql_session(started_cluster): + mysql_query("SELECT 1", "mysql_user", "pass", False) + mysql_query("SELECT 2", "mysql_user", "wrong_pass", True) + mysql_query("SELECT 3", "wrong_mysql_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "mysql_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'Logout'" + ) + assert logout_records == "mysql_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='mysql_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "mysql_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'mysql_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_postgres_session(started_cluster): + postgres_query("SELECT 1", "postgres_user", "pass", False) + postgres_query("SELECT 2", "postgres_user", "wrong_pass", True) + postgres_query("SELECT 3", "wrong_postgres_user", "pass", True) + + instance.query("SYSTEM FLUSH LOGS") + login_success_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginSuccess'" + ) + assert login_success_records == "postgres_user\t1\t1\n" + logout_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'Logout'" + ) + assert logout_records == "postgres_user\t1\t1\n" + login_failure_records = instance.query( + "SELECT user, client_port <> 0, client_address <> toIPv6('::') FROM system.session_log WHERE user='postgres_user' AND type = 'LoginFailure'" + ) + assert login_failure_records == "postgres_user\t1\t1\n" + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'postgres_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "1\n" + + +def test_parallel_sessions(started_cluster): + thread_list = [] + for _ in range(10): + # Sleep time does not significantly matter here, + # test should pass even without sleeping. + for function in [postgres_query, grpc_query, mysql_query]: + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "parallel_user", + "pass", + False, + ), + ) + thread.start() + thread_list.append(thread) + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "parallel_user", + "wrong_pass", + True, + ), + ) + thread.start() + thread_list.append(thread) + thread = threading.Thread( + target=function, + args=( + f"SELECT sleep({random.uniform(0.03, 0.04)})", + "wrong_parallel_user", + "pass", + True, + ), + ) + thread.start() + thread_list.append(thread) + + for thread in thread_list: + thread.join() + + instance.query("SYSTEM FLUSH LOGS") + port_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user'" + ) + assert port_0_sessions == "90\n" + + port_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_port = 0" + ) + assert port_0_sessions == "0\n" + + address_0_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND client_address = toIPv6('::')" + ) + assert address_0_sessions == "0\n" + + grpc_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'gRPC'" + ) + assert grpc_sessions == "30\n" + + mysql_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'MySQL'" + ) + assert mysql_sessions == "30\n" + + postgres_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND interface = 'PostgreSQL'" + ) + assert postgres_sessions == "30\n" + + logins_and_logouts = instance.query( + f"SELECT COUNT(*) FROM (SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginSuccess' INTERSECT SELECT {SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = 'parallel_user' AND type = 'Logout')" + ) + assert logins_and_logouts == "30\n" + + logout_failure_sessions = instance.query( + f"SELECT COUNT(*) FROM system.session_log WHERE user = 'parallel_user' AND type = 'LoginFailure'" + ) + assert logout_failure_sessions == "30\n" From 0e1728801eccb11a9cadf181fc3f555a4e39e125 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 15 Aug 2023 13:31:53 +0000 Subject: [PATCH 571/777] black run --- tests/integration/test_session_log/test.py | 28 +++++++++++----------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py index b860cde1df6..bb7cafa4ee6 100644 --- a/tests/integration/test_session_log/test.py +++ b/tests/integration/test_session_log/test.py @@ -2,10 +2,8 @@ import os import grpc import pymysql.connections -import psycopg2 as py_psql import pytest import random -import logging import sys import threading @@ -92,19 +90,20 @@ def grpc_query(query, user_, pass_, raise_exception): def postgres_query(query, user_, pass_, raise_exception): try: connection_string = f"host={instance.hostname} port={POSTGRES_SERVER_PORT} dbname=default user={user_} password={pass_}" - cluster.exec_in_container(cluster.postgres_id, - [ - "/usr/bin/psql", - connection_string, - "--no-align", - "--field-separator=' '", - "-c", - query - ], - shell=True - ) + cluster.exec_in_container( + cluster.postgres_id, + [ + "/usr/bin/psql", + connection_string, + "--no-align", + "--field-separator=' '", + "-c", + query, + ], + shell=True, + ) except Exception: - assert raise_exception + assert raise_exception def mysql_query(query, user_, pass_, raise_exception): @@ -126,6 +125,7 @@ def mysql_query(query, user_, pass_, raise_exception): except Exception: assert raise_exception + @pytest.fixture(scope="module") def started_cluster(): try: From 78d8557a566ac900a83e1f9aad18e5f49d4a3b96 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 15 Aug 2023 15:37:09 +0200 Subject: [PATCH 572/777] fix --- src/Common/MemoryTracker.cpp | 5 ++++- src/DataTypes/registerDataTypeDateTime.cpp | 10 +++++----- src/Interpreters/Session.cpp | 2 +- src/Server/TCPHandler.cpp | 20 ++++++++++++------- .../0_stateless/02668_ulid_decoding.sql | 2 +- 5 files changed, 24 insertions(+), 15 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 7005a57d175..93bd50a0b49 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -151,8 +151,11 @@ void MemoryTracker::logPeakMemoryUsage() { log_peak_memory_usage_in_destructor = false; const auto * description = description_ptr.load(std::memory_order_relaxed); + auto peak_bytes = peak.load(std::memory_order::relaxed); + if (peak_bytes < 128 * 1024) + return; LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), - "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak)); + "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak_bytes)); } void MemoryTracker::logMemoryUsage(Int64 current) const diff --git a/src/DataTypes/registerDataTypeDateTime.cpp b/src/DataTypes/registerDataTypeDateTime.cpp index 8080179ad47..2b5c4a0a143 100644 --- a/src/DataTypes/registerDataTypeDateTime.cpp +++ b/src/DataTypes/registerDataTypeDateTime.cpp @@ -22,11 +22,11 @@ enum class ArgumentKind Mandatory }; -String getExceptionMessage( +PreformattedMessage getExceptionMessage( const String & message, size_t argument_index, const char * argument_name, const std::string & context_data_type_name, Field::Types::Which field_type) { - return fmt::format("Parameter #{} '{}' for {}{}, expected {} literal", + return PreformattedMessage::create("Parameter #{} '{}' for {}{}, expected {} literal", argument_index, argument_name, context_data_type_name, message, field_type); } @@ -47,10 +47,10 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume else { if (argument && argument->value.getType() != field_type) - throw Exception::createDeprecated(getExceptionMessage(fmt::format(" has wrong type: {}", argument->value.getTypeName()), + throw Exception(getExceptionMessage(fmt::format(" has wrong type: {}", argument->value.getTypeName()), argument_index, argument_name, context_data_type_name, field_type), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); else - throw Exception::createDeprecated(getExceptionMessage(" is missing", argument_index, argument_name, context_data_type_name, field_type), + throw Exception(getExceptionMessage(" is missing", argument_index, argument_name, context_data_type_name, field_type), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } } @@ -67,7 +67,7 @@ static DataTypePtr create(const ASTPtr & arguments) const auto timezone = getArgument(arguments, scale ? 1 : 0, "timezone", "DateTime"); if (!scale && !timezone) - throw Exception::createDeprecated(getExceptionMessage(" has wrong type: ", 0, "scale", "DateTime", Field::Types::Which::UInt64), + throw Exception(getExceptionMessage(" has wrong type: ", 0, "scale", "DateTime", Field::Types::Which::UInt64), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); /// If scale is defined, the data type is DateTime when scale = 0 otherwise the data type is DateTime64 diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index bcfaae40a03..e0b5db44593 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -622,7 +622,7 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t if (auto query_context_user = query_context->getAccess()->tryGetUser()) { - LOG_DEBUG(log, "{} Creating query context from {} context, user_id: {}, parent context user: {}", + LOG_TRACE(log, "{} Creating query context from {} context, user_id: {}, parent context user: {}", toString(auth_id), from_session_context ? "session" : "global", toString(*user_id), diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index cd17c2bcb55..ac3928b4abe 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -263,6 +263,17 @@ void TCPHandler::runImpl() std::unique_ptr exception; bool network_error = false; bool query_duration_already_logged = false; + auto log_query_duration = [this, &query_duration_already_logged]() + { + if (query_duration_already_logged) + return; + query_duration_already_logged = true; + auto elapsed_sec = state.watch.elapsedSeconds(); + /// We already logged more detailed info if we read some rows + if (elapsed_sec < 1.0 && state.progress.read_rows) + return; + LOG_DEBUG(log, "Processed in {} sec.", elapsed_sec); + }; try { @@ -492,9 +503,7 @@ void TCPHandler::runImpl() /// Do it before sending end of stream, to have a chance to show log message in client. query_scope->logPeakMemoryUsage(); - - LOG_DEBUG(log, "Processed in {} sec.", state.watch.elapsedSeconds()); - query_duration_already_logged = true; + log_query_duration(); if (state.is_connection_closed) break; @@ -616,10 +625,7 @@ void TCPHandler::runImpl() LOG_WARNING(log, "Can't skip data packets after query failure."); } - if (!query_duration_already_logged) - { - LOG_DEBUG(log, "Processed in {} sec.", state.watch.elapsedSeconds()); - } + log_query_duration(); /// QueryState should be cleared before QueryScope, since otherwise /// the MemoryTracker will be wrong for possible deallocations. diff --git a/tests/queries/0_stateless/02668_ulid_decoding.sql b/tests/queries/0_stateless/02668_ulid_decoding.sql index df94025b7b5..ecab5004df6 100644 --- a/tests/queries/0_stateless/02668_ulid_decoding.sql +++ b/tests/queries/0_stateless/02668_ulid_decoding.sql @@ -4,7 +4,7 @@ SELECT dateDiff('minute', ULIDStringToDateTime(generateULID()), now()) = 0; SELECT toTimezone(ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E'), 'America/Costa_Rica'); SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E', 'America/Costa_Rica'); SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9', 'America/Costa_Rica'); -- { serverError ILLEGAL_COLUMN } -SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E', 'America/Costa_Ric'); -- { serverError POCO_EXCEPTION } +SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E', 'America/Costa_Ric'); -- { serverError BAD_ARGUMENTS } SELECT ULIDStringToDateTime('01GWJWKW30MFPQJRYEAF4XFZ9E0'); -- { serverError ILLEGAL_COLUMN } SELECT ULIDStringToDateTime(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT ULIDStringToDateTime(1, 2); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From 0fd28bf3309a65e5c0204c814bef0a5f13dada9d Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 15 Aug 2023 13:39:34 +0000 Subject: [PATCH 573/777] added remote session log test --- .../02834_remote_session_log.reference | 13 +++++ .../0_stateless/02834_remote_session_log.sh | 56 +++++++++++++++++++ 2 files changed, 69 insertions(+) create mode 100644 tests/queries/0_stateless/02834_remote_session_log.reference create mode 100755 tests/queries/0_stateless/02834_remote_session_log.sh diff --git a/tests/queries/0_stateless/02834_remote_session_log.reference b/tests/queries/0_stateless/02834_remote_session_log.reference new file mode 100644 index 00000000000..e2680982ab0 --- /dev/null +++ b/tests/queries/0_stateless/02834_remote_session_log.reference @@ -0,0 +1,13 @@ +0 +0 +0 +0 +client_port 0 connections: +0 +client_address '::' connections: +0 +login failures: +0 +TCP Login and logout count is equal +HTTP Login and logout count is equal +MySQL Login and logout count is equal diff --git a/tests/queries/0_stateless/02834_remote_session_log.sh b/tests/queries/0_stateless/02834_remote_session_log.sh new file mode 100755 index 00000000000..3bedfb6c9ee --- /dev/null +++ b/tests/queries/0_stateless/02834_remote_session_log.sh @@ -0,0 +1,56 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ +readonly TEST_USER=$"02834_USER_${PID}" +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER} IDENTIFIED WITH plaintext_password BY 'pass'" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT CREATE TEMPORARY TABLE, MYSQL, REMOTE ON *.* TO ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" + +${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ + -d "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" + +${CLICKHOUSE_CURL} -sS -X POST "${CLICKHOUSE_URL}&user=${TEST_USER}&password=pass" \ + -d "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" + +${CLICKHOUSE_CLIENT} -q "SELECT * FROM remote('127.0.0.1:${CLICKHOUSE_PORT_TCP}', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" +${CLICKHOUSE_CLIENT} -q "SELECT * FROM mysql('127.0.0.1:9004', 'system', 'one', '${TEST_USER}', 'pass')" -u "${TEST_USER}" --password "pass" + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "client_port 0 connections:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_port = 0" + +echo "client_address '::' connections:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and client_address = toIPv6('::')" + +echo "login failures:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' and type = 'LoginFailure'" + +# remote(...) function sometimes reuses old cached sessions for query execution. +# This makes LoginSuccess/Logout entries count unstable, but success and logouts must always match. + +for interface in 'TCP' 'HTTP' 'MySQL' +do + LOGIN_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}'"` + CORRESPONDING_LOGOUT_RECORDS_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' AND interface = '${interface}' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}')"` + + if [ "$LOGIN_COUNT" == "$CORRESPONDING_LOGOUT_RECORDS_COUNT" ]; then + echo "${interface} Login and logout count is equal" + else + TOTAL_LOGOUT_COUNT=`${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout' AND interface = '${interface}'"` + echo "${interface} Login count ${LOGIN_COUNT} != corresponding logout count ${CORRESPONDING_LOGOUT_RECORDS_COUNT}. TOTAL_LOGOUT_COUNT ${TOTAL_LOGOUT_COUNT}" + fi +done + +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" From cbf9f88b90f69a08bd51377338d2a679e629cd82 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 15 Aug 2023 13:42:42 +0000 Subject: [PATCH 574/777] Added concurrent session session_log tests --- .../02833_concurrrent_sessions.reference | 34 +++++ .../0_stateless/02833_concurrrent_sessions.sh | 138 ++++++++++++++++++ 2 files changed, 172 insertions(+) create mode 100644 tests/queries/0_stateless/02833_concurrrent_sessions.reference create mode 100755 tests/queries/0_stateless/02833_concurrrent_sessions.sh diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.reference b/tests/queries/0_stateless/02833_concurrrent_sessions.reference new file mode 100644 index 00000000000..bfe507e8eac --- /dev/null +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.reference @@ -0,0 +1,34 @@ +sessions: +150 +port_0_sessions: +0 +address_0_sessions: +0 +tcp_sessions +60 +http_sessions +30 +http_with_session_id_sessions +30 +my_sql_sessions +30 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 +Corresponding LoginSuccess/Logout +10 +LoginFailure +10 diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh new file mode 100755 index 00000000000..26b48462a76 --- /dev/null +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.sh @@ -0,0 +1,138 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-debug + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ + +# Each user uses a separate thread. +readonly TCP_USERS=( "02833_TCP_USER_${PID}"_{1,2} ) # 2 concurrent TCP users +readonly HTTP_USERS=( "02833_HTTP_USER_${PID}" ) +readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "02833_HTTP_WITH_SESSION_ID_USER_${PID}" ) +readonly MYSQL_USERS=( "02833_MYSQL_USER_${PID}") +readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) + +readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" + +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +for user in "${ALL_USERS[@]}"; do + ${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${user} IDENTIFIED WITH plaintext_password BY 'pass'" + ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.* TO ${user}" + ${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON INFORMATION_SCHEMA.* TO ${user}"; +done + +# All _session functions execute in separate threads. +# These functions try to create a session with successful login and logout. +# Sleep a small, random amount of time to make concurrency more intense. +# and try to login with an invalid password. +function tcp_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM})" --user="${user}" --password="pass" + # login failure + ${CLICKHOUSE_CLIENT} -q "SELECT 2" --user="${user}" --password 'invalid' + done +} + +function http_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT 3, sleep(0.01${RANDOM})" + + # login failure + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=wrong" -d "SELECT 4" + done +} + +function http_with_session_id_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=pass" -d "SELECT 5, sleep 0.01${RANDOM}" + + # login failure + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&session_id=${user}&user=${user}&password=wrong" -d "SELECT 6" + done +} + +function mysql_session() +{ + local user=$1 + local i=0 + while (( (i++) < 10 )); do + # login logout + ${CLICKHOUSE_CLIENT} -q "SELECT 1, sleep(0.01${RANDOM}) FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'pass')" + + # login failure + ${CLICKHOUSE_CLIENT} -q "SELECT 1 FROM mysql('127.0.0.1:9004', 'system', 'one', '${user}', 'wrong', SETTINGS connection_max_tries=1)" + done +} + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" + +export -f tcp_session; +export -f http_session; +export -f http_with_session_id_session; +export -f mysql_session; + +for user in "${TCP_USERS[@]}"; do + timeout 60s bash -c "tcp_session ${user}" >/dev/null 2>&1 & +done + +for user in "${HTTP_USERS[@]}"; do + timeout 60s bash -c "http_session ${user}" >/dev/null 2>&1 & +done + +for user in "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}"; do + timeout 60s bash -c "http_with_session_id_session ${user}" >/dev/null 2>&1 & +done + +for user in "${MYSQL_USERS[@]}"; do + timeout 60s bash -c "mysql_session ${user}" >/dev/null 2>&1 & +done + +wait + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING})" + +echo "port_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_port = 0" + +echo "address_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${ALL_USERS_SQL_COLLECTION_STRING}) AND client_address = toIPv6('::')" + +echo "tcp_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${TCP_USERS_SQL_COLLECTION_STRING}) AND interface = 'TCP'" +echo "http_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" +echo "http_with_session_id_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING}) AND interface = 'HTTP'" +echo "my_sql_sessions" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user IN (${MYSQL_USERS_SQL_COLLECTION_STRING}) AND interface = 'MySQL'" + +for user in "${ALL_USERS[@]}"; do + ${CLICKHOUSE_CLIENT} -q "DROP USER ${user}" + echo "Corresponding LoginSuccess/Logout" + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${user}' AND type = 'Logout')" + echo "LoginFailure" + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${user}' AND type = 'LoginFailure'" + done From 4b5874b512802022e4c5581e17c9ed86c505129e Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Tue, 15 Aug 2023 13:45:06 +0000 Subject: [PATCH 575/777] added drop user during session test --- .../02835_drop_user_during_session.reference | 8 ++ .../02835_drop_user_during_session.sh | 114 ++++++++++++++++++ 2 files changed, 122 insertions(+) create mode 100644 tests/queries/0_stateless/02835_drop_user_during_session.reference create mode 100755 tests/queries/0_stateless/02835_drop_user_during_session.sh diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.reference b/tests/queries/0_stateless/02835_drop_user_during_session.reference new file mode 100644 index 00000000000..7252faab8c6 --- /dev/null +++ b/tests/queries/0_stateless/02835_drop_user_during_session.reference @@ -0,0 +1,8 @@ +port_0_sessions: +0 +address_0_sessions: +0 +Corresponding LoginSuccess/Logout +9 +LoginFailure +0 diff --git a/tests/queries/0_stateless/02835_drop_user_during_session.sh b/tests/queries/0_stateless/02835_drop_user_during_session.sh new file mode 100755 index 00000000000..347ebd22f96 --- /dev/null +++ b/tests/queries/0_stateless/02835_drop_user_during_session.sh @@ -0,0 +1,114 @@ +#!/usr/bin/env bash +# Tags: no-debug + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +readonly PID=$$ + +readonly TEST_USER="02835_USER_${PID}" +readonly TEST_ROLE="02835_ROLE_${PID}" +readonly TEST_PROFILE="02835_PROFILE_${PID}" +readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" + +function tcp_session() +{ + local user=$1 + ${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.numbers" --user="${user}" +} + +function http_session() +{ + local user=$1 + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" +} + +function http_with_session_id_session() +{ + local user=$1 + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=${user}&password=pass" -d "SELECT COUNT(*) FROM system.numbers" +} + +# Busy-waits until user $1, specified amount of queries ($2) will run simultaneously. +function wait_for_queries_start() +{ + local user=$1 + local queries_count=$2 + # 10 seconds waiting + counter=0 retries=100 + while [[ $counter -lt $retries ]]; do + result=$($CLICKHOUSE_CLIENT --query "SELECT COUNT(*) FROM system.processes WHERE user = '${user}'") + if [[ $result == "${queries_count}" ]]; then + break; + fi + sleep 0.1 + ((++counter)) + done +} + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" +${CLICKHOUSE_CLIENT} -q "DELETE FROM system.session_log WHERE user = '${TEST_USER}'" + +# DROP USE CASE +${CLICKHOUSE_CLIENT} -q "CREATE USER IF NOT EXISTS ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" + +export -f tcp_session; +export -f http_session; +export -f http_with_session_id_session; + +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +# DROP ROLE CASE +${CLICKHOUSE_CLIENT} -q "CREATE ROLE IF NOT EXISTS ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} DEFAULT ROLE ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" + +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP ROLE ${TEST_ROLE}" +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +# DROP PROFILE CASE +${CLICKHOUSE_CLIENT} -q "CREATE SETTINGS PROFILE IF NOT EXISTS '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "CREATE USER ${TEST_USER} SETTINGS PROFILE '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "GRANT SELECT ON system.numbers TO ${TEST_USER}" + +timeout 10s bash -c "tcp_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_session ${TEST_USER}" >/dev/null 2>&1 & +timeout 10s bash -c "http_with_session_id_session ${TEST_USER}" >/dev/null 2>&1 & + +wait_for_queries_start $TEST_USER 3 +${CLICKHOUSE_CLIENT} -q "DROP SETTINGS PROFILE '${TEST_PROFILE}'" +${CLICKHOUSE_CLIENT} -q "DROP USER ${TEST_USER}" + +${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE user = '${TEST_USER}' SYNC" >/dev/null & + +wait + +${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS" + +echo "port_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_port = 0" +echo "address_0_sessions:" +${CLICKHOUSE_CLIENT} -q "SELECT count(*) FROM system.session_log WHERE user = '${TEST_USER}' AND client_address = toIPv6('::')" +echo "Corresponding LoginSuccess/Logout" +${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM (SELECT ${SESSION_LOG_MATCHING_FIELDS} FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginSuccess' INTERSECT SELECT ${SESSION_LOG_MATCHING_FIELDS}, FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'Logout')" +echo "LoginFailure" +${CLICKHOUSE_CLIENT} -q "SELECT COUNT(*) FROM system.session_log WHERE user = '${TEST_USER}' AND type = 'LoginFailure'" From f95c8599594c3a8e7101a02a4e376f5cb6ca7b8a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 15 Aug 2023 15:47:11 +0200 Subject: [PATCH 576/777] rename setting --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- tests/integration/test_replicated_database/configs/config.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f1c01d1aadc..57189012317 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1337,7 +1337,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, !getContext()->getZooKeeperMetadataTransaction()->isInitialQuery() && !DatabaseCatalog::instance().hasUUIDMapping(create.uuid) && Context::getGlobalContextInstance()->isServerCompletelyStarted() && - Context::getGlobalContextInstance()->getConfigRef().getBool("allow_moving_table_dir_to_trash", false)) + Context::getGlobalContextInstance()->getConfigRef().getBool("allow_moving_table_directory_to_trash", false)) { /// This is a secondary query from a Replicated database. It cannot be retried with another UUID, we must execute it as is. /// We don't have a table with this UUID (and all metadata is loaded), diff --git a/tests/integration/test_replicated_database/configs/config.xml b/tests/integration/test_replicated_database/configs/config.xml index 7a67d69c031..e598cc28d5d 100644 --- a/tests/integration/test_replicated_database/configs/config.xml +++ b/tests/integration/test_replicated_database/configs/config.xml @@ -1,6 +1,6 @@ 10 - 1 + 1 10 From 14977e60722b2efb76de402eba775c19843137a8 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Tue, 15 Aug 2023 13:54:28 +0000 Subject: [PATCH 577/777] Eliminate duplicate include --- src/Common/TransformEndianness.hpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Common/TransformEndianness.hpp b/src/Common/TransformEndianness.hpp index 7c77e918199..1657305acda 100644 --- a/src/Common/TransformEndianness.hpp +++ b/src/Common/TransformEndianness.hpp @@ -6,8 +6,6 @@ #include -#include - #include namespace DB From 8343ebd4ecfc90e4fafb22d660fd0ad04aaa0e74 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 15 Aug 2023 14:44:13 +0000 Subject: [PATCH 578/777] Clarify comment --- programs/local/LocalServer.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 0dea7e8e643..b38e17ecade 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -710,7 +710,8 @@ void LocalServer::processConfig() if (mmap_cache_size) global_context->setMMappedFileCache(mmap_cache_size); - /// not initializing the query cache in clickhouse-local + /// In Server.cpp (./clickhouse-server), we would initialize the query cache here. + /// Intentionally not doing this in clickhouse-local as it doesn't make sense. #if USE_EMBEDDED_COMPILER size_t compiled_expression_cache_max_size_in_bytes = config().getUInt64("compiled_expression_cache_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE); From 9825f8c76eec8999d0aa4509ab4604d7f45122f2 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 15 Aug 2023 17:03:32 +0000 Subject: [PATCH 579/777] Remove debug logging --- src/Interpreters/Cluster.h | 2 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 11 +++++++---- src/Storages/StorageReplicatedMergeTree.cpp | 9 ++++++--- 4 files changed, 15 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index f509a2d9847..cb75487cbbc 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -273,7 +273,7 @@ public: /// Are distributed DDL Queries (ON CLUSTER Clause) allowed for this cluster bool areDistributedDDLQueriesAllowed() const { return allow_distributed_ddl_queries; } - String getName() const { return name; } + const String & getName() const { return name; } private: SlotToShard slot_to_shard; diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index cbcaca3e971..bf9a87a692e 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -239,7 +239,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact if (context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS) { - LOG_DEBUG(&Poco::Logger::get(__FUNCTION__), "_cluster_for_parallel_replicas: {}", cluster_name); + /// the cluster is defined by Distributed table and passed to shards via `_cluster_for_parallel_replicas` scalar value scalars["_cluster_for_parallel_replicas"] = Block{{DataTypeString().createColumnConst(1, cluster_name), std::make_shared(), "_cluster_for_parallel_replicas"}}; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 3a0d485e5c4..59d86cf3d1e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -220,17 +220,20 @@ void StorageMergeTree::read( local_context, query_info.query, table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); - auto scalars = local_context->hasQueryContext() ? local_context->getQueryContext()->getScalars() : Scalars{}; String cluster_for_parallel_replicas = local_context->getSettingsRef().cluster_for_parallel_replicas; { - auto it = scalars.find("_cluster_for_parallel_replicas"); + /// if parallel replicas query executed over Distributed table, + /// the cluster is defined by Distributed table and passed to shards via `_cluster_for_parallel_replicas` scalar value + const auto scalars = local_context->hasQueryContext() ? local_context->getQueryContext()->getScalars() : Scalars{}; + const auto it = scalars.find("_cluster_for_parallel_replicas"); if (it != scalars.end()) { const Block & block = it->second; - cluster_for_parallel_replicas = block.getColumns()[0]->getDataAt(0).toString(); + chassert(block.columns() == 1); + const auto & column = block.getByPosition(0).column; + cluster_for_parallel_replicas = column->getDataAt(0).toString(); } } - LOG_DEBUG(&Poco::Logger::get("StorageMergeTree::read"), "_cluster_for_parallel_replicas: {}", cluster_for_parallel_replicas); auto cluster = local_context->getCluster(cluster_for_parallel_replicas); Block header; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f91d0b49928..60d5d26490b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5156,14 +5156,17 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl( auto scalars = local_context->hasQueryContext() ? local_context->getQueryContext()->getScalars() : Scalars{}; String cluster_for_parallel_replicas = local_context->getSettingsRef().cluster_for_parallel_replicas; { - auto it = scalars.find("_cluster_for_parallel_replicas"); + /// if parallel replicas query executed over Distributed table, + /// the cluster is defined by Distributed table and passed to shards via `_cluster_for_parallel_replicas` scalar value + const auto it = scalars.find("_cluster_for_parallel_replicas"); if (it != scalars.end()) { const Block & block = it->second; - cluster_for_parallel_replicas = block.getColumns()[0]->getDataAt(0).toString(); + chassert(block.columns() == 1); + const auto & column = block.getByPosition(0).column; + cluster_for_parallel_replicas = column->getDataAt(0).toString(); } } - LOG_DEBUG(&Poco::Logger::get(__FUNCTION__), "_cluster_for_parallel_replicas: {}", cluster_for_parallel_replicas); auto parallel_replicas_cluster = local_context->getCluster(cluster_for_parallel_replicas); ASTPtr modified_query_ast; From f1e040447f13aaff03d8aee5b00b0d486be73088 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 15 Aug 2023 17:14:36 +0000 Subject: [PATCH 580/777] Tests: there is no need for `cluster_for_parallel_replicas` setting in queries over distributed anymore --- .../test_parallel_replicas_over_distributed/test.py | 5 ++--- .../02835_parallel_replicas_over_distributed.sql | 8 ++++---- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_parallel_replicas_over_distributed/test.py b/tests/integration/test_parallel_replicas_over_distributed/test.py index ee52c8c040e..a0f9d2b71ac 100644 --- a/tests/integration/test_parallel_replicas_over_distributed/test.py +++ b/tests/integration/test_parallel_replicas_over_distributed/test.py @@ -88,7 +88,7 @@ def create_tables(cluster, table_name): settings={"insert_distributed_sync": 1}, ) nodes[0].query( - f"INSERT INTO {table_name}_d SELECT number, number FROM numbers(1)", + f"INSERT INTO {table_name}_d SELECT number, number FROM numbers(3)", settings={"insert_distributed_sync": 1}, ) @@ -102,7 +102,7 @@ def test_parallel_replicas_over_distributed(start_cluster, cluster): create_tables(cluster, table_name) node = nodes[0] - expected_result = f"6001\t-1999\t1999\t0\n" + expected_result = f"6003\t-1999\t1999\t3\n" # w/o parallel replicas assert ( @@ -119,7 +119,6 @@ def test_parallel_replicas_over_distributed(start_cluster, cluster): "prefer_localhost_replica": 0, "max_parallel_replicas": 4, "use_hedged_requests": 0, - # "cluster_for_parallel_replicas": cluster, }, ) == expected_result diff --git a/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql index 30dfb32678d..c2fc1d8355d 100644 --- a/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql +++ b/tests/queries/0_stateless/02835_parallel_replicas_over_distributed.sql @@ -14,13 +14,13 @@ insert into test select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; --, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; insert into test select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; --, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; -- 2 shards @@ -38,10 +38,10 @@ insert into test2 select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test2_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; --, cluster_for_parallel_replicas = 'test_cluster_two_shard_three_replicas_localhost'; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; insert into test2 select *, today() from numbers(100); SELECT count(), min(id), max(id), avg(id) FROM test2_d -SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; --, cluster_for_parallel_replicas = 'test_cluster_two_shard_three_replicas_localhost'; +SETTINGS allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, use_hedged_requests=0; From 49134711dc7f98137cb95f42cbe072da038b3e80 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 15 Aug 2023 17:37:01 +0000 Subject: [PATCH 581/777] Polishing --- src/Interpreters/ClusterProxy/executeQuery.cpp | 6 +++--- src/Storages/StorageMergeTree.cpp | 3 +-- src/Storages/StorageReplicatedMergeTree.cpp | 3 +-- 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 0dcae6a7d65..a42ec25cda8 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -276,11 +276,12 @@ void executeQueryWithParallelReplicas( auto scalars = new_context->hasQueryContext() ? new_context->getQueryContext()->getScalars() : Scalars{}; UInt64 shard_num = 0; /// shard_num is 1-based, so 0 - no shard specified - auto it = scalars.find("_shard_num"); + const auto it = scalars.find("_shard_num"); if (it != scalars.end()) { const Block & block = it->second; - shard_num = block.getColumns()[0]->get64(0); + const auto & column = block.safeGetByPosition(0).column; + shard_num = column->getUInt(0); } ClusterPtr new_cluster; @@ -297,7 +298,6 @@ void executeQueryWithParallelReplicas( auto all_replicas_count = std::min(static_cast(settings.max_parallel_replicas), new_cluster->getShardCount()); auto coordinator = std::make_shared(all_replicas_count); - auto remote_plan = std::make_unique(); /// This is a little bit weird, but we construct an "empty" coordinator without /// any specified reading/coordination method (like Default, InOrder, InReverseOrder) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 59d86cf3d1e..9d46b0a07aa 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -229,8 +229,7 @@ void StorageMergeTree::read( if (it != scalars.end()) { const Block & block = it->second; - chassert(block.columns() == 1); - const auto & column = block.getByPosition(0).column; + const auto & column = block.safeGetByPosition(0).column; cluster_for_parallel_replicas = column->getDataAt(0).toString(); } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 60d5d26490b..ba0ce4b76a1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5162,8 +5162,7 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl( if (it != scalars.end()) { const Block & block = it->second; - chassert(block.columns() == 1); - const auto & column = block.getByPosition(0).column; + const auto & column = block.safeGetByPosition(0).column; cluster_for_parallel_replicas = column->getDataAt(0).toString(); } } From cbe4c8adc2973ee8d6583f178bd44915c55f21f1 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 14 Aug 2023 23:24:41 +0000 Subject: [PATCH 582/777] Fix more functions with 'Context has expired' error --- src/Functions/FunctionFactory.h | 4 +-- src/Functions/FunctionsExternalDictionaries.h | 9 +++--- src/Functions/FunctionsJSON.h | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 8 ++--- .../QueryPlan/Optimizations/Optimizations.h | 4 +-- .../Optimizations/addPlansForSets.cpp | 6 ++-- .../QueryPlan/Optimizations/optimizeTree.cpp | 4 +-- src/Processors/QueryPlan/QueryPlan.cpp | 6 ++-- src/Processors/QueryPlan/QueryPlan.h | 2 +- .../02843_context_has_expired.reference | 4 +++ .../0_stateless/02843_context_has_expired.sql | 29 ++++++++++++++----- 11 files changed, 48 insertions(+), 30 deletions(-) diff --git a/src/Functions/FunctionFactory.h b/src/Functions/FunctionFactory.h index deea41e6677..588cae64e16 100644 --- a/src/Functions/FunctionFactory.h +++ b/src/Functions/FunctionFactory.h @@ -20,8 +20,8 @@ using FunctionCreator = std::function; using FunctionFactoryData = std::pair; /** Creates function by name. - * Function could use for initialization (take ownership of shared_ptr, for example) - * some dictionaries from Context. + * The provided Context is guaranteed to outlive the created function. Functions may use it for + * things like settings, current database, permission checks, etc. */ class FunctionFactory : private boost::noncopyable, public IFactoryWithAliases { diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 1b2e2eb3bd6..db6529da73c 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -62,13 +62,14 @@ namespace ErrorCodes */ -class FunctionDictHelper +class FunctionDictHelper : WithContext { public: - explicit FunctionDictHelper(ContextPtr context_) : current_context(context_) {} + explicit FunctionDictHelper(ContextPtr context_) : WithContext(context_) {} std::shared_ptr getDictionary(const String & dictionary_name) { + auto current_context = getContext(); auto dict = current_context->getExternalDictionariesLoader().getDictionary(dictionary_name, current_context); if (!access_checked) @@ -131,12 +132,10 @@ public: DictionaryStructure getDictionaryStructure(const String & dictionary_name) const { - return current_context->getExternalDictionariesLoader().getDictionaryStructure(dictionary_name, current_context); + return getContext()->getExternalDictionariesLoader().getDictionaryStructure(dictionary_name, getContext()); } private: - ContextPtr current_context; - /// Access cannot be not granted, since in this case checkAccess() will throw and access_checked will not be updated. std::atomic access_checked = false; diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index ca797eed856..094de0c27c2 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -336,7 +336,7 @@ private: template typename Impl> -class ExecutableFunctionJSON : public IExecutableFunction, WithContext +class ExecutableFunctionJSON : public IExecutableFunction { public: diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 3a381cd8dab..39cc4df5c2d 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -541,13 +541,13 @@ QueryPipeline InterpreterExplainQuery::executeImpl() InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); interpreter.buildQueryPlan(plan); context = interpreter.getContext(); - // collect the selected marks, rows, parts during build query pipeline. - plan.buildQueryPipeline( + // Collect the selected marks, rows, parts during build query pipeline. + // Hold on to the returned QueryPipelineBuilderPtr because `plan` may have pointers into + // it (through QueryPlanResourceHolder). + auto builder = plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - if (settings.optimize) - plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); plan.explainEstimate(res_columns); insert_buf = false; break; diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 6ecec1359c5..2230e50425c 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -16,7 +16,7 @@ void optimizeTreeFirstPass(const QueryPlanOptimizationSettings & settings, Query void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes); /// Third pass is used to apply filters such as key conditions and skip indexes to the storages that support them. /// After that it add CreateSetsStep for the subqueries that has not be used in the filters. -void optimizeTreeThirdPass(QueryPlan::Node & root, QueryPlan::Nodes & nodes); +void optimizeTreeThirdPass(QueryPlan & plan, QueryPlan::Node & root, QueryPlan::Nodes & nodes); /// Optimization (first pass) is a function applied to QueryPlan::Node. /// It can read and update subtree of specified node. @@ -113,7 +113,7 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes, bool allow_implicit_projections); bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes); -bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes); +bool addPlansForSets(QueryPlan & plan, QueryPlan::Node & node, QueryPlan::Nodes & nodes); /// Enable memory bound merging of aggregation states for remote queries /// in case it was enabled for local plan diff --git a/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp b/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp index e9100ae9d02..47df05301c9 100644 --- a/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp +++ b/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp @@ -6,7 +6,7 @@ namespace DB::QueryPlanOptimizations { -bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes) +bool addPlansForSets(QueryPlan & root_plan, QueryPlan::Node & node, QueryPlan::Nodes & nodes) { auto * delayed = typeid_cast(node.step.get()); if (!delayed) @@ -23,7 +23,9 @@ bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { input_streams.push_back(plan->getCurrentDataStream()); node.children.push_back(plan->getRootNode()); - nodes.splice(nodes.end(), QueryPlan::detachNodes(std::move(*plan))); + auto [add_nodes, add_resources] = QueryPlan::detachNodesAndResources(std::move(*plan)); + nodes.splice(nodes.end(), std::move(add_nodes)); + root_plan.addResources(std::move(add_resources)); } auto creating_sets = std::make_unique(std::move(input_streams)); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index b13dda9a8f0..0caedff67a5 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -181,7 +181,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s "No projection is used when optimize_use_projections = 1 and force_optimize_projection = 1"); } -void optimizeTreeThirdPass(QueryPlan::Node & root, QueryPlan::Nodes & nodes) +void optimizeTreeThirdPass(QueryPlan & plan, QueryPlan::Node & root, QueryPlan::Nodes & nodes) { Stack stack; stack.push_back({.node = &root}); @@ -205,7 +205,7 @@ void optimizeTreeThirdPass(QueryPlan::Node & root, QueryPlan::Nodes & nodes) source_step_with_filter->applyFilters(); } - addPlansForSets(*frame.node, nodes); + addPlansForSets(plan, *frame.node, nodes); stack.pop_back(); } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 687260441ff..ceda9f97bab 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -482,7 +482,7 @@ void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_sett QueryPlanOptimizations::optimizeTreeFirstPass(optimization_settings, *root, nodes); QueryPlanOptimizations::optimizeTreeSecondPass(optimization_settings, *root, nodes); - QueryPlanOptimizations::optimizeTreeThirdPass(*root, nodes); + QueryPlanOptimizations::optimizeTreeThirdPass(*this, *root, nodes); updateDataStreams(*root); } @@ -542,9 +542,9 @@ void QueryPlan::explainEstimate(MutableColumns & columns) } } -QueryPlan::Nodes QueryPlan::detachNodes(QueryPlan && plan) +std::pair QueryPlan::detachNodesAndResources(QueryPlan && plan) { - return std::move(plan.nodes); + return {std::move(plan.nodes), std::move(plan.resources)}; } } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index d89bdc534be..f4a6c9097f2 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -108,7 +108,7 @@ public: using Nodes = std::list; Node * getRootNode() const { return root; } - static Nodes detachNodes(QueryPlan && plan); + static std::pair detachNodesAndResources(QueryPlan && plan); private: QueryPlanResourceHolder resources; diff --git a/tests/queries/0_stateless/02843_context_has_expired.reference b/tests/queries/0_stateless/02843_context_has_expired.reference index 573541ac970..229972f2924 100644 --- a/tests/queries/0_stateless/02843_context_has_expired.reference +++ b/tests/queries/0_stateless/02843_context_has_expired.reference @@ -1 +1,5 @@ 0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/02843_context_has_expired.sql b/tests/queries/0_stateless/02843_context_has_expired.sql index ccef3458ed7..8355ce2c18c 100644 --- a/tests/queries/0_stateless/02843_context_has_expired.sql +++ b/tests/queries/0_stateless/02843_context_has_expired.sql @@ -1,23 +1,36 @@ -DROP DICTIONARY IF EXISTS dict; -DROP TABLE IF EXISTS source; +DROP DICTIONARY IF EXISTS 02843_dict; +DROP TABLE IF EXISTS 02843_source; +DROP TABLE IF EXISTS 02843_join; -CREATE TABLE source +CREATE TABLE 02843_source ( id UInt64, value String ) ENGINE=Memory; -CREATE DICTIONARY dict +CREATE DICTIONARY 02843_dict ( id UInt64, value String ) PRIMARY KEY id -SOURCE(CLICKHOUSE(TABLE 'source')) +SOURCE(CLICKHOUSE(TABLE '02843_source')) LAYOUT(DIRECT()); -SELECT 1 IN (SELECT dictGet('dict', 'value', materialize('1'))); +SELECT 1 IN (SELECT dictGet('02843_dict', 'value', materialize('1'))); -DROP DICTIONARY dict; -DROP TABLE source; +CREATE TABLE 02843_join (id UInt8, value String) ENGINE Join(ANY, LEFT, id); +SELECT 1 IN (SELECT joinGet(02843_join, 'value', materialize(1))); +SELECT 1 IN (SELECT joinGetOrNull(02843_join, 'value', materialize(1))); + +SELECT 1 IN (SELECT materialize(connectionId())); +SELECT 1000000 IN (SELECT materialize(getSetting('max_threads'))); +SELECT 1 in (SELECT file(materialize('a'))); -- { serverError 107 } + +EXPLAIN ESTIMATE SELECT 1 IN (SELECT dictGet('02843_dict', 'value', materialize('1'))); +EXPLAIN ESTIMATE SELECT 1 IN (SELECT joinGet(`02843_join`, 'value', materialize(1))); + +DROP DICTIONARY 02843_dict; +DROP TABLE 02843_source; +DROP TABLE 02843_join; From 3fc65a68c7cd6aabea5250784ee068014ffaa801 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 15 Aug 2023 20:37:39 +0200 Subject: [PATCH 583/777] more parallel tests --- src/Client/ClientBase.cpp | 2 +- src/Interpreters/InterpreterUseQuery.cpp | 2 +- .../ReplaceQueryParameterVisitor.cpp | 6 + src/Parsers/ASTUseQuery.h | 26 +- src/Parsers/ParserCreateQuery.cpp | 24 +- src/Parsers/ParserUseQuery.cpp | 4 +- tests/clickhouse-test | 33 +- .../00080_show_tables_and_system_tables.sql | 12 +- ...s_and_insert_without_explicit_database.sql | 13 +- ...t_database_when_create_materializ_view.sql | 8 +- .../00604_show_create_database.sql | 4 +- ...12_http_max_query_size_for_distributed.sql | 2 - ...3_max_block_size_system_tables_columns.sql | 1 + ...51_default_databasename_for_view.reference | 8 +- .../00751_default_databasename_for_view.sql | 31 +- .../00800_versatile_storage_join.sql | 29 +- .../00857_global_joinsavel_table_alias.sql | 6 +- .../0_stateless/01012_show_tables_limit.sql | 25 +- .../01018_ddl_dictionaries_select.reference | 6 +- .../01018_ddl_dictionaries_select.sql | 99 +- .../01018_ddl_dictionaries_special.sql | 66 +- .../01018_dictionaries_from_dictionaries.sql | 59 +- .../0_stateless/01018_ip_dictionary_long.sql | 1020 ++++++++--------- .../01033_dictionaries_lifetime.sql | 29 +- ...rfluous_dict_reload_on_create_database.sql | 38 +- .../01041_create_dictionary_if_not_exists.sql | 26 +- ...dictionary_attribute_properties_values.sql | 22 +- .../01045_dictionaries_restrictions.sql | 12 +- ...7_window_view_parser_inner_table.reference | 32 +- .../01047_window_view_parser_inner_table.sql | 139 ++- ...0_clickhouse_dict_source_with_subquery.sql | 23 +- .../01053_drop_database_mat_view.sql | 17 +- .../0_stateless/01056_create_table_as.sql | 13 +- ...ndary_index_with_old_format_merge_tree.sql | 1 - .../01073_attach_if_not_exists.sql | 1 - .../01073_show_tables_not_like.sql | 17 +- ..._error_incorrect_size_of_nested_column.sql | 41 +- .../0_stateless/01084_regexp_empty.sql | 9 +- .../0_stateless/01085_window_view_attach.sql | 33 +- 39 files changed, 944 insertions(+), 995 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 9ad6a46866f..9092f12b62b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1811,7 +1811,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin } if (const auto * use_query = parsed_query->as()) { - const String & new_database = use_query->database; + const String & new_database = use_query->getDatabase(); /// If the client initiates the reconnection, it takes the settings from the config. config().setString("database", new_database); /// If the connection initiates the reconnection, it uses its variable. diff --git a/src/Interpreters/InterpreterUseQuery.cpp b/src/Interpreters/InterpreterUseQuery.cpp index d8a5ae57470..b71f3a9cc1c 100644 --- a/src/Interpreters/InterpreterUseQuery.cpp +++ b/src/Interpreters/InterpreterUseQuery.cpp @@ -10,7 +10,7 @@ namespace DB BlockIO InterpreterUseQuery::execute() { - const String & new_database = query_ptr->as().database; + const String & new_database = query_ptr->as().getDatabase(); getContext()->checkAccess(AccessType::SHOW_DATABASES, new_database); getContext()->getSessionContext()->setCurrentDatabase(new_database); return {}; diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index e4a052388ac..89a40baaff9 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -131,6 +131,7 @@ void ReplaceQueryParameterVisitor::visitIdentifier(ASTPtr & ast) if (ast_identifier->children.empty()) return; + bool replaced_parameter = false; auto & name_parts = ast_identifier->name_parts; for (size_t i = 0, j = 0, size = name_parts.size(); i < size; ++i) { @@ -138,9 +139,14 @@ void ReplaceQueryParameterVisitor::visitIdentifier(ASTPtr & ast) { const auto & ast_param = ast_identifier->children[j++]->as(); name_parts[i] = getParamValue(ast_param.name); + replaced_parameter = true; } } + /// Do not touch AST if there are no parameters + if (!replaced_parameter) + return; + /// FIXME: what should this mean? if (!ast_identifier->semantic->special && name_parts.size() >= 2) ast_identifier->semantic->table = ast_identifier->name_parts.end()[-2]; diff --git a/src/Parsers/ASTUseQuery.h b/src/Parsers/ASTUseQuery.h index f767a6bbdb7..873a316e653 100644 --- a/src/Parsers/ASTUseQuery.h +++ b/src/Parsers/ASTUseQuery.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -14,19 +15,34 @@ namespace DB class ASTUseQuery : public IAST { public: - String database; + IAST * database; + + String getDatabase() const + { + String name; + tryGetIdentifierNameInto(database, name); + return name; + } /** Get the text that identifies this element. */ - String getID(char delim) const override { return "UseQuery" + (delim + database); } + String getID(char delim) const override { return "UseQuery" + (delim + getDatabase()); } - ASTPtr clone() const override { return std::make_shared(*this); } + ASTPtr clone() const override + { + auto res = std::make_shared(*this); + res->children.clear(); + if (database) + res->set(res->database, database->clone()); + return res; + } QueryKind getQueryKind() const override { return QueryKind::Use; } protected: - void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "USE " << (settings.hilite ? hilite_none : "") << backQuoteIfNeed(database); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "USE " << (settings.hilite ? hilite_none : ""); + database->formatImpl(settings, state, frame); } }; diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 82674ab1a35..fb6dae248c0 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -502,7 +502,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ParserKeyword s_temporary("TEMPORARY"); ParserKeyword s_table("TABLE"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); - ParserCompoundIdentifier table_name_p(true, true); + ParserCompoundIdentifier table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true); ParserKeyword s_from("FROM"); ParserKeyword s_on("ON"); ParserToken s_dot(TokenType::Dot); @@ -740,7 +740,7 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e ParserKeyword s_create("CREATE"); ParserKeyword s_attach("ATTACH"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); - ParserCompoundIdentifier table_name_p(true, true); + ParserCompoundIdentifier table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true); ParserKeyword s_as("AS"); ParserKeyword s_view("VIEW"); ParserKeyword s_live("LIVE"); @@ -878,7 +878,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ParserKeyword s_temporary("TEMPORARY"); ParserKeyword s_attach("ATTACH"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); - ParserCompoundIdentifier table_name_p(true); + ParserCompoundIdentifier table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true); ParserKeyword s_as("AS"); ParserKeyword s_view("VIEW"); ParserKeyword s_window("WINDOW"); @@ -1015,12 +1015,17 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & query->if_not_exists = if_not_exists; query->is_window_view = true; - StorageID table_id = table->as()->getTableId(); - query->setDatabase(table_id.database_name); - query->setTable(table_id.table_name); - query->uuid = table_id.uuid; + auto * table_id = table->as(); + query->database = table_id->getDatabase(); + query->table = table_id->getTable(); + query->uuid = table_id->uuid; query->cluster = cluster_str; + if (query->database) + query->children.push_back(query->database); + if (query->table) + query->children.push_back(query->table); + if (to_table) query->to_table_id = to_table->as()->getTableId(); @@ -1265,7 +1270,8 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserKeyword s_create("CREATE"); ParserKeyword s_attach("ATTACH"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); - ParserCompoundIdentifier table_name_p(true, true); + ParserCompoundIdentifier table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true); + ParserCompoundIdentifier to_table_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ false); ParserKeyword s_as("AS"); ParserKeyword s_view("VIEW"); ParserKeyword s_materialized("MATERIALIZED"); @@ -1487,7 +1493,7 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E ParserKeyword s_dictionary("DICTIONARY"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); ParserKeyword s_on("ON"); - ParserCompoundIdentifier dict_name_p(true, true); + ParserCompoundIdentifier dict_name_p(/*table_name_with_optional_uuid*/ true, /*allow_query_parameter*/ true); ParserToken s_left_paren(TokenType::OpeningRoundBracket); ParserToken s_right_paren(TokenType::ClosingRoundBracket); ParserToken s_dot(TokenType::Dot); diff --git a/src/Parsers/ParserUseQuery.cpp b/src/Parsers/ParserUseQuery.cpp index 40511d8c142..350e573281c 100644 --- a/src/Parsers/ParserUseQuery.cpp +++ b/src/Parsers/ParserUseQuery.cpp @@ -11,7 +11,7 @@ namespace DB bool ParserUseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_use("USE"); - ParserIdentifier name_p; + ParserIdentifier name_p{/*allow_query_parameter*/ true}; if (!s_use.ignore(pos, expected)) return false; @@ -21,7 +21,7 @@ bool ParserUseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; auto query = std::make_shared(); - tryGetIdentifierNameInto(database, query->database); + query->set(query->database, database); node = query; return true; diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 20c769bd701..7057f3df90f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1196,8 +1196,18 @@ class TestCase: # This is for .sh tests os.environ["CLICKHOUSE_LOG_COMMENT"] = args.testcase_basename + query_params = "" + if "need-query-parameters" in self.tags: + query_params = ( + " --param_CLICKHOUSE_DATABASE=" + + database + + " --param_CLICKHOUSE_DATABASE_1=" + + database + + "_1" + ) + params = { - "client": client + " --database=" + database + " --param_CLICKHOUSE_DATABASE=" + database, + "client": client + " --database=" + database + query_params, "logs_level": server_logs_level, "options": client_options, "test": self.case_file, @@ -1457,13 +1467,13 @@ class TestSuite: else: raise Exception(f"Unknown file_extension: {filename}") - def parse_tags_from_line(line, comment_sign): + def parse_tags_from_line(line, comment_sign) -> set[str]: if not line.startswith(comment_sign): - return None + return set() tags_str = line[len(comment_sign) :].lstrip() # noqa: ignore E203 tags_prefix = "Tags:" if not tags_str.startswith(tags_prefix): - return None + return set() tags_str = tags_str[len(tags_prefix) :] # noqa: ignore E203 tags = tags_str.split(",") tags = {tag.strip() for tag in tags} @@ -1481,12 +1491,23 @@ class TestSuite: def load_tags_from_file(filepath): comment_sign = get_comment_sign(filepath) + need_query_params = False with open(filepath, "r", encoding="utf-8") as file: try: - line = find_tag_line(file) + tag_line = find_tag_line(file) except UnicodeDecodeError: return [] - return parse_tags_from_line(line, comment_sign) + try: + if filepath.endswith(".sql"): + for line in file: + if "{CLICKHOUSE_DATABASE" in line: + need_query_params = True + except UnicodeDecodeError: + pass + parsed_tags = parse_tags_from_line(tag_line, comment_sign) + if need_query_params: + parsed_tags.add("need-query-parameters") + return parsed_tags all_tags = {} start_time = datetime.now() diff --git a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql index 40c26be31c0..a58f9ddb0ac 100644 --- a/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql +++ b/tests/queries/0_stateless/00080_show_tables_and_system_tables.sql @@ -16,16 +16,12 @@ SELECT name FROM system.tables WHERE is_temporary = 1 AND name = 'test_temporary CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test_log(id UInt64) ENGINE = Log; CREATE MATERIALIZED VIEW {CLICKHOUSE_DATABASE:Identifier}.test_materialized ENGINE = Log AS SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_log; -SELECT dependencies_database, dependencies_table FROM system.tables WHERE name = 'test_log'; +SELECT dependencies_database, dependencies_table FROM system.tables WHERE name = 'test_log' and database=currentDatabase(); DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; - -- Check that create_table_query works for system tables and unusual Databases -DROP DATABASE IF EXISTS test_DatabaseMemory; -CREATE DATABASE test_DatabaseMemory ENGINE = Memory; -CREATE TABLE test_DatabaseMemory.A (A UInt8) ENGINE = Null; +CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier} ENGINE = Memory; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.A (A UInt8) ENGINE = Null; -SELECT sum(ignore(*, metadata_modification_time, engine_full, create_table_query)) FROM system.tables WHERE database = 'test_DatabaseMemory'; - -DROP DATABASE test_DatabaseMemory; +SELECT sum(ignore(*, metadata_modification_time, engine_full, create_table_query)) FROM system.tables WHERE database = '{CLICKHOUSE_DATABASE:String}'; diff --git a/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql b/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql index 95d46032601..7d925bc4ff1 100644 --- a/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql +++ b/tests/queries/0_stateless/00101_materialized_views_and_insert_without_explicit_database.sql @@ -1,8 +1,3 @@ --- Tags: no-parallel - -CREATE DATABASE IF NOT EXISTS test_00101_0; - -USE test_00101_0; DROP TABLE IF EXISTS test_table; DROP TABLE IF EXISTS test_view; @@ -25,9 +20,9 @@ DROP TABLE test_view_filtered; -- Check only sophisticated constructors and desctructors: -CREATE DATABASE IF NOT EXISTS test_00101_1; +CREATE DATABASE IF NOT EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; -USE test_00101_1; +USE {CLICKHOUSE_DATABASE_1:Identifier}; DROP TABLE IF EXISTS tmp; DROP TABLE IF EXISTS tmp_mv; @@ -57,5 +52,5 @@ EXISTS TABLE `.inner.tmp_mv4`; DROP TABLE tmp; -DROP DATABASE test_00101_0; -DROP DATABASE test_00101_1; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; +DROP DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; diff --git a/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql b/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql index 87f2e2b5276..d24a57187b4 100644 --- a/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql +++ b/tests/queries/0_stateless/00571_non_exist_database_when_create_materializ_view.sql @@ -1,14 +1,14 @@ -DROP DATABASE IF EXISTS none; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; DROP TABLE IF EXISTS test_materialized_00571; set allow_deprecated_syntax_for_merge_tree=1; -CREATE DATABASE none; +CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; CREATE TABLE test_00571 ( date Date, platform Enum8('a' = 0, 'b' = 1, 'c' = 2), app Enum8('a' = 0, 'b' = 1) ) ENGINE = MergeTree(date, (platform, app), 8192); CREATE MATERIALIZED VIEW test_materialized_00571 ENGINE = MergeTree(date, (platform, app), 8192) POPULATE AS SELECT date, platform, app FROM (SELECT * FROM test_00571); -USE none; +USE {CLICKHOUSE_DATABASE_1:Identifier}; INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.test_00571 VALUES('2018-02-16', 'a', 'a'); @@ -20,6 +20,6 @@ ATTACH TABLE {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; -DROP DATABASE IF EXISTS none; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.test_00571; DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.test_materialized_00571; diff --git a/tests/queries/0_stateless/00604_show_create_database.sql b/tests/queries/0_stateless/00604_show_create_database.sql index d20d2cb3e53..23ebd23c9f1 100644 --- a/tests/queries/0_stateless/00604_show_create_database.sql +++ b/tests/queries/0_stateless/00604_show_create_database.sql @@ -1,5 +1,3 @@ --- Tags: no-ordinary-database +-- Tags: no-ordinary-database, no-replicated-database -create database if not exists {CLICKHOUSE_DATABASE:Identifier}; show create database {CLICKHOUSE_DATABASE:Identifier}; -drop database {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/00612_http_max_query_size_for_distributed.sql b/tests/queries/0_stateless/00612_http_max_query_size_for_distributed.sql index 1802fadc57b..462fd0fef61 100644 --- a/tests/queries/0_stateless/00612_http_max_query_size_for_distributed.sql +++ b/tests/queries/0_stateless/00612_http_max_query_size_for_distributed.sql @@ -1,5 +1,3 @@ --- Tags: no-parallel - DROP TABLE IF EXISTS data_00612; DROP TABLE IF EXISTS dist_00612; diff --git a/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql b/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql index fe1876ed6f8..0c8e4ae237d 100644 --- a/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql +++ b/tests/queries/0_stateless/00693_max_block_size_system_tables_columns.sql @@ -1,3 +1,4 @@ +-- Tags: no-parallel -- NOTE: database = currentDatabase() is not mandatory diff --git a/tests/queries/0_stateless/00751_default_databasename_for_view.reference b/tests/queries/0_stateless/00751_default_databasename_for_view.reference index b3f1875ae91..4899e230924 100644 --- a/tests/queries/0_stateless/00751_default_databasename_for_view.reference +++ b/tests/queries/0_stateless/00751_default_databasename_for_view.reference @@ -1,4 +1,4 @@ -CREATE MATERIALIZED VIEW test_00751.t_mv_00751 +CREATE MATERIALIZED VIEW default.t_mv_00751 ( `date` Date, `platform` Enum8('a' = 0, 'b' = 1), @@ -11,14 +11,14 @@ SELECT date, platform, app -FROM test_00751.t_00751 +FROM default.t_00751 WHERE (app = ( SELECT min(app) - FROM test_00751.u_00751 + FROM default.u_00751 )) AND (platform = ( SELECT ( SELECT min(platform) - FROM test_00751.v_00751 + FROM default.v_00751 ) )) 2000-01-01 a a diff --git a/tests/queries/0_stateless/00751_default_databasename_for_view.sql b/tests/queries/0_stateless/00751_default_databasename_for_view.sql index 9292b612d87..599ef5b89ae 100644 --- a/tests/queries/0_stateless/00751_default_databasename_for_view.sql +++ b/tests/queries/0_stateless/00751_default_databasename_for_view.sql @@ -1,7 +1,3 @@ --- Tags: no-parallel - -CREATE DATABASE IF NOT EXISTS test_00751; -USE test_00751; DROP TABLE IF EXISTS t_00751; DROP TABLE IF EXISTS t_mv_00751; @@ -25,25 +21,24 @@ CREATE MATERIALIZED VIEW t_mv_00751 ENGINE = MergeTree ORDER BY date AS SELECT date, platform, app FROM t_00751 WHERE app = (SELECT min(app) from u_00751) AND platform = (SELECT (SELECT min(platform) from v_00751)); -SHOW CREATE TABLE test_00751.t_mv_00751 FORMAT TabSeparatedRaw; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.t_mv_00751 FORMAT TabSeparatedRaw; USE default; -DETACH TABLE test_00751.t_mv_00751; -ATTACH TABLE test_00751.t_mv_00751; +DETACH TABLE {CLICKHOUSE_DATABASE:Identifier}.t_mv_00751; +ATTACH TABLE {CLICKHOUSE_DATABASE:Identifier}.t_mv_00751; -INSERT INTO test_00751.t_00751 VALUES ('2000-01-01', 'a', 'a') ('2000-01-02', 'b', 'b'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.t_00751 VALUES ('2000-01-01', 'a', 'a') ('2000-01-02', 'b', 'b'); -INSERT INTO test_00751.u_00751 VALUES ('a'); -INSERT INTO test_00751.v_00751 VALUES ('a'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.u_00751 VALUES ('a'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.v_00751 VALUES ('a'); -INSERT INTO test_00751.t_00751 VALUES ('2000-01-03', 'a', 'a') ('2000-01-04', 'b', 'b'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.t_00751 VALUES ('2000-01-03', 'a', 'a') ('2000-01-04', 'b', 'b'); -SELECT * FROM test_00751.t_00751 ORDER BY date; -SELECT * FROM test_00751.t_mv_00751 ORDER BY date; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.t_00751 ORDER BY date; +SELECT * FROM {CLICKHOUSE_DATABASE:Identifier}.t_mv_00751 ORDER BY date; -DROP TABLE test_00751.t_00751; -DROP TABLE test_00751.t_mv_00751; -DROP TABLE test_00751.u_00751; -DROP TABLE test_00751.v_00751; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.t_00751; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.t_mv_00751; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.u_00751; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.v_00751; -DROP DATABASE test_00751; diff --git a/tests/queries/0_stateless/00800_versatile_storage_join.sql b/tests/queries/0_stateless/00800_versatile_storage_join.sql index 3690ea6bf89..cccc655e34b 100644 --- a/tests/queries/0_stateless/00800_versatile_storage_join.sql +++ b/tests/queries/0_stateless/00800_versatile_storage_join.sql @@ -1,8 +1,3 @@ --- Tags: no-parallel - -CREATE DATABASE IF NOT EXISTS test_00800; - -USE test_00800; DROP TABLE IF EXISTS join_any_inner; DROP TABLE IF EXISTS join_any_left; @@ -54,24 +49,22 @@ SELECT joinGet('join_string_key', 'x', 'abc'), joinGet('join_string_key', 'k', ' USE default; -DROP TABLE test_00800.join_any_inner; -DROP TABLE test_00800.join_any_left; -DROP TABLE test_00800.join_any_left_null; -DROP TABLE test_00800.join_all_inner; -DROP TABLE test_00800.join_all_left; -DROP TABLE test_00800.join_string_key; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.join_any_inner; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.join_any_left; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.join_any_left_null; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.join_all_inner; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.join_all_left; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.join_string_key; -- test provided by Alexander Zaitsev -DROP TABLE IF EXISTS test_00800.join_test; -CREATE TABLE test_00800.join_test (a UInt8, b UInt8) Engine = Join(ANY, LEFT, a); +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.join_test; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.join_test (a UInt8, b UInt8) Engine = Join(ANY, LEFT, a); -USE test_00800; +USE {CLICKHOUSE_DATABASE:Identifier}; select joinGet('join_test', 'b', 1); USE system; -SELECT joinGet('test_00800.join_test', 'b', 1); +SELECT joinGet({CLICKHOUSE_DATABASE:String} || '.join_test', 'b', 1); USE default; -DROP TABLE test_00800.join_test; - -DROP DATABASE test_00800; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.join_test; diff --git a/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql b/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql index e7f034131a2..2044a9b8d22 100644 --- a/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql +++ b/tests/queries/0_stateless/00857_global_joinsavel_table_alias.sql @@ -1,7 +1,4 @@ --- Tags: global, no-parallel -CREATE DATABASE IF NOT EXISTS test_00857; -USE test_00857; DROP TABLE IF EXISTS local_table; DROP TABLE IF EXISTS other_table; @@ -27,7 +24,7 @@ INSERT INTO other_table VALUES(100, 'One Hundred', now(), 1000); INSERT INTO other_table VALUES(200, 'Two Hundred', now(), 2000); select t2.name from remote('127.0.0.2', currentDatabase(), 'local_table') as t1 -left join test_00857.other_table as t2 -- FIXME: doesn't work properly on remote without explicit database prefix +left join {CLICKHOUSE_DATABASE:Identifier}.other_table as t2 -- FIXME: doesn't work properly on remote without explicit database prefix on t1.oth_id = t2.id order by t2.name; @@ -58,4 +55,3 @@ order by other_table.name; DROP TABLE local_table; DROP TABLE other_table; -DROP DATABASE test_00857; diff --git a/tests/queries/0_stateless/01012_show_tables_limit.sql b/tests/queries/0_stateless/01012_show_tables_limit.sql index 82cde6917d0..18a11f66d08 100644 --- a/tests/queries/0_stateless/01012_show_tables_limit.sql +++ b/tests/queries/0_stateless/01012_show_tables_limit.sql @@ -1,22 +1,15 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS test_show_limit; - -CREATE DATABASE test_show_limit; - -CREATE TABLE test_show_limit.test1 (test UInt8) ENGINE = TinyLog; -CREATE TABLE test_show_limit.test2 (test UInt8) ENGINE = TinyLog; -CREATE TABLE test_show_limit.test3 (test UInt8) ENGINE = TinyLog; -CREATE TABLE test_show_limit.test4 (test UInt8) ENGINE = TinyLog; -CREATE TABLE test_show_limit.test5 (test UInt8) ENGINE = TinyLog; -CREATE TABLE test_show_limit.test6 (test UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test1 (test UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test2 (test UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test3 (test UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test4 (test UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test5 (test UInt8) ENGINE = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test6 (test UInt8) ENGINE = TinyLog; SELECT '*** Should show 6: ***'; -SHOW TABLES FROM test_show_limit; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier}; SELECT '*** Should show 2: ***'; -SHOW TABLES FROM test_show_limit LIMIT 2; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier} LIMIT 2; SELECT '*** Should show 4: ***'; -SHOW TABLES FROM test_show_limit LIMIT 2 * 2; - -DROP DATABASE test_show_limit; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier} LIMIT 2 * 2; diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_select.reference b/tests/queries/0_stateless/01018_ddl_dictionaries_select.reference index f69302fb90f..17a77b91d45 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_select.reference +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_select.reference @@ -15,12 +15,12 @@ dict1 Dictionary dict2 Dictionary table_for_dict MergeTree -database_for_dict dict1 ComplexKeyCache -database_for_dict dict2 Hashed +default dict1 ComplexKeyCache +default dict2 Hashed 6 6 6 6 6 -database_for_dict.dict3 6 +default.dict3 6 6 diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql b/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql index 748c733bf9a..523b057d4e1 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql @@ -1,13 +1,9 @@ --- Tags: no-parallel, no-fasttest +-- Tags: no-fasttest SET send_logs_level = 'fatal'; SET check_table_dependencies=0; -DROP DATABASE IF EXISTS database_for_dict; - -CREATE DATABASE database_for_dict; - -CREATE TABLE database_for_dict.table_for_dict +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict ( key_column UInt64, second_column UInt8, @@ -17,9 +13,9 @@ CREATE TABLE database_for_dict.table_for_dict ENGINE = MergeTree() ORDER BY key_column; -INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100); -CREATE DICTIONARY database_for_dict.dict1 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -27,35 +23,35 @@ CREATE DICTIONARY database_for_dict.dict1 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); -SELECT second_column FROM database_for_dict.dict1 WHERE key_column = 11; -SELECT dictGetString('database_for_dict.dict1', 'third_column', toUInt64(12)); -SELECT third_column FROM database_for_dict.dict1 WHERE key_column = 12; -SELECT dictGetFloat64('database_for_dict.dict1', 'fourth_column', toUInt64(14)); -SELECT fourth_column FROM database_for_dict.dict1 WHERE key_column = 14; +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', toUInt64(11)); +SELECT second_column FROM {CLICKHOUSE_DATABASE:Identifier}.dict1 WHERE key_column = 11; +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict1', 'third_column', toUInt64(12)); +SELECT third_column FROM {CLICKHOUSE_DATABASE:Identifier}.dict1 WHERE key_column = 12; +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict1', 'fourth_column', toUInt64(14)); +SELECT fourth_column FROM {CLICKHOUSE_DATABASE:Identifier}.dict1 WHERE key_column = 14; -SELECT count(distinct(dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(number)))) from numbers(100); +SELECT count(distinct(dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', toUInt64(number)))) from numbers(100); -DETACH DICTIONARY database_for_dict.dict1; +DETACH DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1; -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); -- {serverError 36} +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', toUInt64(11)); -- {serverError 36} -ATTACH DICTIONARY database_for_dict.dict1; +ATTACH DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1; -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', toUInt64(11)); -DROP DICTIONARY database_for_dict.dict1; +DROP DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1; -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); -- {serverError 36} +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', toUInt64(11)); -- {serverError 36} -- SOURCE(CLICKHOUSE(...)) uses default params if not specified -DROP DICTIONARY IF EXISTS database_for_dict.dict1; +DROP DICTIONARY IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.dict1; -CREATE DICTIONARY database_for_dict.dict1 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -63,17 +59,17 @@ CREATE DICTIONARY database_for_dict.dict1 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(TABLE 'table_for_dict' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(TABLE 'table_for_dict' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(11)); +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', toUInt64(11)); -SELECT count(distinct(dictGetUInt8('database_for_dict.dict1', 'second_column', toUInt64(number)))) from numbers(100); +SELECT count(distinct(dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', toUInt64(number)))) from numbers(100); -DROP DICTIONARY database_for_dict.dict1; +DROP DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1; -CREATE DICTIONARY database_for_dict.dict1 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -81,69 +77,68 @@ CREATE DICTIONARY database_for_dict.dict1 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column, third_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 1)); -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', tuple(toUInt64(11), '121')); -SELECT dictGetFloat64('database_for_dict.dict1', 'fourth_column', tuple(toUInt64(14), '196')); +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', tuple(toUInt64(11), '121')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict1', 'fourth_column', tuple(toUInt64(14), '196')); -DETACH DICTIONARY database_for_dict.dict1; +DETACH DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1; -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', tuple(toUInt64(11), '121')); -- {serverError 36} +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', tuple(toUInt64(11), '121')); -- {serverError 36} -ATTACH DICTIONARY database_for_dict.dict1; +ATTACH DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1; -SELECT dictGetUInt8('database_for_dict.dict1', 'second_column', tuple(toUInt64(11), '121')); +SELECT dictGetUInt8({CLICKHOUSE_DATABASE:String} || '.dict1', 'second_column', tuple(toUInt64(11), '121')); -CREATE DICTIONARY database_for_dict.dict2 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict2 ( key_column UInt64 DEFAULT 0, some_column String EXPRESSION toString(fourth_column), fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(HASHED()); -SELECT dictGetString('database_for_dict.dict2', 'some_column', toUInt64(12)); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict2', 'some_column', toUInt64(12)); -SELECT name, engine FROM system.tables WHERE database = 'database_for_dict' ORDER BY name; +-- NOTE: database = currentDatabase() is not mandatory +SELECT name, engine FROM system.tables WHERE database = {CLICKHOUSE_DATABASE:String} ORDER BY name; -SELECT database, name, type FROM system.dictionaries WHERE database = 'database_for_dict' ORDER BY name; +SELECT database, name, type FROM system.dictionaries WHERE database = {CLICKHOUSE_DATABASE:String} ORDER BY name; -- check dictionary will not update -CREATE DICTIONARY database_for_dict.dict3 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict3 ( key_column UInt64 DEFAULT 0, some_column String EXPRESSION toString(fourth_column), fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase())) LIFETIME(0) LAYOUT(HASHED()); -SELECT dictGetString('database_for_dict.dict3', 'some_column', toUInt64(12)); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict3', 'some_column', toUInt64(12)); -- dictGet with table name -USE database_for_dict; +USE {CLICKHOUSE_DATABASE:Identifier}; SELECT dictGetString(dict3, 'some_column', toUInt64(12)); -SELECT dictGetString(database_for_dict.dict3, 'some_column', toUInt64(12)); +SELECT dictGetString({CLICKHOUSE_DATABASE:Identifier}.dict3, 'some_column', toUInt64(12)); SELECT dictGetString(default.dict3, 'some_column', toUInt64(12)); -- {serverError 36} SELECT dictGet(dict3, 'some_column', toUInt64(12)); -SELECT dictGet(database_for_dict.dict3, 'some_column', toUInt64(12)); +SELECT dictGet({CLICKHOUSE_DATABASE:Identifier}.dict3, 'some_column', toUInt64(12)); SELECT dictGet(default.dict3, 'some_column', toUInt64(12)); -- {serverError 36} USE default; -- alias should be handled correctly -SELECT 'database_for_dict.dict3' as n, dictGet(n, 'some_column', toUInt64(12)); +SELECT {CLICKHOUSE_DATABASE:String} || '.dict3' as n, dictGet(n, 'some_column', toUInt64(12)); -DROP TABLE database_for_dict.table_for_dict; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict; SYSTEM RELOAD DICTIONARIES; -- {serverError 60} -SELECT dictGetString('database_for_dict.dict3', 'some_column', toUInt64(12)); - -DROP DATABASE IF EXISTS database_for_dict; +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict3', 'some_column', toUInt64(12)); diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql b/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql index 1877c289d3f..51e1eb2e12f 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql @@ -1,14 +1,10 @@ --- Tags: no-parallel, no-fasttest +-- Tags: no-fasttest SET send_logs_level = 'fatal'; -DROP DATABASE IF EXISTS database_for_dict; - -CREATE DATABASE database_for_dict; - SELECT '***date dict***'; -CREATE TABLE database_for_dict.date_table +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.date_table ( CountryID UInt64, StartDate Date, @@ -18,11 +14,11 @@ CREATE TABLE database_for_dict.date_table ENGINE = MergeTree() ORDER BY CountryID; -INSERT INTO database_for_dict.date_table VALUES(1, toDate('2019-05-05'), toDate('2019-05-20'), 0.33); -INSERT INTO database_for_dict.date_table VALUES(1, toDate('2019-05-21'), toDate('2019-05-30'), 0.42); -INSERT INTO database_for_dict.date_table VALUES(2, toDate('2019-05-21'), toDate('2019-05-30'), 0.46); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.date_table VALUES(1, toDate('2019-05-05'), toDate('2019-05-20'), 0.33); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.date_table VALUES(1, toDate('2019-05-21'), toDate('2019-05-30'), 0.42); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.date_table VALUES(2, toDate('2019-05-21'), toDate('2019-05-30'), 0.46); -CREATE DICTIONARY database_for_dict.dict1 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1 ( CountryID UInt64, StartDate Date, @@ -30,19 +26,19 @@ CREATE DICTIONARY database_for_dict.dict1 Tax Float64 ) PRIMARY KEY CountryID -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'date_table' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'date_table' DB currentDatabase())) LIFETIME(MIN 1 MAX 1000) LAYOUT(RANGE_HASHED()) RANGE(MIN StartDate MAX EndDate); -SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(1), toDate('2019-05-15')); -SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(1), toDate('2019-05-29')); -SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(2), toDate('2019-05-29')); -SELECT dictGetFloat64('database_for_dict.dict1', 'Tax', toUInt64(2), toDate('2019-05-31')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict1', 'Tax', toUInt64(1), toDate('2019-05-15')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict1', 'Tax', toUInt64(1), toDate('2019-05-29')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict1', 'Tax', toUInt64(2), toDate('2019-05-29')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict1', 'Tax', toUInt64(2), toDate('2019-05-31')); SELECT '***datetime dict***'; -CREATE TABLE database_for_dict.datetime_table +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.datetime_table ( CountryID UInt64, StartDate DateTime, @@ -52,11 +48,11 @@ CREATE TABLE database_for_dict.datetime_table ENGINE = MergeTree() ORDER BY CountryID; -INSERT INTO database_for_dict.datetime_table VALUES(1, toDateTime('2019-05-05 00:00:00'), toDateTime('2019-05-20 00:00:00'), 0.33); -INSERT INTO database_for_dict.datetime_table VALUES(1, toDateTime('2019-05-21 00:00:00'), toDateTime('2019-05-30 00:00:00'), 0.42); -INSERT INTO database_for_dict.datetime_table VALUES(2, toDateTime('2019-05-21 00:00:00'), toDateTime('2019-05-30 00:00:00'), 0.46); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.datetime_table VALUES(1, toDateTime('2019-05-05 00:00:00'), toDateTime('2019-05-20 00:00:00'), 0.33); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.datetime_table VALUES(1, toDateTime('2019-05-21 00:00:00'), toDateTime('2019-05-30 00:00:00'), 0.42); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.datetime_table VALUES(2, toDateTime('2019-05-21 00:00:00'), toDateTime('2019-05-30 00:00:00'), 0.46); -CREATE DICTIONARY database_for_dict.dict2 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict2 ( CountryID UInt64, StartDate DateTime, @@ -64,19 +60,19 @@ CREATE DICTIONARY database_for_dict.dict2 Tax Float64 ) PRIMARY KEY CountryID -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'datetime_table' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'datetime_table' DB currentDatabase())) LIFETIME(MIN 1 MAX 1000) LAYOUT(RANGE_HASHED()) RANGE(MIN StartDate MAX EndDate); -SELECT dictGetFloat64('database_for_dict.dict2', 'Tax', toUInt64(1), toDateTime('2019-05-15 00:00:00')); -SELECT dictGetFloat64('database_for_dict.dict2', 'Tax', toUInt64(1), toDateTime('2019-05-29 00:00:00')); -SELECT dictGetFloat64('database_for_dict.dict2', 'Tax', toUInt64(2), toDateTime('2019-05-29 00:00:00')); -SELECT dictGetFloat64('database_for_dict.dict2', 'Tax', toUInt64(2), toDateTime('2019-05-31 00:00:00')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict2', 'Tax', toUInt64(1), toDateTime('2019-05-15 00:00:00')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict2', 'Tax', toUInt64(1), toDateTime('2019-05-29 00:00:00')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict2', 'Tax', toUInt64(2), toDateTime('2019-05-29 00:00:00')); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict2', 'Tax', toUInt64(2), toDateTime('2019-05-31 00:00:00')); SELECT '***hierarchy dict***'; -CREATE TABLE database_for_dict.table_with_hierarchy +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_with_hierarchy ( RegionID UInt64, ParentRegionID UInt64, @@ -85,24 +81,24 @@ CREATE TABLE database_for_dict.table_with_hierarchy ENGINE = MergeTree() ORDER BY RegionID; -INSERT INTO database_for_dict.table_with_hierarchy VALUES (3, 2, 'Hamovniki'), (2, 1, 'Moscow'), (1, 10000, 'Russia') (7, 10000, 'Ulan-Ude'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_with_hierarchy VALUES (3, 2, 'Hamovniki'), (2, 1, 'Moscow'), (1, 10000, 'Russia') (7, 10000, 'Ulan-Ude'); -CREATE DICTIONARY database_for_dict.dictionary_with_hierarchy +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dictionary_with_hierarchy ( RegionID UInt64, ParentRegionID UInt64 HIERARCHICAL, RegionName String ) PRIMARY KEY RegionID -SOURCE(CLICKHOUSE(host 'localhost' port tcpPort() user 'default' db 'database_for_dict' table 'table_with_hierarchy')) +SOURCE(CLICKHOUSE(host 'localhost' port tcpPort() user 'default' db currentDatabase() table 'table_with_hierarchy')) LAYOUT(HASHED()) LIFETIME(MIN 1 MAX 1000); -SELECT dictGetString('database_for_dict.dictionary_with_hierarchy', 'RegionName', toUInt64(2)); -SELECT dictGetHierarchy('database_for_dict.dictionary_with_hierarchy', toUInt64(3)); -SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(3), toUInt64(2)); -SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(7), toUInt64(10000)); -SELECT dictIsIn('database_for_dict.dictionary_with_hierarchy', toUInt64(1), toUInt64(5)); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dictionary_with_hierarchy', 'RegionName', toUInt64(2)); +SELECT dictGetHierarchy({CLICKHOUSE_DATABASE:String} || '.dictionary_with_hierarchy', toUInt64(3)); +SELECT dictIsIn({CLICKHOUSE_DATABASE:String} || '.dictionary_with_hierarchy', toUInt64(3), toUInt64(2)); +SELECT dictIsIn({CLICKHOUSE_DATABASE:String} || '.dictionary_with_hierarchy', toUInt64(7), toUInt64(10000)); +SELECT dictIsIn({CLICKHOUSE_DATABASE:String} || '.dictionary_with_hierarchy', toUInt64(1), toUInt64(5)); -DROP DATABASE IF EXISTS database_for_dict; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql index e72e113f859..d958d7d58ef 100644 --- a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql +++ b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql @@ -1,12 +1,7 @@ --- Tags: no-parallel SET send_logs_level = 'fatal'; -DROP DATABASE IF EXISTS database_for_dict; - -CREATE DATABASE database_for_dict; - -CREATE TABLE database_for_dict.table_for_dict +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict ( key_column UInt64, second_column UInt8, @@ -16,9 +11,9 @@ CREATE TABLE database_for_dict.table_for_dict ENGINE = MergeTree() ORDER BY key_column; -INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100); -CREATE DICTIONARY database_for_dict.dict1 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -26,13 +21,13 @@ CREATE DICTIONARY database_for_dict.dict1 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); -SELECT count(*) from database_for_dict.dict1; +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; -CREATE DICTIONARY database_for_dict.dict2 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict2 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -40,20 +35,20 @@ CREATE DICTIONARY database_for_dict.dict2 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict1' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict1' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(HASHED()); -SELECT count(*) FROM database_for_dict.dict2; +SELECT count(*) FROM {CLICKHOUSE_DATABASE:Identifier}.dict2; -INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100, 100); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100, 100); SYSTEM RELOAD DICTIONARIES; -SELECT count(*) from database_for_dict.dict2; -SELECT count(*) from database_for_dict.dict1; +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict2; +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; -CREATE DICTIONARY database_for_dict.dict3 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict3 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -61,22 +56,22 @@ CREATE DICTIONARY database_for_dict.dict3 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict2' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict2' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(HASHED()); -SELECT count(*) FROM database_for_dict.dict3; +SELECT count(*) FROM {CLICKHOUSE_DATABASE:Identifier}.dict3; -INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(200, 100); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(200, 100); SYSTEM RELOAD DICTIONARIES; -SELECT count(*) from database_for_dict.dict3; -SELECT count(*) from database_for_dict.dict2; -SELECT count(*) from database_for_dict.dict1; +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict3; +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict2; +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; -CREATE DICTIONARY database_for_dict.dict4 +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict4 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -84,17 +79,17 @@ CREATE DICTIONARY database_for_dict.dict4 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'non_existing_table' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'non_existing_table' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(HASHED()); -SELECT count(*) FROM database_for_dict.dict4; -- {serverError 60} +SELECT count(*) FROM {CLICKHOUSE_DATABASE:Identifier}.dict4; -- {serverError 60} -SELECT name from system.tables WHERE database = 'database_for_dict' ORDER BY name; -SELECT name from system.dictionaries WHERE database = 'database_for_dict' ORDER BY name; +SELECT name from system.tables WHERE database = currentDatabase() ORDER BY name; +SELECT name from system.dictionaries WHERE database = currentDatabase() ORDER BY name; -DROP DATABASE IF EXISTS database_for_dict; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; -SELECT count(*) from database_for_dict.dict3; --{serverError 81} -SELECT count(*) from database_for_dict.dict2; --{serverError 81} -SELECT count(*) from database_for_dict.dict1; --{serverError 81} +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict3; --{serverError 81} +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict2; --{serverError 81} +SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; --{serverError 81} diff --git a/tests/queries/0_stateless/01018_ip_dictionary_long.sql b/tests/queries/0_stateless/01018_ip_dictionary_long.sql index 647c36429cc..bb7f120163c 100644 --- a/tests/queries/0_stateless/01018_ip_dictionary_long.sql +++ b/tests/queries/0_stateless/01018_ip_dictionary_long.sql @@ -1,13 +1,9 @@ --- Tags: long, no-parallel +-- Tags: long SET send_logs_level = 'fatal'; -DROP DATABASE IF EXISTS database_for_dict; - -CREATE DATABASE database_for_dict; - SELECT '***ipv4 trie dict***'; -CREATE TABLE database_for_dict.table_ipv4_trie +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie ( prefix String, asn UInt32, @@ -16,648 +12,648 @@ CREATE TABLE database_for_dict.table_ipv4_trie engine = TinyLog; -- numbers reordered to test sorting criteria too -INSERT INTO database_for_dict.table_ipv4_trie +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie SELECT '255.255.255.255/' || toString((number + 1) * 13 % 33) AS prefix, toUInt32((number + 1) * 13 % 33) AS asn, 'NA' as cca2 FROM system.numbers LIMIT 33; -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.2', 1272, 'RU'); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.0/8', 1270, 'RU'); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.0.2', 1272, 'RU'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.0.0/8', 1270, 'RU'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); -- non-unique entries will be squashed into one -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('101.79.55.22', 11212, 'UK'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('202.79.32.2', 11211, 'NP'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('101.79.55.22', 11212, 'UK'); -CREATE DICTIONARY database_for_dict.dict_ipv4_trie +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie ( prefix String, asn UInt32, cca2 String ) PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ipv4_trie')) +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ipv4_trie')) LAYOUT(IP_TRIE()) LIFETIME(MIN 10 MAX 100); -- fuzzer -SELECT '127.0.0.0/24' = dictGetString('database_for_dict.dict_ipv4_trie', 'prefixprefixprefixprefix', tuple(IPv4StringToNumOrDefault('127.0.0.0127.0.0.0'))); -- { serverError 36 } +SELECT '127.0.0.0/24' = dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefixprefixprefixprefix', tuple(IPv4StringToNumOrDefault('127.0.0.0127.0.0.0'))); -- { serverError 36 } -SELECT 0 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('0.0.0.0'))); -SELECT 1 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('128.0.0.0'))); -SELECT 2 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('192.0.0.0'))); -SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('224.0.0.0'))); -SELECT 4 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('240.0.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('248.0.0.0'))); -SELECT 6 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('252.0.0.0'))); -SELECT 7 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('254.0.0.0'))); -SELECT 8 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.0.0.0'))); -SELECT 9 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.128.0.0'))); -SELECT 10 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.192.0.0'))); -SELECT 11 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.224.0.0'))); -SELECT 12 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.240.0.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.248.0.0'))); -SELECT 14 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.252.0.0'))); -SELECT 15 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.254.0.0'))); -SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.0.0'))); -SELECT 17 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.128.0'))); -SELECT 18 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.192.0'))); -SELECT 19 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.224.0'))); -SELECT 20 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.240.0'))); -SELECT 21 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.248.0'))); -SELECT 22 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.252.0'))); -SELECT 23 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.254.0'))); -SELECT 24 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.0'))); -SELECT 25 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.128'))); -SELECT 26 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.192'))); -SELECT 27 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.224'))); -SELECT 28 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.240'))); -SELECT 29 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.248'))); -SELECT 30 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.252'))); -SELECT 31 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.254'))); -SELECT 32 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.255'))); +SELECT 0 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('0.0.0.0'))); +SELECT 1 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('128.0.0.0'))); +SELECT 2 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('192.0.0.0'))); +SELECT 3 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('224.0.0.0'))); +SELECT 4 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('240.0.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('248.0.0.0'))); +SELECT 6 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('252.0.0.0'))); +SELECT 7 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('254.0.0.0'))); +SELECT 8 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.0.0.0'))); +SELECT 9 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.128.0.0'))); +SELECT 10 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.192.0.0'))); +SELECT 11 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.224.0.0'))); +SELECT 12 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.240.0.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.248.0.0'))); +SELECT 14 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.252.0.0'))); +SELECT 15 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.254.0.0'))); +SELECT 16 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.0.0'))); +SELECT 17 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.128.0'))); +SELECT 18 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.192.0'))); +SELECT 19 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.224.0'))); +SELECT 20 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.240.0'))); +SELECT 21 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.248.0'))); +SELECT 22 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.252.0'))); +SELECT 23 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.254.0'))); +SELECT 24 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.0'))); +SELECT 25 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.128'))); +SELECT 26 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.192'))); +SELECT 27 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.224'))); +SELECT 28 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.240'))); +SELECT 29 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.248'))); +SELECT 30 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.252'))); +SELECT 31 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.254'))); +SELECT 32 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('255.255.255.255'))); -SELECT 'RU' == dictGetString('database_for_dict.dict_ipv4_trie', 'cca2', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 'RU' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'cca2', tuple(IPv4StringToNum('127.0.0.1'))); -SELECT 1270 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.0'))); -SELECT 1270 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.1'))); -SELECT 1272 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.2'))); -SELECT 1270 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.3'))); -SELECT 1270 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.255'))); +SELECT 1270 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.0'))); +SELECT 1270 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 1272 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.2'))); +SELECT 1270 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.3'))); +SELECT 1270 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('127.0.0.255'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.0'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.1'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.2'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.3'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.255'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.0'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.2'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.3'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.255'))); -SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('101.79.55.22'))); -SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv6StringToNum('::ffff:654f:3716'))); -SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); +SELECT 11212 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('101.79.55.22'))); +SELECT 11212 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv6StringToNum('::ffff:654f:3716'))); +SELECT 11212 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); -SELECT 11211 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('202.79.32.2'))); +SELECT 11211 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('202.79.32.2'))); -- check that dictionary works with aliased types `IPv4` and `IPv6` -SELECT 11211 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(toIPv4('202.79.32.2'))); -SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(toIPv6('::ffff:101.79.55.22'))); +SELECT 11211 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(toIPv4('202.79.32.2'))); +SELECT 11212 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'asn', tuple(toIPv6('::ffff:101.79.55.22'))); -CREATE TABLE database_for_dict.table_from_ipv4_trie_dict +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_from_ipv4_trie_dict ( prefix String, asn UInt32, cca2 String -) ENGINE = Dictionary(database_for_dict.dict_ipv4_trie); +) ENGINE = Dictionary({CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie); -SELECT 1272 == asn AND 'RU' == cca2 FROM database_for_dict.table_from_ipv4_trie_dict +SELECT 1272 == asn AND 'RU' == cca2 FROM {CLICKHOUSE_DATABASE:Identifier}.table_from_ipv4_trie_dict WHERE prefix == '127.0.0.2/32'; -SELECT 37 == COUNT(*) FROM database_for_dict.table_from_ipv4_trie_dict; -SELECT 37 == COUNT(DISTINCT prefix) FROM database_for_dict.table_from_ipv4_trie_dict; +SELECT 37 == COUNT(*) FROM {CLICKHOUSE_DATABASE:Identifier}.table_from_ipv4_trie_dict; +SELECT 37 == COUNT(DISTINCT prefix) FROM {CLICKHOUSE_DATABASE:Identifier}.table_from_ipv4_trie_dict; -DROP TABLE IF EXISTS database_for_dict.table_from_ipv4_trie_dict; -DROP DICTIONARY IF EXISTS database_for_dict.dict_ipv4_trie; -DROP TABLE IF EXISTS database_for_dict.table_ipv4_trie; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.table_from_ipv4_trie_dict; +DROP DICTIONARY IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie; SELECT '***ipv4 trie dict mask***'; -CREATE TABLE database_for_dict.table_ipv4_trie +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie ( prefix String, val UInt32 ) engine = TinyLog; -INSERT INTO database_for_dict.table_ipv4_trie +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie SELECT '255.255.255.255/' || toString(number) AS prefix, toUInt32(number) AS val FROM VALUES ('number UInt32', 5, 13, 24, 30); -CREATE DICTIONARY database_for_dict.dict_ipv4_trie +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie ( prefix String, val UInt32 ) PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ipv4_trie')) +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ipv4_trie')) LAYOUT(IP_TRIE()) LIFETIME(MIN 10 MAX 100); -SELECT 0 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('0.0.0.0'))); -SELECT 0 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('128.0.0.0'))); -SELECT 0 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('192.0.0.0'))); -SELECT 0 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('224.0.0.0'))); -SELECT 0 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('240.0.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('248.0.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('252.0.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('254.0.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.0.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.128.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.192.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.224.0.0'))); -SELECT 5 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.240.0.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.248.0.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.252.0.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.254.0.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.0.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.128.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.192.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.224.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.240.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.248.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.252.0'))); -SELECT 13 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.254.0'))); -SELECT 24 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.0'))); -SELECT 24 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.128'))); -SELECT 24 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.192'))); -SELECT 24 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.224'))); -SELECT 24 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.240'))); -SELECT 24 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.248'))); -SELECT 30 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.252'))); -SELECT 30 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.254'))); -SELECT 30 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.255'))); +SELECT 0 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('0.0.0.0'))); +SELECT 0 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('128.0.0.0'))); +SELECT 0 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('192.0.0.0'))); +SELECT 0 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('224.0.0.0'))); +SELECT 0 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('240.0.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('248.0.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('252.0.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('254.0.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.0.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.128.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.192.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.224.0.0'))); +SELECT 5 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.240.0.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.248.0.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.252.0.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.254.0.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.0.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.128.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.192.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.224.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.240.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.248.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.252.0'))); +SELECT 13 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.254.0'))); +SELECT 24 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.0'))); +SELECT 24 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.128'))); +SELECT 24 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.192'))); +SELECT 24 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.224'))); +SELECT 24 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.240'))); +SELECT 24 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.248'))); +SELECT 30 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.252'))); +SELECT 30 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.254'))); +SELECT 30 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('255.255.255.255'))); -DROP DICTIONARY IF EXISTS database_for_dict.dict_ipv4_trie; -DROP TABLE IF EXISTS database_for_dict.table_from_ipv4_trie_dict; -DROP TABLE IF EXISTS database_for_dict.table_ipv4_trie; +DROP DICTIONARY IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.table_from_ipv4_trie_dict; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie; SELECT '***ipv4 trie dict pt2***'; -CREATE TABLE database_for_dict.table_ipv4_trie ( prefix String, val UInt32 ) engine = TinyLog; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie ( prefix String, val UInt32 ) engine = TinyLog; -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.0/8', 1); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.0/16', 2); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.0/24', 3); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.0.1/32', 4); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.127.0/32', 5); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.128.1/32', 6); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.255.0/32', 7); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.255.1/32', 8); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.0.255.255/32', 9); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.1.0.0/16', 10); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.1.1.0', 11); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.1.255.0/24', 12); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.254.0.0/15', 13); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.254.0.127', 14); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.0.0/16', 15); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.128.0/24', 16); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.128.1/32', 17); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.128.10/32', 18); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.128.128/25', 19); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.255.128/32', 20); -INSERT INTO database_for_dict.table_ipv4_trie VALUES ('127.255.255.255/32', 21); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.0.0/8', 1); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.0.0/16', 2); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.0.0/24', 3); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.0.1/32', 4); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.127.0/32', 5); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.128.1/32', 6); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.255.0/32', 7); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.255.1/32', 8); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.0.255.255/32', 9); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.1.0.0/16', 10); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.1.1.0', 11); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.1.255.0/24', 12); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.254.0.0/15', 13); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.254.0.127', 14); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.255.0.0/16', 15); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.255.128.0/24', 16); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.255.128.1/32', 17); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.255.128.10/32', 18); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.255.128.128/25', 19); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.255.255.128/32', 20); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ipv4_trie VALUES ('127.255.255.255/32', 21); -CREATE DICTIONARY database_for_dict.dict_ipv4_trie ( prefix String, val UInt32 ) +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ipv4_trie ( prefix String, val UInt32 ) PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ipv4_trie')) +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ipv4_trie')) LAYOUT(IP_TRIE(ACCESS_TO_KEY_FROM_ATTRIBUTES 1)) LIFETIME(MIN 10 MAX 100); -SELECT '127.0.0.0/24' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.0.0'))); -SELECT '127.0.0.1/32' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.0.1'))); -SELECT '127.0.0.0/24' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.0.127'))); -SELECT '127.0.0.0/16' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.255.127'))); -SELECT '127.255.0.0/16' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.127.127'))); -SELECT '127.255.128.0/24' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.9'))); -SELECT '127.255.128.0/24' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.127'))); -SELECT '127.255.128.10/32' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.10'))); -SELECT '127.255.128.128/25' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.255'))); -SELECT '127.255.255.128/32' == dictGetString('database_for_dict.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.255.128'))); +SELECT '127.0.0.0/24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.0.0'))); +SELECT '127.0.0.1/32' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT '127.0.0.0/24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.0.127'))); +SELECT '127.0.0.0/16' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.0.255.127'))); +SELECT '127.255.0.0/16' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.127.127'))); +SELECT '127.255.128.0/24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.9'))); +SELECT '127.255.128.0/24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.127'))); +SELECT '127.255.128.10/32' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.10'))); +SELECT '127.255.128.128/25' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.128.255'))); +SELECT '127.255.255.128/32' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'prefix', tuple(IPv4StringToNum('127.255.255.128'))); -SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.0'))); -SELECT 4 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.1'))); -SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.127'))); -SELECT 2 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.255.127'))); -SELECT 15 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.127.127'))); -SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.9'))); -SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.127'))); -SELECT 18 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.10'))); -SELECT 19 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.255'))); -SELECT 20 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.255.128'))); +SELECT 3 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.0'))); +SELECT 4 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 3 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.127'))); +SELECT 2 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.255.127'))); +SELECT 15 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.127.127'))); +SELECT 16 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.9'))); +SELECT 16 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.127'))); +SELECT 18 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.10'))); +SELECT 19 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.255'))); +SELECT 20 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.255.128'))); -SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:0'))); -SELECT 4 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:1'))); -SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:7f'))); -SELECT 2 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:ff7f'))); -SELECT 15 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:7f7f'))); -SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:8009'))); -SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:807f'))); -SELECT 18 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:800a'))); -SELECT 19 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:80ff'))); -SELECT 20 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:ff80'))); +SELECT 3 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:0'))); +SELECT 4 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:1'))); +SELECT 3 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:7f'))); +SELECT 2 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:ff7f'))); +SELECT 15 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:7f7f'))); +SELECT 16 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:8009'))); +SELECT 16 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:807f'))); +SELECT 18 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:800a'))); +SELECT 19 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:80ff'))); +SELECT 20 == dictGetUInt32({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:ff80'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.0'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.1'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.127'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.255.127'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.127.127'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.9'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.127'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.10'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.255'))); -SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.255.128'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.0'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.127'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.255.127'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.127.127'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.9'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.127'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.10'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.128.255'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('127.255.255.128'))); -SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('128.127.127.127'))); -SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('128.127.127.0'))); -SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('255.127.127.0'))); -SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('255.0.0.0'))); -SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('0.0.0.0'))); -SELECT 0 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('1.1.1.1'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('128.127.127.127'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('128.127.127.0'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('255.127.127.0'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('255.0.0.0'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('0.0.0.0'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ipv4_trie', tuple(IPv4StringToNum('1.1.1.1'))); SELECT '***ipv6 trie dict***'; -CREATE TABLE database_for_dict.table_ip_trie +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie ( prefix String, val String ) engine = TinyLog; -INSERT INTO database_for_dict.table_ip_trie VALUES ('101.79.55.22', 'JA'), ('127.0.0.1', 'RU'), ('2620:0:870::/48', 'US'), ('2a02:6b8:1::/48', 'UK'), ('2001:db8::/32', 'ZZ'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie VALUES ('101.79.55.22', 'JA'), ('127.0.0.1', 'RU'), ('2620:0:870::/48', 'US'), ('2a02:6b8:1::/48', 'UK'), ('2001:db8::/32', 'ZZ'); -INSERT INTO database_for_dict.table_ip_trie +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie SELECT 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/' || toString((number + 1) * 13 % 129) AS prefix, toString((number + 1) * 13 % 129) AS val FROM system.numbers LIMIT 129; -CREATE DICTIONARY database_for_dict.dict_ip_trie +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ip_trie ( prefix String, val String ) PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ip_trie')) +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ip_trie')) LAYOUT(IP_TRIE(ACCESS_TO_KEY_FROM_ATTRIBUTES 1)) LIFETIME(MIN 10 MAX 100); -SELECT 'US' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('2620:0:870::'))); -SELECT 'UK' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('2a02:6b8:1::'))); -SELECT 'ZZ' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('2001:db8::'))); -SELECT 'ZZ' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('2001:db8:ffff::'))); +SELECT 'US' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('2620:0:870::'))); +SELECT 'UK' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('2a02:6b8:1::'))); +SELECT 'ZZ' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('2001:db8::'))); +SELECT 'ZZ' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('2001:db8:ffff::'))); -SELECT 1 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('2001:db8:ffff::'))); -SELECT 1 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('2001:db8:ffff:ffff::'))); -SELECT 1 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('2001:db8:ffff:1::'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv6StringToNum('2001:db8:ffff::'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv6StringToNum('2001:db8:ffff:ffff::'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv6StringToNum('2001:db8:ffff:1::'))); -SELECT '0' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('654f:3716::'))); +SELECT '0' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('654f:3716::'))); -SELECT 'JA' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:654f:3716'))); -SELECT 'JA' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); -SELECT 'JA' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('101.79.55.22'))); -SELECT 1 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv4StringToNum('127.0.0.1'))); -SELECT 1 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('::ffff:127.0.0.1'))); +SELECT 'JA' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:654f:3716'))); +SELECT 'JA' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); +SELECT 'JA' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('101.79.55.22'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv4StringToNum('127.0.0.1'))); +SELECT 1 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv6StringToNum('::ffff:127.0.0.1'))); -SELECT '2620:0:870::/48' == dictGetString('database_for_dict.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('2620:0:870::'))); -SELECT '2a02:6b8:1::/48' == dictGetString('database_for_dict.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('2a02:6b8:1::1'))); -SELECT '2001:db8::/32' == dictGetString('database_for_dict.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('2001:db8::1'))); -SELECT '::ffff:101.79.55.22/128' == dictGetString('database_for_dict.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('::ffff:654f:3716'))); -SELECT '::ffff:101.79.55.22/128' == dictGetString('database_for_dict.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); +SELECT '2620:0:870::/48' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('2620:0:870::'))); +SELECT '2a02:6b8:1::/48' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('2a02:6b8:1::1'))); +SELECT '2001:db8::/32' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('2001:db8::1'))); +SELECT '::ffff:101.79.55.22/128' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('::ffff:654f:3716'))); +SELECT '::ffff:101.79.55.22/128' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'prefix', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); -SELECT '0' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('::0'))); -SELECT '1' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('8000::'))); -SELECT '2' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('c000::'))); -SELECT '3' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('e000::'))); -SELECT '4' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('f000::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('f800::'))); -SELECT '6' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fc00::'))); -SELECT '7' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fe00::'))); -SELECT '8' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff00::'))); -SELECT '9' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff80::'))); -SELECT '10' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffc0::'))); -SELECT '11' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffe0::'))); -SELECT '12' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff0::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff8::'))); -SELECT '14' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffc::'))); -SELECT '15' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffe::'))); -SELECT '16' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff::'))); -SELECT '17' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:8000::'))); -SELECT '18' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); -SELECT '19' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); -SELECT '20' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); -SELECT '21' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); -SELECT '22' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); -SELECT '18' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); -SELECT '19' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); -SELECT '20' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); -SELECT '21' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); -SELECT '22' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); -SELECT '23' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fe00::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff00::'))); -SELECT '25' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff80::'))); -SELECT '26' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffc0::'))); -SELECT '27' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffe0::'))); -SELECT '28' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff0::'))); -SELECT '29' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff8::'))); -SELECT '30' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffc::'))); -SELECT '31' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffe::'))); -SELECT '32' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff::'))); -SELECT '33' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:8000::'))); -SELECT '34' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:c000::'))); -SELECT '35' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:e000::'))); -SELECT '36' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f000::'))); -SELECT '37' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f800::'))); -SELECT '38' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fc00::'))); -SELECT '39' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fe00::'))); -SELECT '40' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff00::'))); -SELECT '41' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff80::'))); -SELECT '42' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffc0::'))); -SELECT '43' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffe0::'))); -SELECT '44' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff0::'))); -SELECT '45' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff8::'))); -SELECT '46' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffc::'))); -SELECT '47' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffe::'))); -SELECT '48' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:8000::'))); -SELECT '50' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:c000::'))); -SELECT '51' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:e000::'))); -SELECT '52' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f000::'))); -SELECT '53' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f800::'))); -SELECT '54' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fc00::'))); -SELECT '55' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fe00::'))); -SELECT '56' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff00::'))); -SELECT '57' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff80::'))); -SELECT '58' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffc0::'))); -SELECT '59' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffe0::'))); -SELECT '60' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff0::'))); -SELECT '61' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff8::'))); -SELECT '62' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffc::'))); -SELECT '63' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffe::'))); -SELECT '64' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff::'))); -SELECT '65' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:8000::'))); -SELECT '66' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:c000::'))); -SELECT '67' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:e000::'))); -SELECT '68' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f000::'))); -SELECT '69' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f800::'))); -SELECT '70' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fc00::'))); -SELECT '71' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fe00::'))); -SELECT '72' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff00::'))); -SELECT '73' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff80::'))); -SELECT '74' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffc0::'))); -SELECT '75' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffe0::'))); -SELECT '76' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff0::'))); -SELECT '77' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff8::'))); -SELECT '78' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffc::'))); -SELECT '79' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffe::'))); -SELECT '80' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff::'))); -SELECT '81' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:8000::'))); -SELECT '82' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:c000::'))); -SELECT '83' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:e000::'))); -SELECT '84' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f000::'))); -SELECT '85' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f800::'))); -SELECT '86' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fc00::'))); -SELECT '87' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fe00::'))); -SELECT '88' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff00::'))); -SELECT '89' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff80::'))); -SELECT '90' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffc0::'))); -SELECT '91' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffe0::'))); -SELECT '92' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff0::'))); -SELECT '93' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff8::'))); -SELECT '94' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffc::'))); -SELECT '95' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffe::'))); -SELECT '96' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff::'))); -SELECT '97' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:8000:0'))); -SELECT '98' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:c000:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:e000:0'))); -SELECT '100' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f000:0'))); -SELECT '101' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f800:0'))); -SELECT '102' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fc00:0'))); -SELECT '103' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fe00:0'))); -SELECT '104' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff00:0'))); -SELECT '105' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff80:0'))); -SELECT '106' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffc0:0'))); -SELECT '107' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffe0:0'))); -SELECT '108' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff0:0'))); -SELECT '109' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff8:0'))); -SELECT '110' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffc:0'))); -SELECT '111' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffe:0'))); -SELECT '112' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:0'))); -SELECT '113' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:8000'))); -SELECT '114' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:c000'))); -SELECT '115' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:e000'))); -SELECT '116' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f000'))); -SELECT '117' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f800'))); -SELECT '118' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fc00'))); -SELECT '119' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fe00'))); -SELECT '120' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff00'))); -SELECT '121' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff80'))); -SELECT '122' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffc0'))); -SELECT '123' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffe0'))); -SELECT '124' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff0'))); -SELECT '125' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff8'))); -SELECT '126' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffc'))); -SELECT '127' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe'))); -SELECT '128' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'))); +SELECT '0' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('::0'))); +SELECT '1' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('8000::'))); +SELECT '2' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('c000::'))); +SELECT '3' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('e000::'))); +SELECT '4' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('f000::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('f800::'))); +SELECT '6' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fc00::'))); +SELECT '7' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fe00::'))); +SELECT '8' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff00::'))); +SELECT '9' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff80::'))); +SELECT '10' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffc0::'))); +SELECT '11' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffe0::'))); +SELECT '12' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff0::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff8::'))); +SELECT '14' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffc::'))); +SELECT '15' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffe::'))); +SELECT '16' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff::'))); +SELECT '17' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:8000::'))); +SELECT '18' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); +SELECT '19' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); +SELECT '20' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); +SELECT '21' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); +SELECT '22' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); +SELECT '18' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); +SELECT '19' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); +SELECT '20' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); +SELECT '21' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); +SELECT '22' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); +SELECT '23' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fe00::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff00::'))); +SELECT '25' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff80::'))); +SELECT '26' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffc0::'))); +SELECT '27' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffe0::'))); +SELECT '28' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff0::'))); +SELECT '29' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff8::'))); +SELECT '30' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffc::'))); +SELECT '31' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffe::'))); +SELECT '32' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff::'))); +SELECT '33' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:8000::'))); +SELECT '34' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:c000::'))); +SELECT '35' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:e000::'))); +SELECT '36' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f000::'))); +SELECT '37' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f800::'))); +SELECT '38' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fc00::'))); +SELECT '39' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fe00::'))); +SELECT '40' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff00::'))); +SELECT '41' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff80::'))); +SELECT '42' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffc0::'))); +SELECT '43' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffe0::'))); +SELECT '44' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff0::'))); +SELECT '45' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff8::'))); +SELECT '46' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffc::'))); +SELECT '47' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffe::'))); +SELECT '48' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:8000::'))); +SELECT '50' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:c000::'))); +SELECT '51' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:e000::'))); +SELECT '52' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f000::'))); +SELECT '53' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f800::'))); +SELECT '54' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fc00::'))); +SELECT '55' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fe00::'))); +SELECT '56' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff00::'))); +SELECT '57' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff80::'))); +SELECT '58' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffc0::'))); +SELECT '59' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffe0::'))); +SELECT '60' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff0::'))); +SELECT '61' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff8::'))); +SELECT '62' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffc::'))); +SELECT '63' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffe::'))); +SELECT '64' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff::'))); +SELECT '65' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:8000::'))); +SELECT '66' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:c000::'))); +SELECT '67' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:e000::'))); +SELECT '68' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f000::'))); +SELECT '69' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f800::'))); +SELECT '70' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fc00::'))); +SELECT '71' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fe00::'))); +SELECT '72' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff00::'))); +SELECT '73' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff80::'))); +SELECT '74' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffc0::'))); +SELECT '75' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffe0::'))); +SELECT '76' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff0::'))); +SELECT '77' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff8::'))); +SELECT '78' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffc::'))); +SELECT '79' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffe::'))); +SELECT '80' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff::'))); +SELECT '81' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:8000::'))); +SELECT '82' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:c000::'))); +SELECT '83' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:e000::'))); +SELECT '84' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f000::'))); +SELECT '85' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f800::'))); +SELECT '86' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fc00::'))); +SELECT '87' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fe00::'))); +SELECT '88' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff00::'))); +SELECT '89' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff80::'))); +SELECT '90' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffc0::'))); +SELECT '91' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffe0::'))); +SELECT '92' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff0::'))); +SELECT '93' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff8::'))); +SELECT '94' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffc::'))); +SELECT '95' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffe::'))); +SELECT '96' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff::'))); +SELECT '97' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:8000:0'))); +SELECT '98' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:c000:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:e000:0'))); +SELECT '100' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f000:0'))); +SELECT '101' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f800:0'))); +SELECT '102' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fc00:0'))); +SELECT '103' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fe00:0'))); +SELECT '104' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff00:0'))); +SELECT '105' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff80:0'))); +SELECT '106' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffc0:0'))); +SELECT '107' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffe0:0'))); +SELECT '108' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff0:0'))); +SELECT '109' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff8:0'))); +SELECT '110' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffc:0'))); +SELECT '111' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffe:0'))); +SELECT '112' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:0'))); +SELECT '113' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:8000'))); +SELECT '114' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:c000'))); +SELECT '115' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:e000'))); +SELECT '116' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f000'))); +SELECT '117' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f800'))); +SELECT '118' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fc00'))); +SELECT '119' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fe00'))); +SELECT '120' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff00'))); +SELECT '121' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff80'))); +SELECT '122' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffc0'))); +SELECT '123' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffe0'))); +SELECT '124' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff0'))); +SELECT '125' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff8'))); +SELECT '126' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffc'))); +SELECT '127' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe'))); +SELECT '128' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'))); -CREATE TABLE database_for_dict.table_from_ip_trie_dict +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_from_ip_trie_dict ( prefix String, val String -) ENGINE = Dictionary(database_for_dict.dict_ip_trie); +) ENGINE = Dictionary({CLICKHOUSE_DATABASE:Identifier}.dict_ip_trie); -SELECT MIN(val == 'US') FROM database_for_dict.table_from_ip_trie_dict +SELECT MIN(val == 'US') FROM {CLICKHOUSE_DATABASE:Identifier}.table_from_ip_trie_dict WHERE prefix == '2620:0:870::/48'; -SELECT 134 == COUNT(*) FROM database_for_dict.table_from_ip_trie_dict; +SELECT 134 == COUNT(*) FROM {CLICKHOUSE_DATABASE:Identifier}.table_from_ip_trie_dict; -DROP TABLE IF EXISTS database_for_dict.table_from_ip_trie_dict; -DROP DICTIONARY IF EXISTS database_for_dict.dict_ip_trie; -DROP TABLE IF EXISTS database_for_dict.table_ip_trie; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.table_from_ip_trie_dict; +DROP DICTIONARY IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.dict_ip_trie; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie; SELECT '***ipv6 trie dict mask***'; -CREATE TABLE database_for_dict.table_ip_trie +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie ( prefix String, val String ) engine = TinyLog; -INSERT INTO database_for_dict.table_ip_trie +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie SELECT 'ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff/' || toString(number) AS prefix, toString(number) AS val FROM VALUES ('number UInt32', 5, 13, 24, 48, 49, 99, 127); -INSERT INTO database_for_dict.table_ip_trie VALUES ('101.79.55.22', 'JA'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie VALUES ('101.79.55.22', 'JA'); -INSERT INTO database_for_dict.table_ip_trie +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_ip_trie SELECT '255.255.255.255/' || toString(number) AS prefix, toString(number) AS val FROM VALUES ('number UInt32', 5, 13, 24, 30); -CREATE DICTIONARY database_for_dict.dict_ip_trie +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_ip_trie ( prefix String, val String ) PRIMARY KEY prefix -SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db 'database_for_dict' table 'table_ip_trie')) +SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' db currentDatabase() table 'table_ip_trie')) LAYOUT(IP_TRIE()) LIFETIME(MIN 10 MAX 100); -SELECT 0 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('::ffff:1:1'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv6StringToNum('::ffff:1:1'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('654f:3716::'))); -SELECT 0 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('654f:3716::'))); -SELECT 0 == dictHas('database_for_dict.dict_ip_trie', tuple(IPv6StringToNum('654f:3716:ffff::'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('654f:3716::'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv6StringToNum('654f:3716::'))); +SELECT 0 == dictHas({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', tuple(IPv6StringToNum('654f:3716:ffff::'))); -SELECT 'JA' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:654f:3716'))); -SELECT 'JA' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); -SELECT 'JA' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('101.79.55.22'))); +SELECT 'JA' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:654f:3716'))); +SELECT 'JA' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('::ffff:101.79.55.22'))); +SELECT 'JA' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('101.79.55.22'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('::0'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('8000::'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('c000::'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('e000::'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('f000::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('f800::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fc00::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fe00::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff00::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff80::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffc0::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffe0::'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff0::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff8::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffc::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffe::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:8000::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fe00::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff00::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff80::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffc0::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffe0::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff0::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff8::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffc::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffe::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:8000::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:c000::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:e000::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f000::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f800::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fc00::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fe00::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff00::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff80::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffc0::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffe0::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff0::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff8::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffc::'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffe::'))); -SELECT '48' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:8000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:c000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:e000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f800::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fc00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fe00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff80::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffc0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffe0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff8::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffc::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffe::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:8000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:c000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:e000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f800::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fc00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fe00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff80::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffc0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffe0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff8::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffc::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffe::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:8000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:c000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:e000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f000::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f800::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fc00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fe00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff00::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff80::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffc0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffe0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff0::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff8::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffc::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffe::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff::'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:8000:0'))); -SELECT '49' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:c000:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:e000:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f000:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f800:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fc00:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fe00:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff00:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff80:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffc0:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffe0:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff0:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff8:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffc:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffe:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:8000'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:c000'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:e000'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f000'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f800'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fc00'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fe00'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff00'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff80'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffc0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffe0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff0'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff8'))); -SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffc'))); -SELECT '127' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe'))); -SELECT '127' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('::0'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('8000::'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('c000::'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('e000::'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('f000::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('f800::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fc00::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fe00::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff00::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ff80::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffc0::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffe0::'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff0::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fff8::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffc::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('fffe::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:8000::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:c000::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:e000::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f000::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:f800::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fc00::'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fe00::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff00::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ff80::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffc0::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffe0::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff0::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fff8::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffc::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:fffe::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:8000::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:c000::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:e000::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f000::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:f800::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fc00::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fe00::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff00::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ff80::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffc0::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffe0::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff0::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fff8::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffc::'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:fffe::'))); +SELECT '48' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:8000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:c000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:e000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:f800::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fc00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fe00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ff80::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffc0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffe0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fff8::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffc::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:fffe::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:8000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:c000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:e000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:f800::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fc00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fe00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ff80::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffc0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffe0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fff8::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffc::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:fffe::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:8000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:c000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:e000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f000::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:f800::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fc00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fe00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff00::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ff80::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffc0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffe0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff0::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fff8::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffc::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:fffe::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff::'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:8000:0'))); +SELECT '49' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:c000:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:e000:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f000:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:f800:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fc00:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fe00:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff00:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ff80:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffc0:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffe0:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff0:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fff8:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffc:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:fffe:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:8000'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:c000'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:e000'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f000'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:f800'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fc00'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fe00'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff00'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ff80'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffc0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffe0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff0'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fff8'))); +SELECT '99' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffc'))); +SELECT '127' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe'))); +SELECT '127' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('0.0.0.0'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('128.0.0.0'))); -SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('240.0.0.0'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('248.0.0.0'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('252.0.0.0'))); -SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.240.0.0'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.248.0.0'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.252.0.0'))); -SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.254.0'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.0'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.128'))); -SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.248'))); -SELECT '30' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.252'))); -SELECT '30' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.254'))); -SELECT '30' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.255'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('0.0.0.0'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('128.0.0.0'))); +SELECT '' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('240.0.0.0'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('248.0.0.0'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('252.0.0.0'))); +SELECT '5' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.240.0.0'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.248.0.0'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.252.0.0'))); +SELECT '13' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.254.0'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.0'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.128'))); +SELECT '24' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.248'))); +SELECT '30' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.252'))); +SELECT '30' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.254'))); +SELECT '30' == dictGetString({CLICKHOUSE_DATABASE:String} || '.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.255'))); -DROP DATABASE IF EXISTS database_for_dict; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/01033_dictionaries_lifetime.sql b/tests/queries/0_stateless/01033_dictionaries_lifetime.sql index 67e1adf5574..e74ac8bde30 100644 --- a/tests/queries/0_stateless/01033_dictionaries_lifetime.sql +++ b/tests/queries/0_stateless/01033_dictionaries_lifetime.sql @@ -1,12 +1,7 @@ --- Tags: no-parallel SET send_logs_level = 'fatal'; -DROP DATABASE IF EXISTS database_for_dict; - -CREATE DATABASE database_for_dict; - -CREATE TABLE database_for_dict.table_for_dict +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict ( key_column UInt64, second_column UInt8, @@ -15,34 +10,34 @@ CREATE TABLE database_for_dict.table_for_dict ENGINE = MergeTree() ORDER BY key_column; -INSERT INTO database_for_dict.table_for_dict VALUES (1, 100, 'Hello world'); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict VALUES (1, 100, 'Hello world'); -DROP DATABASE IF EXISTS ordinary_db; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; -CREATE DATABASE ordinary_db; +CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; -CREATE DICTIONARY ordinary_db.dict1 +CREATE DICTIONARY {CLICKHOUSE_DATABASE_1:Identifier}.dict1 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, third_column String DEFAULT 'qqq' ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' PASSWORD '' DB currentDatabase())) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); SELECT 'INITIALIZING DICTIONARY'; -SELECT dictGetUInt8('ordinary_db.dict1', 'second_column', toUInt64(100500)); +SELECT dictGetUInt8({CLICKHOUSE_DATABASE_1:String}||'.dict1', 'second_column', toUInt64(100500)); -SELECT lifetime_min, lifetime_max FROM system.dictionaries WHERE database='ordinary_db' AND name = 'dict1'; +SELECT lifetime_min, lifetime_max FROM system.dictionaries WHERE database={CLICKHOUSE_DATABASE_1:String} AND name = 'dict1'; -DROP DICTIONARY IF EXISTS ordinary_db.dict1; +DROP DICTIONARY IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}.dict1; -DROP DATABASE IF EXISTS ordinary_db; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; -DROP TABLE IF EXISTS database_for_dict.table_for_dict; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.table_for_dict; -DROP DATABASE IF EXISTS database_for_dict; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql b/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql index 9040d7b3231..bbd848f81d2 100644 --- a/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql +++ b/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql @@ -1,35 +1,31 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS dict_db_01036; -CREATE DATABASE dict_db_01036; - -CREATE TABLE dict_db_01036.dict_data (key UInt64, val UInt64) Engine=Memory(); -CREATE DICTIONARY dict_db_01036.dict +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.dict_data (key UInt64, val UInt64) Engine=Memory(); +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict ( key UInt64 DEFAULT 0, val UInt64 DEFAULT 10 ) PRIMARY KEY key -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB 'dict_db_01036')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB currentDatabase())) LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()); -SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; -SELECT dictGetUInt64('dict_db_01036.dict', 'val', toUInt64(0)); -SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; +SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; +SELECT dictGetUInt64({CLICKHOUSE_DATABASE:String}||'.dict', 'val', toUInt64(0)); +SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; SELECT 'SYSTEM RELOAD DICTIONARY'; -SYSTEM RELOAD DICTIONARY dict_db_01036.dict; -SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; -SELECT dictGetUInt64('dict_db_01036.dict', 'val', toUInt64(0)); -SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; +SYSTEM RELOAD DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict; +SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; +SELECT dictGetUInt64({CLICKHOUSE_DATABASE:String}||'.dict', 'val', toUInt64(0)); +SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; SELECT 'CREATE DATABASE'; -DROP DATABASE IF EXISTS empty_db_01036; -CREATE DATABASE empty_db_01036; -SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; +CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; +SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; -DROP DICTIONARY dict_db_01036.dict; -DROP TABLE dict_db_01036.dict_data; -DROP DATABASE dict_db_01036; -DROP DATABASE empty_db_01036; +DROP DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.dict_data; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; +DROP DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; diff --git a/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql b/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql index ad364237544..ab0b5a243ba 100644 --- a/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql +++ b/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql @@ -1,10 +1,5 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS dictdb; - -CREATE DATABASE dictdb; - -CREATE TABLE dictdb.table_for_dict +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict ( key_column UInt64, value Float64 @@ -12,33 +7,32 @@ CREATE TABLE dictdb.table_for_dict ENGINE = MergeTree() ORDER BY key_column; -INSERT INTO dictdb.table_for_dict VALUES (1, 1.1); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict VALUES (1, 1.1); -CREATE DICTIONARY IF NOT EXISTS dictdb.dict_exists +CREATE DICTIONARY IF NOT EXISTS {CLICKHOUSE_DATABASE:Identifier}.dict_exists ( key_column UInt64, value Float64 DEFAULT 77.77 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'dictdb')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase())) LIFETIME(1) LAYOUT(FLAT()); -SELECT dictGetFloat64('dictdb.dict_exists', 'value', toUInt64(1)); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict_exists', 'value', toUInt64(1)); -CREATE DICTIONARY IF NOT EXISTS dictdb.dict_exists +CREATE DICTIONARY IF NOT EXISTS {CLICKHOUSE_DATABASE:Identifier}.dict_exists ( key_column UInt64, value Float64 DEFAULT 77.77 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'dictdb')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase())) LIFETIME(1) LAYOUT(FLAT()); -SELECT dictGetFloat64('dictdb.dict_exists', 'value', toUInt64(1)); +SELECT dictGetFloat64({CLICKHOUSE_DATABASE:String} || '.dict_exists', 'value', toUInt64(1)); -DROP DICTIONARY dictdb.dict_exists; -DROP TABLE dictdb.table_for_dict; -DROP DATABASE dictdb; +DROP DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict_exists; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict; diff --git a/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql b/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql index 5d629d4e8db..4f078499972 100644 --- a/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql +++ b/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql @@ -1,13 +1,9 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS dictdb_01043; -CREATE DATABASE dictdb_01043; - -CREATE TABLE dictdb_01043.dicttbl(key Int64, value_default String, value_expression String) ENGINE = MergeTree ORDER BY tuple(); -INSERT INTO dictdb_01043.dicttbl VALUES (12, 'hello', '55:66:77'); +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.dicttbl(key Int64, value_default String, value_expression String) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.dicttbl VALUES (12, 'hello', '55:66:77'); -CREATE DICTIONARY dictdb_01043.dict +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict ( key Int64 DEFAULT -1, value_default String DEFAULT 'world', @@ -15,15 +11,13 @@ CREATE DICTIONARY dictdb_01043.dict ) PRIMARY KEY key -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dicttbl' DB 'dictdb_01043')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dicttbl' DB currentDatabase())) LAYOUT(FLAT()) LIFETIME(1); -SELECT dictGetString('dictdb_01043.dict', 'value_default', toUInt64(12)); -SELECT dictGetString('dictdb_01043.dict', 'value_default', toUInt64(14)); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict', 'value_default', toUInt64(12)); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict', 'value_default', toUInt64(14)); -SELECT dictGetString('dictdb_01043.dict', 'value_expression', toUInt64(12)); -SELECT dictGetString('dictdb_01043.dict', 'value_expression', toUInt64(14)); - -DROP DATABASE IF EXISTS dictdb_01043; +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict', 'value_expression', toUInt64(12)); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.dict', 'value_expression', toUInt64(14)); diff --git a/tests/queries/0_stateless/01045_dictionaries_restrictions.sql b/tests/queries/0_stateless/01045_dictionaries_restrictions.sql index d41be7482f8..b4dbd741767 100644 --- a/tests/queries/0_stateless/01045_dictionaries_restrictions.sql +++ b/tests/queries/0_stateless/01045_dictionaries_restrictions.sql @@ -1,10 +1,5 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS dictdb_01045; - -CREATE DATABASE dictdb_01045; - -CREATE DICTIONARY dictdb_01045.restricted_dict ( +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.restricted_dict ( key UInt64, value String ) @@ -14,10 +9,9 @@ LIFETIME(MIN 0 MAX 1) LAYOUT(CACHE(SIZE_IN_CELLS 10)); -- because of lazy load we can check only in dictGet query -select dictGetString('dictdb_01045.restricted_dict', 'value', toUInt64(1)); -- {serverError 482} +select dictGetString({CLICKHOUSE_DATABASE:String} || '.restricted_dict', 'value', toUInt64(1)); -- {serverError 482} select 'Ok.'; -DROP DICTIONARY IF EXISTS dictdb_01045.restricted_dict; +DROP DICTIONARY IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.restricted_dict; -DROP DATABASE IF EXISTS dictdb_01045; diff --git a/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference b/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference index 53df91c5523..bb4fb9ddb47 100644 --- a/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference +++ b/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference @@ -1,32 +1,32 @@ ---TUMBLE--- ||---WINDOW COLUMN NAME--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---DATA COLUMN ALIAS--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY b\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY b\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---PARTITION--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `count(a)` AggregateFunction(count, Int32),\n `windowID(____timestamp, toIntervalSecond(\'1\'))` UInt32\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `windowID(____timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(____timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `count(a)` AggregateFunction(count, Int32),\n `windowID(____timestamp, toIntervalSecond(\'1\'))` UInt32\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `windowID(____timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(____timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---JOIN--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---DATA COLUMN ALIAS--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY b\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY b\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---PARTITION--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `count(a)` AggregateFunction(count, Int32),\n `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `count(a)` AggregateFunction(count, Int32),\n `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---JOIN--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql index bf1ac254783..e292447512c 100644 --- a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql +++ b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql @@ -1,110 +1,109 @@ --- Tags: no-parallel SET allow_experimental_analyzer = 0; SET allow_experimental_window_view = 1; -DROP DATABASE IF EXISTS test_01047; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; set allow_deprecated_database_ordinary=1; -CREATE DATABASE test_01047 ENGINE=Ordinary; +CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier} ENGINE=Ordinary; -DROP TABLE IF EXISTS test_01047.mt; -DROP TABLE IF EXISTS test_01047.mt_2; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.mt; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.mt_2; -CREATE TABLE test_01047.mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE TABLE test_01047.mt_2(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.mt_2(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); SELECT '---TUMBLE---'; SELECT '||---WINDOW COLUMN NAME---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY tumble(timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(a), tumbleEnd(wid) AS count FROM test_01047.mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND) as wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY tumble(timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(a), tumbleEnd(wid) AS count FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND) as wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count(a) AS count, tumble(timestamp, INTERVAL '1' SECOND) AS wid FROM test_01047.mt GROUP BY wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count(a) AS count, tumble(timestamp, INTERVAL '1' SECOND) AS wid FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---DATA COLUMN ALIAS---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY id ENGINE Memory AS SELECT count(a) AS count, b as id FROM test_01047.mt GROUP BY id, tumble(timestamp, INTERVAL '1' SECOND); -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY id ENGINE Memory AS SELECT count(a) AS count, b as id FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY id, tumble(timestamp, INTERVAL '1' SECOND); +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---IDENTIFIER---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY (tumble(timestamp, INTERVAL '1' SECOND), b) PRIMARY KEY tumble(timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM test_01047.mt GROUP BY b, tumble(timestamp, INTERVAL '1' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY (tumble(timestamp, INTERVAL '1' SECOND), b) PRIMARY KEY tumble(timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY b, tumble(timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---FUNCTION---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY (tumble(timestamp, INTERVAL '1' SECOND), plus(a, b)) PRIMARY KEY tumble(timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM test_01047.mt GROUP BY plus(a, b) as _type, tumble(timestamp, INTERVAL '1' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY (tumble(timestamp, INTERVAL '1' SECOND), plus(a, b)) PRIMARY KEY tumble(timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY plus(a, b) as _type, tumble(timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---PARTITION---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid ENGINE Memory AS SELECT count(a) AS count, tumble(now(), INTERVAL '1' SECOND) AS wid FROM test_01047.mt GROUP BY wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid ENGINE Memory AS SELECT count(a) AS count, tumble(now(), INTERVAL '1' SECOND) AS wid FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---JOIN---'; -DROP TABLE IF EXISTS test_01047.wv; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY tumble(test_01047.mt.timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count(test_01047.mt.a), count(test_01047.mt_2.b), wid FROM test_01047.mt JOIN test_01047.mt_2 ON test_01047.mt.timestamp = test_01047.mt_2.timestamp GROUP BY tumble(test_01047.mt.timestamp, INTERVAL '1' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY tumble({CLICKHOUSE_DATABASE:Identifier}.mt.timestamp, INTERVAL '1' SECOND) ENGINE Memory AS SELECT count({CLICKHOUSE_DATABASE:Identifier}.mt.a), count({CLICKHOUSE_DATABASE:Identifier}.mt_2.b), wid FROM {CLICKHOUSE_DATABASE:Identifier}.mt JOIN {CLICKHOUSE_DATABASE:Identifier}.mt_2 ON {CLICKHOUSE_DATABASE:Identifier}.mt.timestamp = {CLICKHOUSE_DATABASE:Identifier}.mt_2.timestamp GROUP BY tumble({CLICKHOUSE_DATABASE:Identifier}.mt.timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; -DROP TABLE IF EXISTS test_01047.wv; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count(test_01047.mt.a), count(test_01047.mt_2.b), wid FROM test_01047.mt JOIN test_01047.mt_2 ON test_01047.mt.timestamp = test_01047.mt_2.timestamp GROUP BY tumble(test_01047.mt.timestamp, INTERVAL '1' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count({CLICKHOUSE_DATABASE:Identifier}.mt.a), count({CLICKHOUSE_DATABASE:Identifier}.mt_2.b), wid FROM {CLICKHOUSE_DATABASE:Identifier}.mt JOIN {CLICKHOUSE_DATABASE:Identifier}.mt_2 ON {CLICKHOUSE_DATABASE:Identifier}.mt.timestamp = {CLICKHOUSE_DATABASE:Identifier}.mt_2.timestamp GROUP BY tumble({CLICKHOUSE_DATABASE:Identifier}.mt.timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '---HOP---'; SELECT '||---WINDOW COLUMN NAME---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(a) AS count, hopEnd(wid) FROM test_01047.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(a) AS count, hopEnd(wid) FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count(a) AS count, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM test_01047.mt GROUP BY wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count(a) AS count, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---DATA COLUMN ALIAS---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY id ENGINE Memory AS SELECT count(a) AS count, b as id FROM test_01047.mt GROUP BY id, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND); -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY id ENGINE Memory AS SELECT count(a) AS count, b as id FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY id, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND); +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---IDENTIFIER---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY (hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), b) PRIMARY KEY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM test_01047.mt GROUP BY b, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY (hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), b) PRIMARY KEY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY b, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---FUNCTION---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY (hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), plus(a, b)) PRIMARY KEY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM test_01047.mt GROUP BY plus(a, b) as _type, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY (hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), plus(a, b)) PRIMARY KEY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(a) AS count FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY plus(a, b) as _type, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---PARTITION---'; -DROP TABLE IF EXISTS test_01047.wv; -DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid ENGINE Memory AS SELECT count(a) AS count, hopEnd(wid) FROM test_01047.mt GROUP BY hop(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid ENGINE Memory AS SELECT count(a) AS count, hopEnd(wid) FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY hop(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; SELECT '||---JOIN---'; -DROP TABLE IF EXISTS test_01047.wv; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY hop(test_01047.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count(test_01047.mt.a), count(test_01047.mt_2.b), wid FROM test_01047.mt JOIN test_01047.mt_2 ON test_01047.mt.timestamp = test_01047.mt_2.timestamp GROUP BY hop(test_01047.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY hop({CLICKHOUSE_DATABASE:Identifier}.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) ENGINE Memory AS SELECT count({CLICKHOUSE_DATABASE:Identifier}.mt.a), count({CLICKHOUSE_DATABASE:Identifier}.mt_2.b), wid FROM {CLICKHOUSE_DATABASE:Identifier}.mt JOIN {CLICKHOUSE_DATABASE:Identifier}.mt_2 ON {CLICKHOUSE_DATABASE:Identifier}.mt.timestamp = {CLICKHOUSE_DATABASE:Identifier}.mt_2.timestamp GROUP BY hop({CLICKHOUSE_DATABASE:Identifier}.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; -DROP TABLE IF EXISTS test_01047.wv; -CREATE WINDOW VIEW test_01047.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count(test_01047.mt.a), count(test_01047.mt_2.b), wid FROM test_01047.mt JOIN test_01047.mt_2 ON test_01047.mt.timestamp = test_01047.mt_2.timestamp GROUP BY hop(test_01047.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; -SHOW CREATE TABLE test_01047.`.inner.wv`; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv INNER ENGINE AggregatingMergeTree ORDER BY wid ENGINE Memory AS SELECT count({CLICKHOUSE_DATABASE:Identifier}.mt.a), count({CLICKHOUSE_DATABASE:Identifier}.mt_2.b), wid FROM {CLICKHOUSE_DATABASE:Identifier}.mt JOIN {CLICKHOUSE_DATABASE:Identifier}.mt_2 ON {CLICKHOUSE_DATABASE:Identifier}.mt.timestamp = {CLICKHOUSE_DATABASE:Identifier}.mt_2.timestamp GROUP BY hop({CLICKHOUSE_DATABASE:Identifier}.mt.timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.`.inner.wv`; -DROP TABLE test_01047.wv; -DROP TABLE test_01047.mt; -DROP TABLE test_01047.mt_2; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.wv; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.mt; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.mt_2; diff --git a/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.sql b/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.sql index ec440baf8d6..a790f384f7a 100644 --- a/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.sql +++ b/tests/queries/0_stateless/01050_clickhouse_dict_source_with_subquery.sql @@ -1,18 +1,17 @@ --- Tags: no-parallel -drop dictionary if exists default.test_dict_01051_d; -drop table if exists default.test_01051_d; -drop table if exists default.test_view_01051_d; +drop dictionary if exists {CLICKHOUSE_DATABASE:Identifier}.test_dict_01051_d; +drop table if exists {CLICKHOUSE_DATABASE:Identifier}.test_01051_d; +drop table if exists {CLICKHOUSE_DATABASE:Identifier}.test_view_01051_d; -create table default.test_01051_d (key UInt64, value String) engine = MergeTree order by key; -create view default.test_view_01051_d (key UInt64, value String) as select k2 + 1 as key, v2 || '_x' as value from (select key + 2 as k2, value || '_y' as v2 from default.test_01051_d); +create table {CLICKHOUSE_DATABASE:Identifier}.test_01051_d (key UInt64, value String) engine = MergeTree order by key; +create view {CLICKHOUSE_DATABASE:Identifier}.test_view_01051_d (key UInt64, value String) as select k2 + 1 as key, v2 || '_x' as value from (select key + 2 as k2, value || '_y' as v2 from test_01051_d); -insert into default.test_01051_d values (1, 'a'); +insert into {CLICKHOUSE_DATABASE:Identifier}.test_01051_d values (1, 'a'); -create dictionary default.test_dict_01051_d (key UInt64, value String) primary key key source(clickhouse(host 'localhost' port '9000' user 'default' password '' db 'default' table 'test_view_01051_d')) layout(flat()) lifetime(100500); +create dictionary {CLICKHOUSE_DATABASE:Identifier}.test_dict_01051_d (key UInt64, value String) primary key key source(clickhouse(host 'localhost' port '9000' user 'default' password '' db currentDatabase() table 'test_view_01051_d')) layout(flat()) lifetime(100500); -select dictGet('default.test_dict_01051_d', 'value', toUInt64(4)); +select dictGet({CLICKHOUSE_DATABASE:String} || '.test_dict_01051_d', 'value', toUInt64(4)); -drop dictionary if exists default.test_dict_01051_d; -drop table if exists default.test_01051_d; -drop table if exists default.test_view_01051_d; +drop dictionary if exists {CLICKHOUSE_DATABASE:Identifier}.test_dict_01051_d; +drop table if exists {CLICKHOUSE_DATABASE:Identifier}.test_01051_d; +drop table if exists {CLICKHOUSE_DATABASE:Identifier}.test_view_01051_d; diff --git a/tests/queries/0_stateless/01053_drop_database_mat_view.sql b/tests/queries/0_stateless/01053_drop_database_mat_view.sql index 7651ac4885c..2642430eb05 100644 --- a/tests/queries/0_stateless/01053_drop_database_mat_view.sql +++ b/tests/queries/0_stateless/01053_drop_database_mat_view.sql @@ -1,14 +1,13 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS some_tests; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; set allow_deprecated_database_ordinary=1; -CREATE DATABASE some_tests ENGINE=Ordinary; -- Different inner table name with Atomic +CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier} ENGINE=Ordinary; -- Different inner table name with Atomic set allow_deprecated_syntax_for_merge_tree=1; -create table some_tests.my_table ENGINE = MergeTree(day, (day), 8192) as select today() as day, 'mystring' as str; -show tables from some_tests; -create materialized view some_tests.my_materialized_view ENGINE = MergeTree(day, (day), 8192) as select * from some_tests.my_table; -show tables from some_tests; -select * from some_tests.my_materialized_view; +create table {CLICKHOUSE_DATABASE:Identifier}.my_table ENGINE = MergeTree(day, (day), 8192) as select today() as day, 'mystring' as str; +show tables from {CLICKHOUSE_DATABASE:Identifier}; +create materialized view {CLICKHOUSE_DATABASE:Identifier}.my_materialized_view ENGINE = MergeTree(day, (day), 8192) as select * from {CLICKHOUSE_DATABASE:Identifier}.my_table; +show tables from {CLICKHOUSE_DATABASE:Identifier}; +select * from {CLICKHOUSE_DATABASE:Identifier}.my_materialized_view; -DROP DATABASE some_tests; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/01056_create_table_as.sql b/tests/queries/0_stateless/01056_create_table_as.sql index 6df660dba61..aa2dffb6e2d 100644 --- a/tests/queries/0_stateless/01056_create_table_as.sql +++ b/tests/queries/0_stateless/01056_create_table_as.sql @@ -1,4 +1,3 @@ --- Tags: no-parallel DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; @@ -23,9 +22,9 @@ DROP TABLE v; -- dictionary DROP DICTIONARY IF EXISTS dict; -DROP DATABASE if exists test_01056_dict_data; -CREATE DATABASE test_01056_dict_data; -CREATE TABLE test_01056_dict_data.dict_data (key Int, value UInt16) Engine=Memory(); +DROP DATABASE if exists {CLICKHOUSE_DATABASE_1:Identifier}; +CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; +CREATE TABLE {CLICKHOUSE_DATABASE_1:Identifier}.dict_data (key Int, value UInt16) Engine=Memory(); CREATE DICTIONARY dict ( `key` UInt64, @@ -34,7 +33,7 @@ CREATE DICTIONARY dict PRIMARY KEY key SOURCE(CLICKHOUSE( HOST '127.0.0.1' PORT tcpPort() - TABLE 'dict_data' DB 'test_01056_dict_data' USER 'default' PASSWORD '')) + TABLE 'dict_data' DB concat(currentDatabase(), '_1') USER 'default' PASSWORD '')) LIFETIME(MIN 0 MAX 0) LAYOUT(SPARSE_HASHED()); CREATE TABLE t3 AS dict; -- { serverError 80 } @@ -42,9 +41,9 @@ CREATE TABLE t3 AS dict; -- { serverError 80 } DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t3; DROP DICTIONARY dict; -DROP TABLE test_01056_dict_data.dict_data; +DROP TABLE {CLICKHOUSE_DATABASE_1:Identifier}.dict_data; -DROP DATABASE test_01056_dict_data; +DROP DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; CREATE TABLE t1 (x String) ENGINE = Memory AS SELECT 1; SELECT x, toTypeName(x) FROM t1; diff --git a/tests/queries/0_stateless/01071_prohibition_secondary_index_with_old_format_merge_tree.sql b/tests/queries/0_stateless/01071_prohibition_secondary_index_with_old_format_merge_tree.sql index 992973c97e8..f92b6779587 100644 --- a/tests/queries/0_stateless/01071_prohibition_secondary_index_with_old_format_merge_tree.sql +++ b/tests/queries/0_stateless/01071_prohibition_secondary_index_with_old_format_merge_tree.sql @@ -1,4 +1,3 @@ --- Tags: no-parallel set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE old_syntax_01071_test (date Date, id UInt8) ENGINE = MergeTree(date, id, 8192); diff --git a/tests/queries/0_stateless/01073_attach_if_not_exists.sql b/tests/queries/0_stateless/01073_attach_if_not_exists.sql index 8bd356b466c..a99d5fb5041 100644 --- a/tests/queries/0_stateless/01073_attach_if_not_exists.sql +++ b/tests/queries/0_stateless/01073_attach_if_not_exists.sql @@ -1,4 +1,3 @@ --- Tags: no-parallel CREATE TABLE aine (a Int) ENGINE = Log; ATTACH TABLE aine; -- { serverError 57 } diff --git a/tests/queries/0_stateless/01073_show_tables_not_like.sql b/tests/queries/0_stateless/01073_show_tables_not_like.sql index 405a6cb199b..9ff2afe7f27 100644 --- a/tests/queries/0_stateless/01073_show_tables_not_like.sql +++ b/tests/queries/0_stateless/01073_show_tables_not_like.sql @@ -1,11 +1,6 @@ --- Tags: no-parallel SHOW TABLES NOT LIKE '%'; -DROP DATABASE IF EXISTS test_01073; -CREATE DATABASE test_01073; -USE test_01073; - SHOW TABLES; SELECT '---'; CREATE TABLE test1 (x UInt8) ENGINE = Memory; @@ -22,14 +17,14 @@ SELECT '--'; SHOW TABLES NOT LIKE 'tes%2'; SELECT '---'; -SHOW TABLES FROM test_01073; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier}; SELECT '--'; -SHOW TABLES FROM test_01073 LIKE 'tes%'; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier} LIKE 'tes%'; SELECT '--'; -SHOW TABLES FROM test_01073 NOT LIKE 'tes%'; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier} NOT LIKE 'tes%'; SELECT '--'; -SHOW TABLES FROM test_01073 LIKE 'tes%1'; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier} LIKE 'tes%1'; SELECT '--'; -SHOW TABLES FROM test_01073 NOT LIKE 'tes%2'; +SHOW TABLES FROM {CLICKHOUSE_DATABASE:Identifier} NOT LIKE 'tes%2'; -DROP DATABASE test_01073; +DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; diff --git a/tests/queries/0_stateless/01080_check_for_error_incorrect_size_of_nested_column.sql b/tests/queries/0_stateless/01080_check_for_error_incorrect_size_of_nested_column.sql index 1eb6e849851..d7b05bb7d78 100644 --- a/tests/queries/0_stateless/01080_check_for_error_incorrect_size_of_nested_column.sql +++ b/tests/queries/0_stateless/01080_check_for_error_incorrect_size_of_nested_column.sql @@ -1,40 +1,33 @@ --- Tags: no-parallel --- TODO: can't just remove default prefix, it breaks the test! +drop table if exists {CLICKHOUSE_DATABASE:Identifier}.test_table_01080; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.test_table_01080 (dim_key Int64, dim_id String) ENGINE = MergeTree Order by (dim_key); +insert into {CLICKHOUSE_DATABASE:Identifier}.test_table_01080 values(1,'test1'); -drop database if exists db_01080; -create database db_01080; +drop DICTIONARY if exists {CLICKHOUSE_DATABASE:Identifier}.test_dict_01080; -drop table if exists db_01080.test_table_01080; -CREATE TABLE db_01080.test_table_01080 (dim_key Int64, dim_id String) ENGINE = MergeTree Order by (dim_key); -insert into db_01080.test_table_01080 values(1,'test1'); - -drop DICTIONARY if exists db_01080.test_dict_01080; - -CREATE DICTIONARY db_01080.test_dict_01080 ( dim_key Int64, dim_id String ) +CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.test_dict_01080 ( dim_key Int64, dim_id String ) PRIMARY KEY dim_key -source(clickhouse(host 'localhost' port tcpPort() user 'default' password '' db 'db_01080' table 'test_table_01080')) +source(clickhouse(host 'localhost' port tcpPort() user 'default' password '' db currentDatabase() table 'test_table_01080')) LIFETIME(MIN 0 MAX 0) LAYOUT(complex_key_hashed()); -SELECT dictGetString('db_01080.test_dict_01080', 'dim_id', tuple(toInt64(1))); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', tuple(toInt64(1))); -SELECT dictGetString('db_01080.test_dict_01080', 'dim_id', tuple(toInt64(0))); +SELECT dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', tuple(toInt64(0))); -select dictGetString('db_01080.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(0)) as x); +select dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(0)) as x); -select dictGetString('db_01080.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(1)) as x); +select dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(1)) as x); -select dictGetString('db_01080.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(number)) as x from numbers(5)); +select dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(number)) as x from numbers(5)); -select dictGetString('db_01080.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(rand64()*0)) as x); +select dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(rand64()*0)) as x); -select dictGetString('db_01080.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(blockSize()=0)) as x); +select dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(blockSize()=0)) as x); -select dictGetString('db_01080.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(materialize(0))) as x); +select dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(materialize(0))) as x); -select dictGetString('db_01080.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(materialize(1))) as x); +select dictGetString({CLICKHOUSE_DATABASE:String} || '.test_dict_01080', 'dim_id', x) from (select tuple(toInt64(materialize(1))) as x); -drop DICTIONARY db_01080.test_dict_01080; -drop table db_01080.test_table_01080; -drop database db_01080; +drop DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.test_dict_01080; +drop table {CLICKHOUSE_DATABASE:Identifier}.test_table_01080; diff --git a/tests/queries/0_stateless/01084_regexp_empty.sql b/tests/queries/0_stateless/01084_regexp_empty.sql index 3ccd4af80ab..5dd060ab58c 100644 --- a/tests/queries/0_stateless/01084_regexp_empty.sql +++ b/tests/queries/0_stateless/01084_regexp_empty.sql @@ -1,10 +1,9 @@ --- Tags: no-parallel -DROP DATABASE IF EXISTS test_01084; -CREATE DATABASE test_01084; -USE test_01084; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; +CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; +USE {CLICKHOUSE_DATABASE_1:Identifier}; CREATE TABLE t (x UInt8) ENGINE = Memory; SELECT * FROM merge('', ''); -DROP DATABASE test_01084; +DROP DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; diff --git a/tests/queries/0_stateless/01085_window_view_attach.sql b/tests/queries/0_stateless/01085_window_view_attach.sql index 051557a6a76..51a88a04f95 100644 --- a/tests/queries/0_stateless/01085_window_view_attach.sql +++ b/tests/queries/0_stateless/01085_window_view_attach.sql @@ -1,30 +1,29 @@ --- Tags: no-parallel SET allow_experimental_analyzer = 0; SET allow_experimental_window_view = 1; -DROP DATABASE IF EXISTS test_01085; +DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; set allow_deprecated_database_ordinary=1; -CREATE DATABASE test_01085 ENGINE=Ordinary; +CREATE DATABASE {CLICKHOUSE_DATABASE:Identifier} ENGINE=Ordinary; -DROP TABLE IF EXISTS test_01085.mt; -DROP TABLE IF EXISTS test_01085.wv; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.mt; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wv; -CREATE TABLE test_01085.mt(a Int32, market Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test_01085.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, market, tumbleEnd(wid) AS w_end FROM test_01085.mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND) AS wid, market; +CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.mt(a Int32, market Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, market, tumbleEnd(wid) AS w_end FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND) AS wid, market; -SHOW tables FROM test_01085; +SHOW tables FROM {CLICKHOUSE_DATABASE:Identifier}; -DROP TABLE test_01085.wv SYNC; -SHOW tables FROM test_01085; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.wv SYNC; +SHOW tables FROM {CLICKHOUSE_DATABASE:Identifier}; -CREATE WINDOW VIEW test_01085.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, market, tumbleEnd(wid) AS w_end FROM test_01085.mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND) AS wid, market; +CREATE WINDOW VIEW {CLICKHOUSE_DATABASE:Identifier}.wv ENGINE Memory WATERMARK=ASCENDING AS SELECT count(a) AS count, market, tumbleEnd(wid) AS w_end FROM {CLICKHOUSE_DATABASE:Identifier}.mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND) AS wid, market; -DETACH TABLE test_01085.wv; -SHOW tables FROM test_01085; +DETACH TABLE {CLICKHOUSE_DATABASE:Identifier}.wv; +SHOW tables FROM {CLICKHOUSE_DATABASE:Identifier}; -ATTACH TABLE test_01085.wv; -SHOW tables FROM test_01085; +ATTACH TABLE {CLICKHOUSE_DATABASE:Identifier}.wv; +SHOW tables FROM {CLICKHOUSE_DATABASE:Identifier}; -DROP TABLE test_01085.wv SYNC; -SHOW tables FROM test_01085; +DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.wv SYNC; +SHOW tables FROM {CLICKHOUSE_DATABASE:Identifier}; From ba44d7260e12ba6e32ff1033cf9538ce9abe0645 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 16 Aug 2023 00:20:28 +0200 Subject: [PATCH 584/777] fix --- programs/keeper/Keeper.cpp | 2 -- src/Access/LDAPClient.cpp | 4 ++-- src/Formats/ProtobufReader.cpp | 10 ++++------ src/Processors/Merges/Algorithms/Graphite.cpp | 8 ++------ tests/integration/test_grpc_protocol/test.py | 5 ++++- 5 files changed, 12 insertions(+), 17 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 77fbc9430ef..4791f42312e 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -110,8 +110,6 @@ void Keeper::createServer(const std::string & listen_host, const char * port_nam } catch (const Poco::Exception &) { - std::string message = "Listen [" + listen_host + "]:" + std::to_string(port) + " failed: " + getCurrentExceptionMessage(false); - if (listen_try) { LOG_WARNING(&logger(), "Listen [{}]:{} failed: {}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, " diff --git a/src/Access/LDAPClient.cpp b/src/Access/LDAPClient.cpp index ac2f1683f0c..2af779aa9ae 100644 --- a/src/Access/LDAPClient.cpp +++ b/src/Access/LDAPClient.cpp @@ -549,7 +549,7 @@ LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params) if (rc != LDAP_SUCCESS) { - String message = "LDAP search failed"; + String message; const char * raw_err_str = ldap_err2string(rc); if (raw_err_str && *raw_err_str != '\0') @@ -570,7 +570,7 @@ LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params) message += matched_msg; } - throw Exception::createDeprecated(message, ErrorCodes::LDAP_ERROR); + throw Exception(ErrorCodes::LDAP_ERROR, "LDAP search failed{}", message); } break; diff --git a/src/Formats/ProtobufReader.cpp b/src/Formats/ProtobufReader.cpp index 159a1d33c49..577342bf299 100644 --- a/src/Formats/ProtobufReader.cpp +++ b/src/Formats/ProtobufReader.cpp @@ -429,12 +429,10 @@ void ProtobufReader::ignoreGroup() [[noreturn]] void ProtobufReader::throwUnknownFormat() const { - throw Exception::createDeprecated( - std::string("Protobuf messages are corrupted or don't match the provided schema.") - + (root_message_has_length_delimiter - ? " Please note that Protobuf stream is length-delimited: every message is prefixed by its length in varint." - : ""), - ErrorCodes::UNKNOWN_PROTOBUF_FORMAT); + throw Exception(ErrorCodes::UNKNOWN_PROTOBUF_FORMAT, "Protobuf messages are corrupted or don't match the provided schema.{}", + root_message_has_length_delimiter + ? " Please note that Protobuf stream is length-delimited: every message is prefixed by its length in varint." + : ""); } } diff --git a/src/Processors/Merges/Algorithms/Graphite.cpp b/src/Processors/Merges/Algorithms/Graphite.cpp index 418bf5e3f13..a3ff7a4ef87 100644 --- a/src/Processors/Merges/Algorithms/Graphite.cpp +++ b/src/Processors/Merges/Algorithms/Graphite.cpp @@ -163,12 +163,8 @@ static bool compareRetentions(const Retention & a, const Retention & b) { return false; } - String error_msg = "age and precision should only grow up: " - + std::to_string(a.age) + ":" + std::to_string(a.precision) + " vs " - + std::to_string(b.age) + ":" + std::to_string(b.precision); - throw Exception::createDeprecated( - error_msg, - DB::ErrorCodes::BAD_ARGUMENTS); + throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Age and precision should only grow up: {}:{} vs {}:{}", + a.age, a.precision, b.age, b.precision); } bool operator==(const Retention & a, const Retention & b) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index dead4d447ec..4a28dae900b 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -352,7 +352,10 @@ def test_authentication(): def test_logs(): - logs = query_and_get_logs("SELECT 1", settings={"send_logs_level": "debug"}) + logs = query_and_get_logs( + "SELECT has(groupArray(number), 42) FROM numbers(1000)", + settings={"send_logs_level": "debug"}, + ) assert "SELECT 1" in logs assert "Read 1 rows" in logs assert "Peak memory usage" in logs From 387ce81895d0d9a6a8e994bf24801b00dc3af049 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Aug 2023 00:46:53 +0200 Subject: [PATCH 585/777] Clean all containers properly --- tests/ci/install_check.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index 010b0dab408..700550bf077 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -191,6 +191,9 @@ def test_install(image: DockerImage, tests: Dict[str, str]) -> TestResults: retcode = process.wait() if retcode == 0: status = OK + subprocess.check_call( + f"docker kill -s 9 {container_id}", shell=True + ) break status = FAIL @@ -198,8 +201,8 @@ def test_install(image: DockerImage, tests: Dict[str, str]) -> TestResults: archive_path = TEMP_PATH / f"{container_name}-{retry}.tar.gz" compress_fast(LOGS_PATH, archive_path) logs.append(archive_path) + subprocess.check_call(f"docker kill -s 9 {container_id}", shell=True) - subprocess.check_call(f"docker kill -s 9 {container_id}", shell=True) test_results.append(TestResult(name, status, stopwatch.duration_seconds, logs)) return test_results From 790475385acc5b722460e5b9581f637ac6ff9b1e Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Aug 2023 00:47:39 +0200 Subject: [PATCH 586/777] Improve downloading: skip dbg, do not pull images on --no-download --- tests/ci/install_check.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index 700550bf077..2ca947192da 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -279,7 +279,7 @@ def main(): sys.exit(0) docker_images = { - name: get_image_with_version(REPORTS_PATH, name) + name: get_image_with_version(REPORTS_PATH, name, args.download) for name in (RPM_IMAGE, DEB_IMAGE) } prepare_test_scripts() @@ -296,6 +296,8 @@ def main(): is_match = is_match or path.endswith(".rpm") if args.tgz: is_match = is_match or path.endswith(".tgz") + # We don't need debug packages, so let's filter them out + is_match = is_match and "-dbg" not in path return is_match download_builds_filter( From 3cd9fa395d2d3483e9e71274076cf151ef8ff682 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Aug 2023 00:51:44 +0200 Subject: [PATCH 587/777] Add test for systemd + /etc/default/clickhouse --- tests/ci/install_check.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index 2ca947192da..b08e94c52b4 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -50,8 +50,11 @@ def prepare_test_scripts(): server_test = r"""#!/bin/bash set -e trap "bash -ex /packages/preserve_logs.sh" ERR +test_env='TEST_THE_DEFAULT_PARAMETER=15' +echo "$test_env" >> /etc/default/clickhouse systemctl start clickhouse-server -clickhouse-client -q 'SELECT version()'""" +clickhouse-client -q 'SELECT version()' +grep "$test_env" /proc/$(cat /var/run/clickhouse-server/clickhouse-server.pid)/environ""" keeper_test = r"""#!/bin/bash set -e trap "bash -ex /packages/preserve_logs.sh" ERR From 651a45b04d1cc4ec0b8be5b0fbb3068b09813fce Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Aug 2023 00:57:22 +0200 Subject: [PATCH 588/777] Add tests for initd start --- docker/test/install/deb/Dockerfile | 1 + tests/ci/install_check.py | 12 ++++++++++++ 2 files changed, 13 insertions(+) diff --git a/docker/test/install/deb/Dockerfile b/docker/test/install/deb/Dockerfile index 9614473c69b..e9c928b1fe7 100644 --- a/docker/test/install/deb/Dockerfile +++ b/docker/test/install/deb/Dockerfile @@ -12,6 +12,7 @@ ENV \ # install systemd packages RUN apt-get update && \ apt-get install -y --no-install-recommends \ + sudo \ systemd \ && \ apt-get clean && \ diff --git a/tests/ci/install_check.py b/tests/ci/install_check.py index b08e94c52b4..a5788e2af3f 100644 --- a/tests/ci/install_check.py +++ b/tests/ci/install_check.py @@ -54,6 +54,14 @@ test_env='TEST_THE_DEFAULT_PARAMETER=15' echo "$test_env" >> /etc/default/clickhouse systemctl start clickhouse-server clickhouse-client -q 'SELECT version()' +grep "$test_env" /proc/$(cat /var/run/clickhouse-server/clickhouse-server.pid)/environ""" + initd_test = r"""#!/bin/bash +set -e +trap "bash -ex /packages/preserve_logs.sh" ERR +test_env='TEST_THE_DEFAULT_PARAMETER=15' +echo "$test_env" >> /etc/default/clickhouse +/etc/init.d/clickhouse-server start +clickhouse-client -q 'SELECT version()' grep "$test_env" /proc/$(cat /var/run/clickhouse-server/clickhouse-server.pid)/environ""" keeper_test = r"""#!/bin/bash set -e @@ -105,6 +113,7 @@ chmod a+rw -R /tests_logs exit 1 """ (TEMP_PATH / "server_test.sh").write_text(server_test, encoding="utf-8") + (TEMP_PATH / "initd_test.sh").write_text(initd_test, encoding="utf-8") (TEMP_PATH / "keeper_test.sh").write_text(keeper_test, encoding="utf-8") (TEMP_PATH / "binary_test.sh").write_text(binary_test, encoding="utf-8") (TEMP_PATH / "preserve_logs.sh").write_text(preserve_logs, encoding="utf-8") @@ -115,6 +124,9 @@ def test_install_deb(image: DockerImage) -> TestResults: "Install server deb": r"""#!/bin/bash -ex apt-get install /packages/clickhouse-{server,client,common}*deb bash -ex /packages/server_test.sh""", + "Run server init.d": r"""#!/bin/bash -ex +apt-get install /packages/clickhouse-{server,client,common}*deb +bash -ex /packages/initd_test.sh""", "Install keeper deb": r"""#!/bin/bash -ex apt-get install /packages/clickhouse-keeper*deb bash -ex /packages/keeper_test.sh""", From 8ba1da755d2a6a407fda06c5aa88cc156b37d887 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 16 Aug 2023 11:18:44 +0800 Subject: [PATCH 589/777] update example --- src/Processors/examples/native_orc.cpp | 31 ++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/src/Processors/examples/native_orc.cpp b/src/Processors/examples/native_orc.cpp index 673dcd4c4e2..69a72dc7485 100644 --- a/src/Processors/examples/native_orc.cpp +++ b/src/Processors/examples/native_orc.cpp @@ -32,5 +32,36 @@ int main() auto schema = schema_reader.readSchema(); std::cout << "schema:" << schema.toString() << std::endl; } + + { + String content + = "\x4f\x52\x43\x0a\x0b\x0a\x03\x00\x00\x00\x12\x04\x08\x01\x50\x00\x0a\x15\x0a\x05\x00\x00\x00\x00\x00\x12\x0c\x08\x01\x12\x06" + "\x08\x00\x10\x00\x18\x00\x50\x00\x0a\x12\x0a\x06\x00\x00\x00\x00\x00\x00\x12\x08\x08\x01\x42\x02\x08\x06\x50\x00\x0a\x12\x0a" + "\x06\x00\x00\x00\x00\x00\x00\x12\x08\x08\x01\x42\x02\x08\x04\x50\x00\x0a\x29\x0a\x04\x00\x00\x00\x00\x12\x21\x08\x01\x1a\x1b" + "\x09\x00\x00\x00\x00\x00\x00\xe0\x3f\x11\x00\x00\x00\x00\x00\x00\xe0\x3f\x19\x00\x00\x00\x00\x00\x00\xe0\x3f\x50\x00\x0a\x15" + "\x0a\x05\x00\x00\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\xff\x80\xff\x80\xff\x00\xff\x80\xff\x03" + "\x42\x41\x44\xff\x80\xff\x02\x41\x4d\xff\x80\x00\x00\x00\x3f\xff\x80\xff\x01\x0a\x06\x08\x06\x10\x00\x18\x0d\x0a\x06\x08\x06" + "\x10\x01\x18\x17\x0a\x06\x08\x06\x10\x02\x18\x14\x0a\x06\x08\x06\x10\x03\x18\x14\x0a\x06\x08\x06\x10\x04\x18\x2b\x0a\x06\x08" + "\x06\x10\x05\x18\x17\x0a\x06\x08\x00\x10\x00\x18\x02\x0a\x06\x08\x00\x10\x01\x18\x02\x0a\x06\x08\x01\x10\x01\x18\x02\x0a\x06" + "\x08\x00\x10\x02\x18\x02\x0a\x06\x08\x02\x10\x02\x18\x02\x0a\x06\x08\x01\x10\x02\x18\x03\x0a\x06\x08\x00\x10\x03\x18\x02\x0a" + "\x06\x08\x02\x10\x03\x18\x02\x0a\x06\x08\x01\x10\x03\x18\x02\x0a\x06\x08\x00\x10\x04\x18\x02\x0a\x06\x08\x01\x10\x04\x18\x04" + "\x0a\x06\x08\x00\x10\x05\x18\x02\x0a\x06\x08\x01\x10\x05\x18\x02\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08" + "\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x1a\x03\x47\x4d\x54\x0a\x59\x0a\x04\x08" + "\x01\x50\x00\x0a\x0c\x08\x01\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x0a\x08\x08\x01\x42\x02\x08\x06\x50\x00\x0a\x08\x08\x01" + "\x42\x02\x08\x04\x50\x00\x0a\x21\x08\x01\x1a\x1b\x09\x00\x00\x00\x00\x00\x00\xe0\x3f\x11\x00\x00\x00\x00\x00\x00\xe0\x3f\x19" + "\x00\x00\x00\x00\x00\x00\xe0\x3f\x50\x00\x0a\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\x08\x03\x10\xec\x02\x1a\x0c" + "\x08\x03\x10\x8e\x01\x18\x1d\x20\xc1\x01\x28\x01\x22\x2e\x08\x0c\x12\x05\x01\x02\x03\x04\x05\x1a\x02\x69\x64\x1a\x07\x62\x6c" + "\x6f\x63\x6b\x4e\x6f\x1a\x04\x76\x61\x6c\x31\x1a\x04\x76\x61\x6c\x32\x1a\x04\x76\x61\x6c\x33\x20\x00\x28\x00\x30\x00\x22\x08" + "\x08\x04\x20\x00\x28\x00\x30\x00\x22\x08\x08\x08\x20\x00\x28\x00\x30\x00\x22\x08\x08\x08\x20\x00\x28\x00\x30\x00\x22\x08\x08" + "\x05\x20\x00\x28\x00\x30\x00\x22\x08\x08\x01\x20\x00\x28\x00\x30\x00\x30\x01\x3a\x04\x08\x01\x50\x00\x3a\x0c\x08\x01\x12\x06" + "\x08\x00\x10\x00\x18\x00\x50\x00\x3a\x08\x08\x01\x42\x02\x08\x06\x50\x00\x3a\x08\x08\x01\x42\x02\x08\x04\x50\x00\x3a\x21\x08" + "\x01\x1a\x1b\x09\x00\x00\x00\x00\x00\x00\xe0\x3f\x11\x00\x00\x00\x00\x00\x00\xe0\x3f\x19\x00\x00\x00\x00\x00\x00\xe0\x3f\x50" + "\x00\x3a\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\x40\x90\x4e\x48\x01\x08\xd5\x01\x10\x00\x18\x80\x80\x04\x22\x02" + "\x00\x0b\x28\x5b\x30\x06\x82\xf4\x03\x03\x4f\x52\x43\x18"; + ReadBufferFromString in(content); + ORCSchemaReader schema_reader(in, {}); + auto schema = schema_reader.readSchema(); + std::cout << "schema:" << schema.toString() << std::endl; + } return 0; } From 23353f5aff80ad704ba6c7b1d6c83e77056405eb Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 16 Aug 2023 12:06:36 +0200 Subject: [PATCH 590/777] fix --- tests/integration/test_graphite_merge_tree/test.py | 2 +- tests/integration/test_grpc_protocol/test.py | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_graphite_merge_tree/test.py b/tests/integration/test_graphite_merge_tree/test.py index c4364a03fd9..7c19888df6b 100644 --- a/tests/integration/test_graphite_merge_tree/test.py +++ b/tests/integration/test_graphite_merge_tree/test.py @@ -509,6 +509,6 @@ CREATE TABLE test.graphite_not_created ) # The order of retentions is not guaranteed - assert "age and precision should only grow up: " in str(exc.value) + assert "Age and precision should only grow up: " in str(exc.value) assert "36000:600" in str(exc.value) assert "72000:300" in str(exc.value) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 4a28dae900b..5b56c3da8dc 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -352,11 +352,12 @@ def test_authentication(): def test_logs(): + query = "SELECT has(groupArray(number), 42) FROM numbers(1000)" logs = query_and_get_logs( - "SELECT has(groupArray(number), 42) FROM numbers(1000)", + query, settings={"send_logs_level": "debug"}, ) - assert "SELECT 1" in logs + assert query in logs assert "Read 1 rows" in logs assert "Peak memory usage" in logs From 428a05a560dd9561f1729c38b963250b980c2f19 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 16 Aug 2023 14:04:14 +0300 Subject: [PATCH 591/777] Follow-up: Do not send logs to CI if the credentials are not set (#53456) * Follow-up * Automatic style fix * Update tests/ci/ast_fuzzer_check.py * Update tests/ci/functional_test_check.py * Update tests/ci/stress_check.py * Automatic style fix --------- Co-authored-by: robot-clickhouse Co-authored-by: Alexander Tokmakov --- tests/ci/ast_fuzzer_check.py | 2 +- tests/ci/functional_test_check.py | 2 +- tests/ci/stress_check.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 1a75d02bef4..fecf207589e 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -146,7 +146,7 @@ def main(): "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - if ci_logs_host != "CLICKHOUSE_CI_LOGS_HOST": + if ci_logs_host not in ("CLICKHOUSE_CI_LOGS_HOST", ""): subprocess.check_call( f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}' '{main_log_path}'", shell=True, diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 22210390b09..2bab330bd66 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -394,7 +394,7 @@ def main(): ci_logs_password = os.getenv( "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - if ci_logs_host != "CLICKHOUSE_CI_LOGS_HOST": + if ci_logs_host not in ("CLICKHOUSE_CI_LOGS_HOST", ""): subprocess.check_call( f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", shell=True, diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 9c18bcbfe40..21c3178faab 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -209,7 +209,7 @@ def run_stress_test(docker_image_name): ci_logs_password = os.getenv( "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" ) - if ci_logs_host != "CLICKHOUSE_CI_LOGS_HOST": + if ci_logs_host not in ("CLICKHOUSE_CI_LOGS_HOST", ""): subprocess.check_call( f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}'", shell=True, From 672fa0734f85e4c21b4431a220f692ddd8545ef4 Mon Sep 17 00:00:00 2001 From: Davit Vardanyan <78792753+davvard@users.noreply.github.com> Date: Wed, 16 Aug 2023 17:27:02 +0400 Subject: [PATCH 592/777] Fix: USearch test --- tests/queries/0_stateless/02354_hnsw_index.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02354_hnsw_index.sql b/tests/queries/0_stateless/02354_hnsw_index.sql index 059705254d8..01e788458a2 100644 --- a/tests/queries/0_stateless/02354_hnsw_index.sql +++ b/tests/queries/0_stateless/02354_hnsw_index.sql @@ -7,8 +7,8 @@ SELECT '--- Negative tests ---'; DROP TABLE IF EXISTS tab; --- must have at most 2 arguments -CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch('too', 'many', 'arguments')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } +-- must have at most 1 arguments +CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch('too', 'many')) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } -- first argument (distance_function) must be String CREATE TABLE tab(id Int32, vector Array(Float32), INDEX usearch_index vector TYPE usearch(3)) ENGINE = MergeTree ORDER BY id; -- { serverError INCORRECT_QUERY } @@ -100,7 +100,7 @@ SETTINGS max_limit_for_ann_queries=2; -- doesn't use the ann index DROP TABLE tab; --- Test Tuple embeddings. Triggers different logic than Array inside MergeTreeIndexUSearch but the same logic as Array above MergeTreeIndexUSearch. +-- Test Tuple embeddings. Triggers different logic than Array inside MergeTreeIndexUSearch but the same logic as Array above MergeTreeIndexusearch. -- Therefore test Tuple case just once. SELECT '--- Test with Tuple, GRANULARITY = 1, index_granularity = 5 ---'; From e5b459246d2a642216724f315ee688aa70735183 Mon Sep 17 00:00:00 2001 From: Davit Vardanyan <78792753+davvard@users.noreply.github.com> Date: Wed, 16 Aug 2023 17:28:16 +0400 Subject: [PATCH 593/777] Add: USearch default granularity --- src/Parsers/ASTIndexDeclaration.h | 1 + src/Parsers/ParserCreateIndexQuery.cpp | 2 ++ src/Parsers/ParserCreateQuery.cpp | 2 ++ tests/queries/0_stateless/02354_hnsw_index.reference | 4 ++-- 4 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ASTIndexDeclaration.h b/src/Parsers/ASTIndexDeclaration.h index 6ed241f75ab..1fbf5e12695 100644 --- a/src/Parsers/ASTIndexDeclaration.h +++ b/src/Parsers/ASTIndexDeclaration.h @@ -14,6 +14,7 @@ class ASTIndexDeclaration : public IAST public: static const auto DEFAULT_INDEX_GRANULARITY = 1uz; static const auto DEFAULT_ANNOY_INDEX_GRANULARITY = 100'000'000uz; + static const auto DEFAULT_USEARCH_INDEX_GRANULARITY = 100'000'000uz; String name; IAST * expr; diff --git a/src/Parsers/ParserCreateIndexQuery.cpp b/src/Parsers/ParserCreateIndexQuery.cpp index 67051d84999..81954e3c247 100644 --- a/src/Parsers/ParserCreateIndexQuery.cpp +++ b/src/Parsers/ParserCreateIndexQuery.cpp @@ -66,6 +66,8 @@ bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected { if (index->type && index->type->name == "annoy") index->granularity = ASTIndexDeclaration::DEFAULT_ANNOY_INDEX_GRANULARITY; + else if (index->type && index->type->name == "usearch") + index->granularity = ASTIndexDeclaration::DEFAULT_USEARCH_INDEX_GRANULARITY; else index->granularity = ASTIndexDeclaration::DEFAULT_INDEX_GRANULARITY; } diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 82674ab1a35..e11c94d0309 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -148,6 +148,8 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe { if (index->type->name == "annoy") index->granularity = ASTIndexDeclaration::DEFAULT_ANNOY_INDEX_GRANULARITY; + else if (index->type->name == "usearch") + index->granularity = ASTIndexDeclaration::DEFAULT_USEARCH_INDEX_GRANULARITY; else index->granularity = ASTIndexDeclaration::DEFAULT_INDEX_GRANULARITY; } diff --git a/tests/queries/0_stateless/02354_hnsw_index.reference b/tests/queries/0_stateless/02354_hnsw_index.reference index 102edf6d026..9896f149d45 100644 --- a/tests/queries/0_stateless/02354_hnsw_index.reference +++ b/tests/queries/0_stateless/02354_hnsw_index.reference @@ -1,7 +1,7 @@ --- Negative tests --- --- Test default GRANULARITY (should be 100 mio. for usearch)--- -CREATE TABLE default.tab\n(\n `id` Int32,\n `vector` Array(Float32),\n INDEX usearch_index vector TYPE usearch GRANULARITY 1\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 -CREATE TABLE default.tab\n(\n `id` Int32,\n `vector` Array(Float32),\n INDEX usearch_index vector TYPE usearch GRANULARITY 1\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `id` Int32,\n `vector` Array(Float32),\n INDEX usearch_index vector TYPE usearch GRANULARITY 100000000\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 +CREATE TABLE default.tab\n(\n `id` Int32,\n `vector` Array(Float32),\n INDEX usearch_index vector TYPE usearch GRANULARITY 100000000\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 --- Test with Array, GRANULARITY = 1, index_granularity = 5 --- WHERE type, L2Distance, check that index is used Expression ((Projection + Before ORDER BY)) From ef2957cb915200cb1d8c94616f7b174b21b622c2 Mon Sep 17 00:00:00 2001 From: Davit Vardanyan <78792753+davvard@users.noreply.github.com> Date: Wed, 16 Aug 2023 17:29:50 +0400 Subject: [PATCH 594/777] Add: allow_experimental_usearch_index checks --- src/Databases/DatabaseReplicated.cpp | 1 + src/Interpreters/InterpreterCreateQuery.cpp | 3 +++ 2 files changed, 4 insertions(+) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 304c0a20e26..acd3efc74bb 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -830,6 +830,7 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep query_context->setSetting("allow_experimental_hash_functions", 1); query_context->setSetting("allow_experimental_object_type", 1); query_context->setSetting("allow_experimental_annoy_index", 1); + query_context->setSetting("allow_experimental_usearch_index", 1); query_context->setSetting("allow_experimental_bigint_types", 1); query_context->setSetting("allow_experimental_window_functions", 1); query_context->setSetting("allow_experimental_geo_types", 1); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f4aee9ad4db..c7695df379a 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -704,6 +704,9 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti if (index_desc.type == "annoy" && !settings.allow_experimental_annoy_index) throw Exception(ErrorCodes::INCORRECT_QUERY, "Annoy index is disabled. Turn on allow_experimental_annoy_index"); + if (index_desc.type == "usearch" && !settings.allow_experimental_usearch_index) + throw Exception(ErrorCodes::INCORRECT_QUERY, "USearch index is disabled. Turn on allow_experimental_usearch_index"); + properties.indices.push_back(index_desc); } if (create.columns_list->projections) From 620733dd46e38598dd0319ef7b26674c763b7cad Mon Sep 17 00:00:00 2001 From: Davit Vardanyan <78792753+davvard@users.noreply.github.com> Date: Wed, 16 Aug 2023 17:30:29 +0400 Subject: [PATCH 595/777] Fix: Exception handling --- src/Storages/MergeTree/MergeTreeIndexHnsw.cpp | 29 ++++++++++++++----- 1 file changed, 21 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp b/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp index 67726f41e8c..73ac7252595 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp @@ -53,7 +53,7 @@ void USearchIndexWithSerialization::deserialize([[maybe_unused]] ReadBuf auto copy = Base::copy(); if (!copy) - throw std::runtime_error("Can't copy index"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not copy usearch index"); Base::swap(copy.index); } @@ -151,10 +151,18 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t index = std::make_shared>(size); /// Add all rows of block - index->reserve(unum::usearch::ceil2(index->size() + num_rows + 1)); - index->add(index->size(), array.data()); - for (size_t current_row = 1; current_row < num_rows; ++current_row) - index->add(index->size(), &array[offsets[current_row - 1]]); + if (!index->reserve(unum::usearch::ceil2(index->size() + num_rows))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not reserve"); + + auto add_result = index->add(index->size(), array.data()); + if (!add_result) + throw Exception(ErrorCodes::LOGICAL_ERROR, add_result.error.release()); + for (size_t current_row = 1; current_row < num_rows; ++current_row){ + add_result = index->add(index->size(), &array[offsets[current_row - 1]]); + if (!add_result) + throw Exception(ErrorCodes::LOGICAL_ERROR, add_result.error.release()); + } + } else if (const auto & column_tuple = typeid_cast(column_cut.get())) { @@ -171,9 +179,14 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t throw Exception(ErrorCodes::LOGICAL_ERROR, "Tuple has 0 rows, {} rows expected", rows_read); index = std::make_shared>(data[0].size()); - index->reserve(unum::usearch::ceil2(index->size() + data.size())); - for (const auto & item : data) - index->add(index->size(), item.data()); + if (!index->reserve(unum::usearch::ceil2(index->size() + data.size()))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not reserve"); + + for (const auto & item : data){ + auto add_result = index->add(index->size(), item.data()); + if (!add_result) + throw Exception(ErrorCodes::LOGICAL_ERROR, add_result.error.release()); + } } else throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array or Tuple column"); From a0ec274766b1ab952bdcb1313266828d25110444 Mon Sep 17 00:00:00 2001 From: Davit Vardanyan <78792753+davvard@users.noreply.github.com> Date: Wed, 16 Aug 2023 17:30:56 +0400 Subject: [PATCH 596/777] Refactor: Naming --- src/Storages/MergeTree/MergeTreeIndexHnsw.h | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexHnsw.h b/src/Storages/MergeTree/MergeTreeIndexHnsw.h index 6be4a649ebe..89f5e3bb8cb 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHnsw.h +++ b/src/Storages/MergeTree/MergeTreeIndexHnsw.h @@ -9,7 +9,7 @@ namespace DB { -template +template class USearchIndexWithSerialization : public unum::usearch::index_dense_t { using Base = unum::usearch::index_dense_t; @@ -21,15 +21,15 @@ public: size_t getDimensions() const; }; -template -using USearchIndexWithSerializationPtr = std::shared_ptr>; +template +using USearchIndexWithSerializationPtr = std::shared_ptr>; -template +template struct MergeTreeIndexGranuleUSearch final : public IMergeTreeIndexGranule { MergeTreeIndexGranuleUSearch(const String & index_name_, const Block & index_sample_block_); MergeTreeIndexGranuleUSearch( - const String & index_name_, const Block & index_sample_block_, USearchIndexWithSerializationPtr index_); + const String & index_name_, const Block & index_sample_block_, USearchIndexWithSerializationPtr index_); ~MergeTreeIndexGranuleUSearch() override = default; @@ -40,10 +40,10 @@ struct MergeTreeIndexGranuleUSearch final : public IMergeTreeIndexGranule const String index_name; const Block index_sample_block; - USearchIndexWithSerializationPtr index; + USearchIndexWithSerializationPtr index; }; -template +template struct MergeTreeIndexAggregatorUSearch final : IMergeTreeIndexAggregator { MergeTreeIndexAggregatorUSearch(const String & index_name_, const Block & index_sample_block); @@ -55,7 +55,7 @@ struct MergeTreeIndexAggregatorUSearch final : IMergeTreeIndexAggregator const String index_name; const Block index_sample_block; - USearchIndexWithSerializationPtr index; + USearchIndexWithSerializationPtr index; }; @@ -72,7 +72,7 @@ public: std::vector getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const override; private: - template + template std::vector getUsefulRangesImpl(MergeTreeIndexGranulePtr idx_granule) const; const ApproximateNearestNeighborCondition ann_condition; From c685ac3df910a0aacc4698798ae9eb5411009d7d Mon Sep 17 00:00:00 2001 From: Davit Vardanyan <78792753+davvard@users.noreply.github.com> Date: Wed, 16 Aug 2023 17:31:46 +0400 Subject: [PATCH 597/777] Docs: Update USearch docs --- docs/en/engines/table-engines/mergetree-family/annindexes.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 4fd1d0d17db..76289ee7f60 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -257,3 +257,6 @@ USearch currently supports two distance functions: ([Wikipedia](https://en.wikipedia.org/wiki/Euclidean_distance)). - `cosineDistance`, also called cosine similarity, is the cosine of the angle between two (non-zero) vectors ([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)). + +For normalized data, `L2Distance` is usually a better choice, otherwise `cosineDistance` is recommended to compensate for scale. If no +distance function was specified during index creation, `L2Distance` is used as default. \ No newline at end of file From 5a87a09c8477ae013cd8877c31aecd6f73b6ed50 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 16 Aug 2023 17:14:58 +0000 Subject: [PATCH 598/777] Tweak exception handling --- src/Storages/MergeTree/MergeTreeIndexHnsw.cpp | 29 +++++++++---------- 1 file changed, 13 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp b/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp index 73ac7252595..369779e97fd 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp @@ -17,6 +17,7 @@ namespace DB namespace ErrorCodes { + extern const int CANNOT_ALLOCATE_MEMORY; extern const int ILLEGAL_COLUMN; extern const int INCORRECT_DATA; extern const int INCORRECT_NUMBER_OF_COLUMNS; @@ -152,16 +153,13 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t /// Add all rows of block if (!index->reserve(unum::usearch::ceil2(index->size() + num_rows))) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not reserve"); + throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for usearch index"); - auto add_result = index->add(index->size(), array.data()); - if (!add_result) - throw Exception(ErrorCodes::LOGICAL_ERROR, add_result.error.release()); - for (size_t current_row = 1; current_row < num_rows; ++current_row){ - add_result = index->add(index->size(), &array[offsets[current_row - 1]]); - if (!add_result) - throw Exception(ErrorCodes::LOGICAL_ERROR, add_result.error.release()); - } + if (auto rc = index->add(index->size(), array.data()); !rc) + throw Exception(ErrorCodes::INCORRECT_DATA, rc.error.release()); + for (size_t current_row = 1; current_row < num_rows; ++current_row) + if (auto rc = index->add(index->size(), &array[offsets[current_row - 1]]); !rc) + throw Exception(ErrorCodes::INCORRECT_DATA, add_result.error.release()); } else if (const auto & column_tuple = typeid_cast(column_cut.get())) @@ -179,14 +177,13 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t throw Exception(ErrorCodes::LOGICAL_ERROR, "Tuple has 0 rows, {} rows expected", rows_read); index = std::make_shared>(data[0].size()); - if (!index->reserve(unum::usearch::ceil2(index->size() + data.size()))) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not reserve"); - for (const auto & item : data){ - auto add_result = index->add(index->size(), item.data()); - if (!add_result) - throw Exception(ErrorCodes::LOGICAL_ERROR, add_result.error.release()); - } + if (!index->reserve(unum::usearch::ceil2(index->size() + data.size()))) + throw Exception(ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Could not reserve memory for usearch index"); + + for (const auto & item : data) + if (auto rc = index->add(index->size(), item.data()); !rc) + throw Exception(ErrorCodes::INCORRECT_DATA, add_result.error.release()); } else throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array or Tuple column"); From 565b98c39ee41e97cbc57d20b3105014d2c49cd6 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Thu, 20 Apr 2023 05:34:20 +0000 Subject: [PATCH 599/777] add partition format in the partlog Signed-off-by: Jianfei Hu --- src/Interpreters/PartLog.cpp | 6 ++++++ src/Interpreters/PartLog.h | 1 + 2 files changed, 7 insertions(+) diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index a97f1f405bc..493ca38e9bb 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -107,6 +107,7 @@ NamesAndTypesList PartLogElement::getNamesAndTypes() {"table_uuid", std::make_shared()}, {"part_name", std::make_shared()}, {"partition_id", std::make_shared()}, + {"partition", std::make_shared()}, {"part_type", std::make_shared()}, {"disk_name", std::make_shared()}, {"path_on_disk", std::make_shared()}, @@ -157,6 +158,7 @@ void PartLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(table_uuid); columns[i++]->insert(part_name); columns[i++]->insert(partition_id); + columns[i++]->insert(partition); columns[i++]->insert(part_type.toString()); columns[i++]->insert(disk_name); columns[i++]->insert(path_on_disk); @@ -229,6 +231,10 @@ bool PartLog::addNewParts( elem.table_name = table_id.table_name; elem.table_uuid = table_id.uuid; elem.partition_id = part->info.partition_id; + FormatSettings format_settings; + WriteBufferFromOwnString out; + part->partition.serializeText(part->storage, out, format_settings); + elem.partition = out.str(); elem.part_name = part->name; elem.disk_name = part->getDataPartStorage().getDiskName(); elem.path_on_disk = part->getDataPartStorage().getFullPath(); diff --git a/src/Interpreters/PartLog.h b/src/Interpreters/PartLog.h index 843792d03a9..462314f2768 100644 --- a/src/Interpreters/PartLog.h +++ b/src/Interpreters/PartLog.h @@ -63,6 +63,7 @@ struct PartLogElement UUID table_uuid{UUIDHelpers::Nil}; String part_name; String partition_id; + String partition; String disk_name; String path_on_disk; From 59a81b82bc9aec7a117b8172973e75d4856c26a4 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Thu, 20 Apr 2023 06:01:01 +0000 Subject: [PATCH 600/777] wip the merge partition Signed-off-by: Jianfei Hu --- src/Storages/MergeTree/MergeList.cpp | 6 ++++++ src/Storages/MergeTree/MergeList.h | 2 ++ src/Storages/System/StorageSystemMerges.cpp | 2 ++ 3 files changed, 10 insertions(+) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 1cd2d11da50..0950f5fc961 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -25,6 +25,8 @@ MergeListElement::MergeListElement( , merge_type{future_part->merge_type} , merge_algorithm{MergeAlgorithm::Undecided} { + WriteBufferFromOwnString out; + FormatSettings format_settings; for (const auto & source_part : future_part->parts) { source_part_names.emplace_back(source_part->name); @@ -34,6 +36,10 @@ MergeListElement::MergeListElement( total_size_bytes_uncompressed += source_part->getTotalColumnsSize().data_uncompressed; total_size_marks += source_part->getMarksCount(); total_rows_count += source_part->index_granularity.getTotalRows(); + if (partition.empty()) { + future_part->getPartition().serializeText(source_part->storage, out, format_settings); + partition = out.str(); + } } if (!future_part->parts.empty()) diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index ffa87e75505..fe95053f79d 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -67,6 +67,8 @@ struct MergeListElement : boost::noncopyable { const StorageID table_id; std::string partition_id; + // TODO: here the merge list element and merge list. how to get()->partition. + std::string partition; const std::string result_part_name; const std::string result_part_path; diff --git a/src/Storages/System/StorageSystemMerges.cpp b/src/Storages/System/StorageSystemMerges.cpp index 1f32a0ff700..889fa9e94db 100644 --- a/src/Storages/System/StorageSystemMerges.cpp +++ b/src/Storages/System/StorageSystemMerges.cpp @@ -20,6 +20,7 @@ NamesAndTypesList StorageSystemMerges::getNamesAndTypes() {"source_part_paths", std::make_shared(std::make_shared())}, {"result_part_path", std::make_shared()}, {"partition_id", std::make_shared()}, + {"partition", std::make_shared()}, {"is_mutation", std::make_shared()}, {"total_size_bytes_compressed", std::make_shared()}, {"total_size_bytes_uncompressed", std::make_shared()}, @@ -58,6 +59,7 @@ void StorageSystemMerges::fillData(MutableColumns & res_columns, ContextPtr cont res_columns[i++]->insert(merge.source_part_paths); res_columns[i++]->insert(merge.result_part_path); res_columns[i++]->insert(merge.partition_id); + // res_columns[i++]->insert(context->getMergeList().get().; res_columns[i++]->insert(merge.is_mutation); res_columns[i++]->insert(merge.total_size_bytes_compressed); res_columns[i++]->insert(merge.total_size_bytes_uncompressed); From bd4df60df6a1a42fe1b31e86526d2f5b46705a49 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Thu, 20 Apr 2023 21:37:50 +0000 Subject: [PATCH 601/777] fix merges Signed-off-by: Jianfei Hu --- src/Storages/MergeTree/MergeList.cpp | 5 ++++- src/Storages/MergeTree/MergeList.h | 1 + src/Storages/System/StorageSystemMerges.cpp | 2 +- 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 0950f5fc961..7d8a6434789 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -27,6 +27,7 @@ MergeListElement::MergeListElement( { WriteBufferFromOwnString out; FormatSettings format_settings; + bool partition_initialized = false; for (const auto & source_part : future_part->parts) { source_part_names.emplace_back(source_part->name); @@ -36,7 +37,8 @@ MergeListElement::MergeListElement( total_size_bytes_uncompressed += source_part->getTotalColumnsSize().data_uncompressed; total_size_marks += source_part->getMarksCount(); total_rows_count += source_part->index_granularity.getTotalRows(); - if (partition.empty()) { + if (partition_initialized) { + partition_initialized = true; future_part->getPartition().serializeText(source_part->storage, out, format_settings); partition = out.str(); } @@ -59,6 +61,7 @@ MergeInfo MergeListElement::getInfo() const res.result_part_name = result_part_name; res.result_part_path = result_part_path; res.partition_id = partition_id; + res.partition = partition; res.is_mutation = is_mutation; res.elapsed = watch.elapsedSeconds(); res.progress = progress.load(std::memory_order_relaxed); diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index fe95053f79d..62177ccdc9d 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -35,6 +35,7 @@ struct MergeInfo Array source_part_names; Array source_part_paths; std::string partition_id; + std::string partition; bool is_mutation; Float64 elapsed; Float64 progress; diff --git a/src/Storages/System/StorageSystemMerges.cpp b/src/Storages/System/StorageSystemMerges.cpp index 889fa9e94db..2dbe2964eb9 100644 --- a/src/Storages/System/StorageSystemMerges.cpp +++ b/src/Storages/System/StorageSystemMerges.cpp @@ -59,7 +59,7 @@ void StorageSystemMerges::fillData(MutableColumns & res_columns, ContextPtr cont res_columns[i++]->insert(merge.source_part_paths); res_columns[i++]->insert(merge.result_part_path); res_columns[i++]->insert(merge.partition_id); - // res_columns[i++]->insert(context->getMergeList().get().; + res_columns[i++]->insert(merge.partition); res_columns[i++]->insert(merge.is_mutation); res_columns[i++]->insert(merge.total_size_bytes_compressed); res_columns[i++]->insert(merge.total_size_bytes_uncompressed); From c0ac6484c32a1a5295c9733fa2d7691bafd84e81 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 21 Apr 2023 15:03:55 +0300 Subject: [PATCH 602/777] Update PartLog.cpp --- src/Interpreters/PartLog.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index 493ca38e9bb..d0344468153 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -231,10 +231,10 @@ bool PartLog::addNewParts( elem.table_name = table_id.table_name; elem.table_uuid = table_id.uuid; elem.partition_id = part->info.partition_id; - FormatSettings format_settings; - WriteBufferFromOwnString out; - part->partition.serializeText(part->storage, out, format_settings); - elem.partition = out.str(); + { + WriteBufferFromString out(elem.partition); + part->partition.serializeText(part->storage, out, {}); + } elem.part_name = part->name; elem.disk_name = part->getDataPartStorage().getDiskName(); elem.path_on_disk = part->getDataPartStorage().getFullPath(); From ecd1fb56b4f1980663870472afa1a04f4d600333 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 21 Apr 2023 15:05:41 +0300 Subject: [PATCH 603/777] Update MergeList.cpp --- src/Storages/MergeTree/MergeList.cpp | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 7d8a6434789..b49e16a28ef 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -10,7 +10,6 @@ namespace DB { - MergeListElement::MergeListElement( const StorageID & table_id_, FutureMergedMutatedPartPtr future_part, @@ -25,9 +24,6 @@ MergeListElement::MergeListElement( , merge_type{future_part->merge_type} , merge_algorithm{MergeAlgorithm::Undecided} { - WriteBufferFromOwnString out; - FormatSettings format_settings; - bool partition_initialized = false; for (const auto & source_part : future_part->parts) { source_part_names.emplace_back(source_part->name); @@ -37,10 +33,11 @@ MergeListElement::MergeListElement( total_size_bytes_uncompressed += source_part->getTotalColumnsSize().data_uncompressed; total_size_marks += source_part->getMarksCount(); total_rows_count += source_part->index_granularity.getTotalRows(); - if (partition_initialized) { - partition_initialized = true; - future_part->getPartition().serializeText(source_part->storage, out, format_settings); - partition = out.str(); + + if (partition.empty()) + { + WriteBufferFromString out(partition); + future_part->getPartition().serializeText(source_part->storage, out, {}); } } From 664bcfc8b3e5aaf14b45a00129f6e5221d9c2581 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 21 Apr 2023 15:07:25 +0300 Subject: [PATCH 604/777] Update MergeList.cpp --- src/Storages/MergeTree/MergeList.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index b49e16a28ef..fdc99d077e9 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -33,18 +33,15 @@ MergeListElement::MergeListElement( total_size_bytes_uncompressed += source_part->getTotalColumnsSize().data_uncompressed; total_size_marks += source_part->getMarksCount(); total_rows_count += source_part->index_granularity.getTotalRows(); - - if (partition.empty()) - { - WriteBufferFromString out(partition); - future_part->getPartition().serializeText(source_part->storage, out, {}); - } } if (!future_part->parts.empty()) { source_data_version = future_part->parts[0]->info.getDataVersion(); is_mutation = (result_part_info.getDataVersion() != source_data_version); + + WriteBufferFromString out(partition); + future_part->getPartition().serializeText(future_part->parts[0]->storage, out, {}); } thread_group = ThreadGroup::createForBackgroundProcess(context); From 7376b7ec8c7437ccf92cb2300b999d4b92b2fbf6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 21 Apr 2023 15:08:12 +0300 Subject: [PATCH 605/777] Update MergeList.h --- src/Storages/MergeTree/MergeList.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 62177ccdc9d..93f4b38c4c8 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -68,8 +68,6 @@ struct MergeListElement : boost::noncopyable { const StorageID table_id; std::string partition_id; - // TODO: here the merge list element and merge list. how to get()->partition. - std::string partition; const std::string result_part_name; const std::string result_part_path; From bc77aab2e8c5f4a586628e9ebf18189eb3ca2b09 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Apr 2023 16:11:54 +0300 Subject: [PATCH 606/777] Update MergeList.h --- src/Storages/MergeTree/MergeList.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 93f4b38c4c8..d40af6abf43 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -68,6 +68,7 @@ struct MergeListElement : boost::noncopyable { const StorageID table_id; std::string partition_id; + std::string partition; const std::string result_part_name; const std::string result_part_path; From 5e3a69b4c488d53f7b5be6201e32ce63c6061462 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Apr 2023 21:43:30 +0300 Subject: [PATCH 607/777] Update MergeList.cpp --- src/Storages/MergeTree/MergeList.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index fdc99d077e9..a52704a9fb3 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -39,7 +39,7 @@ MergeListElement::MergeListElement( { source_data_version = future_part->parts[0]->info.getDataVersion(); is_mutation = (result_part_info.getDataVersion() != source_data_version); - + WriteBufferFromString out(partition); future_part->getPartition().serializeText(future_part->parts[0]->storage, out, {}); } From 7769dad0c471e047834c68e3dd6e912e18419fe3 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Tue, 20 Jun 2023 05:17:07 +0000 Subject: [PATCH 608/777] Add integration test for partition. Signed-off-by: Jianfei Hu --- tests/integration/test_part_log_table/test.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/tests/integration/test_part_log_table/test.py b/tests/integration/test_part_log_table/test.py index ea4499d5877..f9268f4c731 100644 --- a/tests/integration/test_part_log_table/test.py +++ b/tests/integration/test_part_log_table/test.py @@ -56,6 +56,20 @@ def test_config_with_standard_part_log(start_cluster): assert node2.query("SELECT * FROM system.part_log") != "" +def test_part_log_contains_partition(start_cluster): + node2.query( + "CREATE TABLE test_partition_table (date Date, word String, value UInt64) ENGINE=MergeTree() " + + "PARTITION BY toYYYYMM(date) Order by value" + ) + node2.query("INSERT INTO test_partition_table VALUES " + + "('2023-06-20', 'a', 10), ('2023-06-21', 'b', 11)," + + "('2023-05-20', 'cc', 14),('2023-05-21', 'd1', 15);" + ) + node2.query("SYSTEM FLUSH LOGS") + resp = node2.query("SELECT partition from system.part_log where table = 'test_partition_table'") + assert(resp == "202306\n202305\n") + + def test_config_with_non_standard_part_log(start_cluster): node3.query( "CREATE TABLE test_table(word String, value UInt64) ENGINE=MergeTree() Order by value" From 4b3564b2b09f94d329f00b2f453ac312bd755b19 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Tue, 20 Jun 2023 05:31:05 +0000 Subject: [PATCH 609/777] fix style py Signed-off-by: Jianfei Hu --- tests/integration/test_part_log_table/test.py | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_part_log_table/test.py b/tests/integration/test_part_log_table/test.py index f9268f4c731..0c2cab4d90b 100644 --- a/tests/integration/test_part_log_table/test.py +++ b/tests/integration/test_part_log_table/test.py @@ -58,15 +58,17 @@ def test_config_with_standard_part_log(start_cluster): def test_part_log_contains_partition(start_cluster): node2.query( - "CREATE TABLE test_partition_table (date Date, word String, value UInt64) ENGINE=MergeTree() " + - "PARTITION BY toYYYYMM(date) Order by value" + "CREATE TABLE test_partition_table (date Date, word String, value UInt64) ENGINE=MergeTree() " + + "PARTITION BY toYYYYMM(date) Order by value" ) - node2.query("INSERT INTO test_partition_table VALUES " + - "('2023-06-20', 'a', 10), ('2023-06-21', 'b', 11)," + - "('2023-05-20', 'cc', 14),('2023-05-21', 'd1', 15);" + node2.query("INSERT INTO test_partition_table VALUES " + + "('2023-06-20', 'a', 10), ('2023-06-21', 'b', 11)," + + "('2023-05-20', 'cc', 14),('2023-05-21', 'd1', 15);" ) node2.query("SYSTEM FLUSH LOGS") - resp = node2.query("SELECT partition from system.part_log where table = 'test_partition_table'") + resp = node2.query( + "SELECT partition from system.part_log where table = 'test_partition_table'" + ) assert(resp == "202306\n202305\n") From 8b8397de5253bb9eecd440d4de75f1a8a0239afb Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Tue, 20 Jun 2023 07:26:46 +0000 Subject: [PATCH 610/777] fix the missing column Signed-off-by: Jianfei Hu --- tests/integration/test_part_log_table/test.py | 5 +++-- .../0_stateless/02117_show_create_table_system.reference | 1 + 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_part_log_table/test.py b/tests/integration/test_part_log_table/test.py index 0c2cab4d90b..15d65cc31dd 100644 --- a/tests/integration/test_part_log_table/test.py +++ b/tests/integration/test_part_log_table/test.py @@ -61,7 +61,8 @@ def test_part_log_contains_partition(start_cluster): "CREATE TABLE test_partition_table (date Date, word String, value UInt64) ENGINE=MergeTree() " + "PARTITION BY toYYYYMM(date) Order by value" ) - node2.query("INSERT INTO test_partition_table VALUES " + node2.query( + "INSERT INTO test_partition_table VALUES " + "('2023-06-20', 'a', 10), ('2023-06-21', 'b', 11)," + "('2023-05-20', 'cc', 14),('2023-05-21', 'd1', 15);" ) @@ -69,7 +70,7 @@ def test_part_log_contains_partition(start_cluster): resp = node2.query( "SELECT partition from system.part_log where table = 'test_partition_table'" ) - assert(resp == "202306\n202305\n") + assert resp == "202306\n202305\n" def test_config_with_non_standard_part_log(start_cluster): diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 0616756328b..0a58e41f9d5 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -364,6 +364,7 @@ CREATE TABLE system.merges `source_part_paths` Array(String), `result_part_path` String, `partition_id` String, + `partition` String, `is_mutation` UInt8, `total_size_bytes_compressed` UInt64, `total_size_bytes_uncompressed` UInt64, From afaaef6875c28ad438c1ce37daea80e0ed2cb017 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Mon, 17 Jul 2023 16:15:28 +0000 Subject: [PATCH 611/777] wip checking whether system.merge cause sigsev. Signed-off-by: Jianfei Hu --- src/Storages/MergeTree/MergeList.cpp | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index a52704a9fb3..6a6ad7719b1 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -1,19 +1,16 @@ +#include #include #include -#include +#include #include #include #include -#include namespace DB { -MergeListElement::MergeListElement( - const StorageID & table_id_, - FutureMergedMutatedPartPtr future_part, - const ContextPtr & context) +MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMutatedPartPtr future_part, const ContextPtr & context) : table_id{table_id_} , partition_id{future_part->part_info.partition_id} , result_part_name{future_part->name} @@ -40,8 +37,9 @@ MergeListElement::MergeListElement( source_data_version = future_part->parts[0]->info.getDataVersion(); is_mutation = (result_part_info.getDataVersion() != source_data_version); - WriteBufferFromString out(partition); - future_part->getPartition().serializeText(future_part->parts[0]->storage, out, {}); + partition = "jianfei-debugging-sigsev"; + // WriteBufferFromString out(partition); + // future_part->getPartition().serializeText(future_part->parts[0]->storage, out, {}); } thread_group = ThreadGroup::createForBackgroundProcess(context); From 7227cc6d2b16675b1434bf6c245371b569cc1bcc Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Tue, 18 Jul 2023 06:10:28 +0000 Subject: [PATCH 612/777] try parts[0] instead of front(). Signed-off-by: Jianfei Hu --- src/Storages/MergeTree/MergeList.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 6a6ad7719b1..5afe6e25136 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -37,9 +37,10 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta source_data_version = future_part->parts[0]->info.getDataVersion(); is_mutation = (result_part_info.getDataVersion() != source_data_version); - partition = "jianfei-debugging-sigsev"; - // WriteBufferFromString out(partition); - // future_part->getPartition().serializeText(future_part->parts[0]->storage, out, {}); + // partition = "jianfei-debugging-sigsev"; + WriteBufferFromString out(partition); + const auto & part = future_part->parts[0]; + part->partition.serializeText(part->storage, out, {}); } thread_group = ThreadGroup::createForBackgroundProcess(context); From 02b7bd94eb5bd40a49a9842280e92084de12510c Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Thu, 20 Jul 2023 04:23:13 +0000 Subject: [PATCH 613/777] suspect some root cause of value[0]. Signed-off-by: Jianfei Hu --- src/Storages/MergeTree/MergeList.cpp | 1 - src/Storages/MergeTree/MergeTreePartition.cpp | 150 +++++++++++------- src/Storages/MergeTree/MergeTreePartition.h | 18 ++- 3 files changed, 105 insertions(+), 64 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 5afe6e25136..8fbb163384e 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -37,7 +37,6 @@ MergeListElement::MergeListElement(const StorageID & table_id_, FutureMergedMuta source_data_version = future_part->parts[0]->info.getDataVersion(); is_mutation = (result_part_info.getDataVersion() != source_data_version); - // partition = "jianfei-debugging-sigsev"; WriteBufferFromString out(partition); const auto & part = future_part->parts[0]; part->partition.serializeText(part->storage, out, {}); diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index bce33438229..b8d1879cd67 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -1,20 +1,20 @@ -#include -#include -#include -#include -#include -#include +#include +#include #include #include #include -#include -#include -#include -#include -#include +#include +#include +#include +#include +#include #include -#include - +#include "Common/logger_useful.h" +#include +#include +#include +#include +#include namespace DB { @@ -38,51 +38,52 @@ namespace { private: SipHash & hash; - public: - explicit LegacyFieldVisitorHash(SipHash & hash_) : hash(hash_) {} - void operator() (const Null &) const + public: + explicit LegacyFieldVisitorHash(SipHash & hash_) : hash(hash_) { } + + void operator()(const Null &) const { UInt8 type = Field::Types::Null; hash.update(type); } - void operator() (const UInt64 & x) const + void operator()(const UInt64 & x) const { UInt8 type = Field::Types::UInt64; hash.update(type); hash.update(x); } - void operator() (const UInt128 & x) const + void operator()(const UInt128 & x) const { UInt8 type = Field::Types::UInt128; hash.update(type); hash.update(x); } - void operator() (const UInt256 & x) const + void operator()(const UInt256 & x) const { UInt8 type = Field::Types::UInt256; hash.update(type); hash.update(x); } - void operator() (const Int64 & x) const + void operator()(const Int64 & x) const { UInt8 type = Field::Types::Int64; hash.update(type); hash.update(x); } - void operator() (const Int128 & x) const + void operator()(const Int128 & x) const { UInt8 type = Field::Types::Int128; hash.update(type); hash.update(x); } - void operator() (const Int256 & x) const + void operator()(const Int256 & x) const { UInt8 type = Field::Types::Int256; hash.update(type); hash.update(x); } - void operator() (const UUID & x) const + void operator()(const UUID & x) const { #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ auto tmp_x = x.toUnderType(); @@ -94,30 +95,27 @@ namespace operator()(x.toUnderType()); #endif } - void operator() (const IPv4 & x) const + void operator()(const IPv4 & x) const { UInt8 type = Field::Types::IPv4; hash.update(type); hash.update(x); } - void operator() (const IPv6 & x) const - { - return operator()(String(reinterpret_cast(&x), 16)); - } - void operator() (const Float64 & x) const + void operator()(const IPv6 & x) const { return operator()(String(reinterpret_cast(&x), 16)); } + void operator()(const Float64 & x) const { UInt8 type = Field::Types::Float64; hash.update(type); hash.update(x); } - void operator() (const String & x) const + void operator()(const String & x) const { UInt8 type = Field::Types::String; hash.update(type); hash.update(x.size()); hash.update(x.data(), x.size()); } - void operator() (const Array & x) const + void operator()(const Array & x) const { UInt8 type = Field::Types::Array; hash.update(type); @@ -126,7 +124,7 @@ namespace for (const auto & elem : x) applyVisitor(*this, elem); } - void operator() (const Tuple & x) const + void operator()(const Tuple & x) const { UInt8 type = Field::Types::Tuple; hash.update(type); @@ -135,7 +133,7 @@ namespace for (const auto & elem : x) applyVisitor(*this, elem); } - void operator() (const Map & x) const + void operator()(const Map & x) const { UInt8 type = Field::Types::Map; hash.update(type); @@ -144,43 +142,43 @@ namespace for (const auto & elem : x) applyVisitor(*this, elem); } - void operator() (const Object & x) const + void operator()(const Object & x) const { UInt8 type = Field::Types::Object; hash.update(type); hash.update(x.size()); - for (const auto & [key, value]: x) + for (const auto & [key, value] : x) { hash.update(key); applyVisitor(*this, value); } } - void operator() (const DecimalField & x) const + void operator()(const DecimalField & x) const { UInt8 type = Field::Types::Decimal32; hash.update(type); hash.update(x.getValue().value); } - void operator() (const DecimalField & x) const + void operator()(const DecimalField & x) const { UInt8 type = Field::Types::Decimal64; hash.update(type); hash.update(x.getValue().value); } - void operator() (const DecimalField & x) const + void operator()(const DecimalField & x) const { UInt8 type = Field::Types::Decimal128; hash.update(type); hash.update(x.getValue().value); } - void operator() (const DecimalField & x) const + void operator()(const DecimalField & x) const { UInt8 type = Field::Types::Decimal256; hash.update(type); hash.update(x.getValue().value); } - void operator() (const AggregateFunctionStateData & x) const + void operator()(const AggregateFunctionStateData & x) const { UInt8 type = Field::Types::AggregateFunctionState; hash.update(type); @@ -189,7 +187,7 @@ namespace hash.update(x.data.size()); hash.update(x.data.data(), x.data.size()); } - void operator() (const CustomType & x) const + void operator()(const CustomType & x) const { UInt8 type = Field::Types::CustomType; hash.update(type); @@ -198,7 +196,7 @@ namespace hash.update(result.size()); hash.update(result.data(), result.size()); } - void operator() (const bool & x) const + void operator()(const bool & x) const { UInt8 type = Field::Types::Bool; hash.update(type); @@ -291,7 +289,12 @@ std::optional MergeTreePartition::tryParseValueFromID(const String & partit return res; } - enum KeyType { DATE, UNSIGNED, SIGNED }; + enum KeyType + { + DATE, + UNSIGNED, + SIGNED + }; std::vector key_types; key_types.reserve(num_keys); @@ -321,8 +324,7 @@ std::optional MergeTreePartition::tryParseValueFromID(const String & partit { switch (key_types[i]) { - case DATE: - { + case DATE: { UInt32 date_yyyymmdd; readText(date_yyyymmdd, buf); constexpr UInt32 min_yyyymmdd = 10000000; @@ -335,15 +337,13 @@ std::optional MergeTreePartition::tryParseValueFromID(const String & partit res.emplace_back(date); break; } - case UNSIGNED: - { + case UNSIGNED: { UInt64 value; readText(value, buf); res.emplace_back(value); break; } - case SIGNED: - { + case SIGNED: { Int64 value; readText(value, buf); res.emplace_back(value); @@ -359,8 +359,8 @@ std::optional MergeTreePartition::tryParseValueFromID(const String & partit String expected_partition_id = MergeTreePartition{res}.getID(partition_key_sample); if (expected_partition_id != partition_id) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Partition ID was parsed incorrectly: expected {}, got {}", - expected_partition_id, partition_id); + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Partition ID was parsed incorrectly: expected {}, got {}", expected_partition_id, partition_id); return res; } @@ -379,8 +379,38 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe { const DataTypePtr & type = partition_key_sample.getByPosition(0).type; auto column = type->createColumn(); - column->insert(value[0]); - type->getDefaultSerialization()->serializeText(*column, 0, out, format_settings); + // metadata_snapshot-> + // TODO: here print out the type that causing the segfault. + const auto & all_column_names = metadata_snapshot->getColumns().getAll(); + // if (type.get() != nullptr) + // { + // LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug type is not nil, name {}.", type->getName()); + // } + if (value.empty()) + { + LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug: find the root cause, value is empty"); + writeCString("tuple()", out); + LOG_INFO( + &Poco::Logger::get("DNSCacheUpdater"), + "jianfeih debug: type is nil: {}, colum is nil: {}, storage has partition key: {}, column size: {}, " + "column names: {} ", + type.get() == nullptr, + column == nullptr, + metadata_snapshot->hasPartitionKey(), + metadata_snapshot->columns.size(), + all_column_names.toString()); + return; + } + else + { + // LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug: value[0] to be inserted: {}", value[0]); + // should be here causing the segfault. column might be empty. some type created column is not okay. + // let's check the table definition; type; parts info at this point. + // logging is not right, need to figure out include header properly. + column->insert(value[0]); + type->getDefaultSerialization()->serializeText(*column, 0, out, format_settings); + // LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug: finish the serialization no error."); + } } else { @@ -415,7 +445,8 @@ void MergeTreePartition::load(const MergeTreeData & storage, const PartMetadataM partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *file, {}); } -std::unique_ptr MergeTreePartition::store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const +std::unique_ptr +MergeTreePartition::store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); const auto & context = storage.getContext(); @@ -423,7 +454,11 @@ std::unique_ptr MergeTreePartition::store(const MergeTr return store(partition_key_sample, data_part_storage, checksums, context->getWriteSettings()); } -std::unique_ptr MergeTreePartition::store(const Block & partition_key_sample, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums, const WriteSettings & settings) const +std::unique_ptr MergeTreePartition::store( + const Block & partition_key_sample, + IDataPartStorage & data_part_storage, + MergeTreeDataPartChecksums & checksums, + const WriteSettings & settings) const { if (!partition_key_sample) return nullptr; @@ -469,7 +504,8 @@ void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Bl } } -NamesAndTypesList MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context) +NamesAndTypesList +MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context) { auto adjusted_partition_key = adjustPartitionKey(metadata_snapshot, context); adjusted_partition_key.expression->execute(block); @@ -496,7 +532,7 @@ KeyDescription MergeTreePartition::adjustPartitionKey(const StorageMetadataPtr & } -void MergeTreePartition::appendFiles(const MergeTreeData & storage, Strings& files) +void MergeTreePartition::appendFiles(const MergeTreeData & storage, Strings & files) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (!metadata_snapshot->hasPartitionKey()) diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 78b141f26ec..855f912877f 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -1,11 +1,12 @@ #pragma once -#include +#include #include #include #include #include -#include +#include +#include namespace DB { @@ -28,10 +29,10 @@ struct MergeTreePartition public: MergeTreePartition() = default; - explicit MergeTreePartition(Row value_) : value(std::move(value_)) {} + explicit MergeTreePartition(Row value_) : value(std::move(value_)) { } /// For month-based partitioning. - explicit MergeTreePartition(UInt32 yyyymm) : value(1, yyyymm) {} + explicit MergeTreePartition(UInt32 yyyymm) : value(1, yyyymm) { } String getID(const MergeTreeData & storage) const; String getID(const Block & partition_key_sample) const; @@ -44,8 +45,13 @@ public: /// Store functions return write buffer with written but not finalized data. /// User must call finish() for returned object. - [[nodiscard]] std::unique_ptr store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const; - [[nodiscard]] std::unique_ptr store(const Block & partition_key_sample, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums, const WriteSettings & settings) const; + [[nodiscard]] std::unique_ptr + store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const; + [[nodiscard]] std::unique_ptr store( + const Block & partition_key_sample, + IDataPartStorage & data_part_storage, + MergeTreeDataPartChecksums & checksums, + const WriteSettings & settings) const; void assign(const MergeTreePartition & other) { value = other.value; } From 49502ac0ce66b9ff465b312511bb430e9065a0e5 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Sat, 29 Jul 2023 23:47:56 +0000 Subject: [PATCH 614/777] revert unrelated style change. Signed-off-by: Jianfei Hu --- src/Storages/MergeTree/MergeTreePartition.cpp | 132 +++++++++--------- src/Storages/MergeTree/MergeTreePartition.h | 18 +-- 2 files changed, 69 insertions(+), 81 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index b8d1879cd67..e4482aadecd 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -1,20 +1,20 @@ -#include -#include +#include +#include +#include +#include +#include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include "Common/logger_useful.h" -#include -#include -#include +#include #include +#include +#include #include +#include +#include + namespace DB { @@ -38,52 +38,51 @@ namespace { private: SipHash & hash; - public: - explicit LegacyFieldVisitorHash(SipHash & hash_) : hash(hash_) { } + explicit LegacyFieldVisitorHash(SipHash & hash_) : hash(hash_) {} - void operator()(const Null &) const + void operator() (const Null &) const { UInt8 type = Field::Types::Null; hash.update(type); } - void operator()(const UInt64 & x) const + void operator() (const UInt64 & x) const { UInt8 type = Field::Types::UInt64; hash.update(type); hash.update(x); } - void operator()(const UInt128 & x) const + void operator() (const UInt128 & x) const { UInt8 type = Field::Types::UInt128; hash.update(type); hash.update(x); } - void operator()(const UInt256 & x) const + void operator() (const UInt256 & x) const { UInt8 type = Field::Types::UInt256; hash.update(type); hash.update(x); } - void operator()(const Int64 & x) const + void operator() (const Int64 & x) const { UInt8 type = Field::Types::Int64; hash.update(type); hash.update(x); } - void operator()(const Int128 & x) const + void operator() (const Int128 & x) const { UInt8 type = Field::Types::Int128; hash.update(type); hash.update(x); } - void operator()(const Int256 & x) const + void operator() (const Int256 & x) const { UInt8 type = Field::Types::Int256; hash.update(type); hash.update(x); } - void operator()(const UUID & x) const + void operator() (const UUID & x) const { #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ auto tmp_x = x.toUnderType(); @@ -95,27 +94,30 @@ namespace operator()(x.toUnderType()); #endif } - void operator()(const IPv4 & x) const + void operator() (const IPv4 & x) const { UInt8 type = Field::Types::IPv4; hash.update(type); hash.update(x); } - void operator()(const IPv6 & x) const { return operator()(String(reinterpret_cast(&x), 16)); } - void operator()(const Float64 & x) const + void operator() (const IPv6 & x) const + { + return operator()(String(reinterpret_cast(&x), 16)); + } + void operator() (const Float64 & x) const { UInt8 type = Field::Types::Float64; hash.update(type); hash.update(x); } - void operator()(const String & x) const + void operator() (const String & x) const { UInt8 type = Field::Types::String; hash.update(type); hash.update(x.size()); hash.update(x.data(), x.size()); } - void operator()(const Array & x) const + void operator() (const Array & x) const { UInt8 type = Field::Types::Array; hash.update(type); @@ -124,7 +126,7 @@ namespace for (const auto & elem : x) applyVisitor(*this, elem); } - void operator()(const Tuple & x) const + void operator() (const Tuple & x) const { UInt8 type = Field::Types::Tuple; hash.update(type); @@ -133,7 +135,7 @@ namespace for (const auto & elem : x) applyVisitor(*this, elem); } - void operator()(const Map & x) const + void operator() (const Map & x) const { UInt8 type = Field::Types::Map; hash.update(type); @@ -142,43 +144,43 @@ namespace for (const auto & elem : x) applyVisitor(*this, elem); } - void operator()(const Object & x) const + void operator() (const Object & x) const { UInt8 type = Field::Types::Object; hash.update(type); hash.update(x.size()); - for (const auto & [key, value] : x) + for (const auto & [key, value]: x) { hash.update(key); applyVisitor(*this, value); } } - void operator()(const DecimalField & x) const + void operator() (const DecimalField & x) const { UInt8 type = Field::Types::Decimal32; hash.update(type); hash.update(x.getValue().value); } - void operator()(const DecimalField & x) const + void operator() (const DecimalField & x) const { UInt8 type = Field::Types::Decimal64; hash.update(type); hash.update(x.getValue().value); } - void operator()(const DecimalField & x) const + void operator() (const DecimalField & x) const { UInt8 type = Field::Types::Decimal128; hash.update(type); hash.update(x.getValue().value); } - void operator()(const DecimalField & x) const + void operator() (const DecimalField & x) const { UInt8 type = Field::Types::Decimal256; hash.update(type); hash.update(x.getValue().value); } - void operator()(const AggregateFunctionStateData & x) const + void operator() (const AggregateFunctionStateData & x) const { UInt8 type = Field::Types::AggregateFunctionState; hash.update(type); @@ -187,7 +189,7 @@ namespace hash.update(x.data.size()); hash.update(x.data.data(), x.data.size()); } - void operator()(const CustomType & x) const + void operator() (const CustomType & x) const { UInt8 type = Field::Types::CustomType; hash.update(type); @@ -196,7 +198,7 @@ namespace hash.update(result.size()); hash.update(result.data(), result.size()); } - void operator()(const bool & x) const + void operator() (const bool & x) const { UInt8 type = Field::Types::Bool; hash.update(type); @@ -289,12 +291,7 @@ std::optional MergeTreePartition::tryParseValueFromID(const String & partit return res; } - enum KeyType - { - DATE, - UNSIGNED, - SIGNED - }; + enum KeyType { DATE, UNSIGNED, SIGNED }; std::vector key_types; key_types.reserve(num_keys); @@ -324,7 +321,8 @@ std::optional MergeTreePartition::tryParseValueFromID(const String & partit { switch (key_types[i]) { - case DATE: { + case DATE: + { UInt32 date_yyyymmdd; readText(date_yyyymmdd, buf); constexpr UInt32 min_yyyymmdd = 10000000; @@ -337,13 +335,15 @@ std::optional MergeTreePartition::tryParseValueFromID(const String & partit res.emplace_back(date); break; } - case UNSIGNED: { + case UNSIGNED: + { UInt64 value; readText(value, buf); res.emplace_back(value); break; } - case SIGNED: { + case SIGNED: + { Int64 value; readText(value, buf); res.emplace_back(value); @@ -359,8 +359,8 @@ std::optional MergeTreePartition::tryParseValueFromID(const String & partit String expected_partition_id = MergeTreePartition{res}.getID(partition_key_sample); if (expected_partition_id != partition_id) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "Partition ID was parsed incorrectly: expected {}, got {}", expected_partition_id, partition_id); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Partition ID was parsed incorrectly: expected {}, got {}", + expected_partition_id, partition_id); return res; } @@ -388,17 +388,17 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe // } if (value.empty()) { - LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug: find the root cause, value is empty"); + // LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug: find the root cause, value is empty"); writeCString("tuple()", out); - LOG_INFO( - &Poco::Logger::get("DNSCacheUpdater"), - "jianfeih debug: type is nil: {}, colum is nil: {}, storage has partition key: {}, column size: {}, " - "column names: {} ", - type.get() == nullptr, - column == nullptr, - metadata_snapshot->hasPartitionKey(), - metadata_snapshot->columns.size(), - all_column_names.toString()); + // LOG_INFO( + // &Poco::Logger::get("DNSCacheUpdater"), + // "jianfeih debug: type is nil: {}, colum is nil: {}, storage has partition key: {}, column size: {}, " + // "column names: {} ", + // type.get() == nullptr, + // column == nullptr, + // metadata_snapshot->hasPartitionKey(), + // metadata_snapshot->columns.size(), + // all_column_names.toString()); return; } else @@ -445,8 +445,7 @@ void MergeTreePartition::load(const MergeTreeData & storage, const PartMetadataM partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *file, {}); } -std::unique_ptr -MergeTreePartition::store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const +std::unique_ptr MergeTreePartition::store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); const auto & context = storage.getContext(); @@ -454,11 +453,7 @@ MergeTreePartition::store(const MergeTreeData & storage, IDataPartStorage & data return store(partition_key_sample, data_part_storage, checksums, context->getWriteSettings()); } -std::unique_ptr MergeTreePartition::store( - const Block & partition_key_sample, - IDataPartStorage & data_part_storage, - MergeTreeDataPartChecksums & checksums, - const WriteSettings & settings) const +std::unique_ptr MergeTreePartition::store(const Block & partition_key_sample, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums, const WriteSettings & settings) const { if (!partition_key_sample) return nullptr; @@ -504,8 +499,7 @@ void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Bl } } -NamesAndTypesList -MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context) +NamesAndTypesList MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context) { auto adjusted_partition_key = adjustPartitionKey(metadata_snapshot, context); adjusted_partition_key.expression->execute(block); @@ -532,7 +526,7 @@ KeyDescription MergeTreePartition::adjustPartitionKey(const StorageMetadataPtr & } -void MergeTreePartition::appendFiles(const MergeTreeData & storage, Strings & files) +void MergeTreePartition::appendFiles(const MergeTreeData & storage, Strings& files) { auto metadata_snapshot = storage.getInMemoryMetadataPtr(); if (!metadata_snapshot->hasPartitionKey()) diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 855f912877f..78b141f26ec 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -1,12 +1,11 @@ #pragma once -#include +#include #include #include #include #include -#include -#include +#include namespace DB { @@ -29,10 +28,10 @@ struct MergeTreePartition public: MergeTreePartition() = default; - explicit MergeTreePartition(Row value_) : value(std::move(value_)) { } + explicit MergeTreePartition(Row value_) : value(std::move(value_)) {} /// For month-based partitioning. - explicit MergeTreePartition(UInt32 yyyymm) : value(1, yyyymm) { } + explicit MergeTreePartition(UInt32 yyyymm) : value(1, yyyymm) {} String getID(const MergeTreeData & storage) const; String getID(const Block & partition_key_sample) const; @@ -45,13 +44,8 @@ public: /// Store functions return write buffer with written but not finalized data. /// User must call finish() for returned object. - [[nodiscard]] std::unique_ptr - store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const; - [[nodiscard]] std::unique_ptr store( - const Block & partition_key_sample, - IDataPartStorage & data_part_storage, - MergeTreeDataPartChecksums & checksums, - const WriteSettings & settings) const; + [[nodiscard]] std::unique_ptr store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const; + [[nodiscard]] std::unique_ptr store(const Block & partition_key_sample, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums, const WriteSettings & settings) const; void assign(const MergeTreePartition & other) { value = other.value; } From 1e600351d91c3d8e188edb39fe2521bb94fabf26 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Mon, 14 Aug 2023 22:25:47 +0000 Subject: [PATCH 615/777] try again the if value.empty() in the beginning. Signed-off-by: Jianfei Hu --- src/Storages/MergeTree/MergeTreePartition.cpp | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index e4482aadecd..5fc33e1ae67 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -371,6 +371,11 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe const auto & partition_key_sample = metadata_snapshot->getPartitionKey().sample_block; size_t key_size = partition_key_sample.columns(); + if (value.empty()) + { + writeCString("tuple()", out); + return; + } if (key_size == 0) { writeCString("tuple()", out); @@ -386,10 +391,10 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe // { // LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug type is not nil, name {}.", type->getName()); // } - if (value.empty()) - { + // if (value.empty()) + // { // LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug: find the root cause, value is empty"); - writeCString("tuple()", out); + // writeCString("tuple()", out); // LOG_INFO( // &Poco::Logger::get("DNSCacheUpdater"), // "jianfeih debug: type is nil: {}, colum is nil: {}, storage has partition key: {}, column size: {}, " @@ -399,10 +404,10 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe // metadata_snapshot->hasPartitionKey(), // metadata_snapshot->columns.size(), // all_column_names.toString()); - return; - } - else - { + // return; + // } + // else + // { // LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug: value[0] to be inserted: {}", value[0]); // should be here causing the segfault. column might be empty. some type created column is not okay. // let's check the table definition; type; parts info at this point. @@ -410,7 +415,7 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe column->insert(value[0]); type->getDefaultSerialization()->serializeText(*column, 0, out, format_settings); // LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug: finish the serialization no error."); - } + // } } else { From fcad27372448280bc68e3e9b395bd68e40c2cdd4 Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Mon, 14 Aug 2023 23:25:31 +0000 Subject: [PATCH 616/777] remove the comments. Signed-off-by: Jianfei Hu --- src/Storages/MergeTree/MergeTreePartition.cpp | 34 ++----------------- 1 file changed, 3 insertions(+), 31 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 5fc33e1ae67..72017e823b0 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -371,6 +371,7 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe const auto & partition_key_sample = metadata_snapshot->getPartitionKey().sample_block; size_t key_size = partition_key_sample.columns(); + // In some cases we create empty parts and value is empty. if (value.empty()) { writeCString("tuple()", out); @@ -384,38 +385,9 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe { const DataTypePtr & type = partition_key_sample.getByPosition(0).type; auto column = type->createColumn(); - // metadata_snapshot-> - // TODO: here print out the type that causing the segfault. const auto & all_column_names = metadata_snapshot->getColumns().getAll(); - // if (type.get() != nullptr) - // { - // LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug type is not nil, name {}.", type->getName()); - // } - // if (value.empty()) - // { - // LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug: find the root cause, value is empty"); - // writeCString("tuple()", out); - // LOG_INFO( - // &Poco::Logger::get("DNSCacheUpdater"), - // "jianfeih debug: type is nil: {}, colum is nil: {}, storage has partition key: {}, column size: {}, " - // "column names: {} ", - // type.get() == nullptr, - // column == nullptr, - // metadata_snapshot->hasPartitionKey(), - // metadata_snapshot->columns.size(), - // all_column_names.toString()); - // return; - // } - // else - // { - // LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug: value[0] to be inserted: {}", value[0]); - // should be here causing the segfault. column might be empty. some type created column is not okay. - // let's check the table definition; type; parts info at this point. - // logging is not right, need to figure out include header properly. - column->insert(value[0]); - type->getDefaultSerialization()->serializeText(*column, 0, out, format_settings); - // LOG_INFO(&Poco::Logger::get("DNSCacheUpdater"), "jianfeih debug: finish the serialization no error."); - // } + column->insert(value[0]); + type->getDefaultSerialization()->serializeText(*column, 0, out, format_settings); } else { From 0ec69f23606bc95f6823497ed37e507bd9bea5ff Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 16 Aug 2023 18:03:50 +0000 Subject: [PATCH 617/777] krb5: Fix CVE-2023-36054 --- contrib/krb5 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/krb5 b/contrib/krb5 index b56ce6ba690..1d5c970e936 160000 --- a/contrib/krb5 +++ b/contrib/krb5 @@ -1 +1 @@ -Subproject commit b56ce6ba690e1f320df1a64afa34980c3e462617 +Subproject commit 1d5c970e9369f444caf81d1d06a231a6bad8581f From 56888912368c2e4620fbe95322cd53e443e79626 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 16 Aug 2023 19:00:27 +0000 Subject: [PATCH 618/777] curl: update to latest master (fixes CVE-2023-32001) --- contrib/curl | 2 +- contrib/curl-cmake/CMakeLists.txt | 4 +--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/contrib/curl b/contrib/curl index b0edf0b7dae..eb3b049df52 160000 --- a/contrib/curl +++ b/contrib/curl @@ -1 +1 @@ -Subproject commit b0edf0b7dae44d9e66f270a257cf654b35d5263d +Subproject commit eb3b049df526bf125eda23218e680ce7fa9ec46c diff --git a/contrib/curl-cmake/CMakeLists.txt b/contrib/curl-cmake/CMakeLists.txt index 70d9c2816dc..ca4edd77b3a 100644 --- a/contrib/curl-cmake/CMakeLists.txt +++ b/contrib/curl-cmake/CMakeLists.txt @@ -40,7 +40,6 @@ set (SRCS "${LIBRARY_DIR}/lib/strcase.c" "${LIBRARY_DIR}/lib/easy.c" "${LIBRARY_DIR}/lib/curl_fnmatch.c" - "${LIBRARY_DIR}/lib/curl_log.c" "${LIBRARY_DIR}/lib/fileinfo.c" "${LIBRARY_DIR}/lib/krb5.c" "${LIBRARY_DIR}/lib/memdebug.c" @@ -113,6 +112,7 @@ set (SRCS "${LIBRARY_DIR}/lib/psl.c" "${LIBRARY_DIR}/lib/doh.c" "${LIBRARY_DIR}/lib/urlapi.c" + "${LIBRARY_DIR}/lib/curl_trc.c" "${LIBRARY_DIR}/lib/curl_get_line.c" "${LIBRARY_DIR}/lib/altsvc.c" "${LIBRARY_DIR}/lib/socketpair.c" @@ -142,12 +142,10 @@ set (SRCS "${LIBRARY_DIR}/lib/vtls/openssl.c" "${LIBRARY_DIR}/lib/vtls/gtls.c" "${LIBRARY_DIR}/lib/vtls/vtls.c" - "${LIBRARY_DIR}/lib/vtls/nss.c" "${LIBRARY_DIR}/lib/vtls/wolfssl.c" "${LIBRARY_DIR}/lib/vtls/schannel.c" "${LIBRARY_DIR}/lib/vtls/schannel_verify.c" "${LIBRARY_DIR}/lib/vtls/sectransp.c" - "${LIBRARY_DIR}/lib/vtls/gskit.c" "${LIBRARY_DIR}/lib/vtls/mbedtls.c" "${LIBRARY_DIR}/lib/vtls/bearssl.c" "${LIBRARY_DIR}/lib/vtls/keylog.c" From ea8a99c103342c28f20d0b07ed19318e9f2d5b9f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 16 Aug 2023 19:04:28 +0000 Subject: [PATCH 619/777] Sort stuff --- contrib/curl-cmake/CMakeLists.txt | 224 +++++++++++++++--------------- 1 file changed, 112 insertions(+), 112 deletions(-) diff --git a/contrib/curl-cmake/CMakeLists.txt b/contrib/curl-cmake/CMakeLists.txt index ca4edd77b3a..733865d5101 100644 --- a/contrib/curl-cmake/CMakeLists.txt +++ b/contrib/curl-cmake/CMakeLists.txt @@ -8,125 +8,122 @@ endif() set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/curl") set (SRCS - "${LIBRARY_DIR}/lib/fopen.c" - "${LIBRARY_DIR}/lib/noproxy.c" - "${LIBRARY_DIR}/lib/idn.c" - "${LIBRARY_DIR}/lib/cfilters.c" - "${LIBRARY_DIR}/lib/cf-socket.c" + "${LIBRARY_DIR}/lib/altsvc.c" + "${LIBRARY_DIR}/lib/amigaos.c" + "${LIBRARY_DIR}/lib/asyn-thread.c" + "${LIBRARY_DIR}/lib/base64.c" + "${LIBRARY_DIR}/lib/bufq.c" + "${LIBRARY_DIR}/lib/bufref.c" + "${LIBRARY_DIR}/lib/cf-h1-proxy.c" "${LIBRARY_DIR}/lib/cf-haproxy.c" "${LIBRARY_DIR}/lib/cf-https-connect.c" - "${LIBRARY_DIR}/lib/file.c" - "${LIBRARY_DIR}/lib/timeval.c" - "${LIBRARY_DIR}/lib/base64.c" - "${LIBRARY_DIR}/lib/hostip.c" - "${LIBRARY_DIR}/lib/progress.c" - "${LIBRARY_DIR}/lib/formdata.c" - "${LIBRARY_DIR}/lib/cookie.c" - "${LIBRARY_DIR}/lib/http.c" - "${LIBRARY_DIR}/lib/sendf.c" - "${LIBRARY_DIR}/lib/url.c" - "${LIBRARY_DIR}/lib/dict.c" - "${LIBRARY_DIR}/lib/if2ip.c" - "${LIBRARY_DIR}/lib/speedcheck.c" - "${LIBRARY_DIR}/lib/ldap.c" - "${LIBRARY_DIR}/lib/version.c" - "${LIBRARY_DIR}/lib/getenv.c" - "${LIBRARY_DIR}/lib/escape.c" - "${LIBRARY_DIR}/lib/mprintf.c" - "${LIBRARY_DIR}/lib/telnet.c" - "${LIBRARY_DIR}/lib/netrc.c" - "${LIBRARY_DIR}/lib/getinfo.c" - "${LIBRARY_DIR}/lib/transfer.c" - "${LIBRARY_DIR}/lib/strcase.c" - "${LIBRARY_DIR}/lib/easy.c" - "${LIBRARY_DIR}/lib/curl_fnmatch.c" - "${LIBRARY_DIR}/lib/fileinfo.c" - "${LIBRARY_DIR}/lib/krb5.c" - "${LIBRARY_DIR}/lib/memdebug.c" - "${LIBRARY_DIR}/lib/http_chunks.c" - "${LIBRARY_DIR}/lib/strtok.c" + "${LIBRARY_DIR}/lib/cf-socket.c" + "${LIBRARY_DIR}/lib/cfilters.c" + "${LIBRARY_DIR}/lib/conncache.c" "${LIBRARY_DIR}/lib/connect.c" - "${LIBRARY_DIR}/lib/llist.c" - "${LIBRARY_DIR}/lib/hash.c" - "${LIBRARY_DIR}/lib/multi.c" "${LIBRARY_DIR}/lib/content_encoding.c" - "${LIBRARY_DIR}/lib/share.c" - "${LIBRARY_DIR}/lib/http_digest.c" - "${LIBRARY_DIR}/lib/md4.c" - "${LIBRARY_DIR}/lib/md5.c" - "${LIBRARY_DIR}/lib/http_negotiate.c" - "${LIBRARY_DIR}/lib/inet_pton.c" - "${LIBRARY_DIR}/lib/strtoofft.c" - "${LIBRARY_DIR}/lib/strerror.c" - "${LIBRARY_DIR}/lib/amigaos.c" + "${LIBRARY_DIR}/lib/cookie.c" + "${LIBRARY_DIR}/lib/curl_addrinfo.c" + "${LIBRARY_DIR}/lib/curl_des.c" + "${LIBRARY_DIR}/lib/curl_endian.c" + "${LIBRARY_DIR}/lib/curl_fnmatch.c" + "${LIBRARY_DIR}/lib/curl_get_line.c" + "${LIBRARY_DIR}/lib/curl_gethostname.c" + "${LIBRARY_DIR}/lib/curl_gssapi.c" + "${LIBRARY_DIR}/lib/curl_memrchr.c" + "${LIBRARY_DIR}/lib/curl_multibyte.c" + "${LIBRARY_DIR}/lib/curl_ntlm_core.c" + "${LIBRARY_DIR}/lib/curl_ntlm_wb.c" + "${LIBRARY_DIR}/lib/curl_path.c" + "${LIBRARY_DIR}/lib/curl_range.c" + "${LIBRARY_DIR}/lib/curl_rtmp.c" + "${LIBRARY_DIR}/lib/curl_sasl.c" + "${LIBRARY_DIR}/lib/curl_sspi.c" + "${LIBRARY_DIR}/lib/curl_threads.c" + "${LIBRARY_DIR}/lib/curl_trc.c" + "${LIBRARY_DIR}/lib/dict.c" + "${LIBRARY_DIR}/lib/doh.c" + "${LIBRARY_DIR}/lib/dynbuf.c" + "${LIBRARY_DIR}/lib/dynhds.c" + "${LIBRARY_DIR}/lib/easy.c" + "${LIBRARY_DIR}/lib/escape.c" + "${LIBRARY_DIR}/lib/file.c" + "${LIBRARY_DIR}/lib/fileinfo.c" + "${LIBRARY_DIR}/lib/fopen.c" + "${LIBRARY_DIR}/lib/formdata.c" + "${LIBRARY_DIR}/lib/getenv.c" + "${LIBRARY_DIR}/lib/getinfo.c" + "${LIBRARY_DIR}/lib/gopher.c" + "${LIBRARY_DIR}/lib/hash.c" + "${LIBRARY_DIR}/lib/headers.c" + "${LIBRARY_DIR}/lib/hmac.c" "${LIBRARY_DIR}/lib/hostasyn.c" + "${LIBRARY_DIR}/lib/hostip.c" "${LIBRARY_DIR}/lib/hostip4.c" "${LIBRARY_DIR}/lib/hostip6.c" "${LIBRARY_DIR}/lib/hostsyn.c" + "${LIBRARY_DIR}/lib/hsts.c" + "${LIBRARY_DIR}/lib/http.c" + "${LIBRARY_DIR}/lib/http2.c" + "${LIBRARY_DIR}/lib/http_aws_sigv4.c" + "${LIBRARY_DIR}/lib/http_chunks.c" + "${LIBRARY_DIR}/lib/http_digest.c" + "${LIBRARY_DIR}/lib/http_negotiate.c" + "${LIBRARY_DIR}/lib/http_ntlm.c" + "${LIBRARY_DIR}/lib/http_proxy.c" + "${LIBRARY_DIR}/lib/idn.c" + "${LIBRARY_DIR}/lib/if2ip.c" + "${LIBRARY_DIR}/lib/imap.c" "${LIBRARY_DIR}/lib/inet_ntop.c" + "${LIBRARY_DIR}/lib/inet_pton.c" + "${LIBRARY_DIR}/lib/krb5.c" + "${LIBRARY_DIR}/lib/ldap.c" + "${LIBRARY_DIR}/lib/llist.c" + "${LIBRARY_DIR}/lib/md4.c" + "${LIBRARY_DIR}/lib/md5.c" + "${LIBRARY_DIR}/lib/memdebug.c" + "${LIBRARY_DIR}/lib/mime.c" + "${LIBRARY_DIR}/lib/mprintf.c" + "${LIBRARY_DIR}/lib/mqtt.c" + "${LIBRARY_DIR}/lib/multi.c" + "${LIBRARY_DIR}/lib/netrc.c" + "${LIBRARY_DIR}/lib/nonblock.c" + "${LIBRARY_DIR}/lib/noproxy.c" + "${LIBRARY_DIR}/lib/openldap.c" "${LIBRARY_DIR}/lib/parsedate.c" + "${LIBRARY_DIR}/lib/pingpong.c" + "${LIBRARY_DIR}/lib/pop3.c" + "${LIBRARY_DIR}/lib/progress.c" + "${LIBRARY_DIR}/lib/psl.c" + "${LIBRARY_DIR}/lib/rand.c" + "${LIBRARY_DIR}/lib/rename.c" + "${LIBRARY_DIR}/lib/rtsp.c" "${LIBRARY_DIR}/lib/select.c" - "${LIBRARY_DIR}/lib/splay.c" - "${LIBRARY_DIR}/lib/strdup.c" + "${LIBRARY_DIR}/lib/sendf.c" + "${LIBRARY_DIR}/lib/setopt.c" + "${LIBRARY_DIR}/lib/sha256.c" + "${LIBRARY_DIR}/lib/share.c" + "${LIBRARY_DIR}/lib/slist.c" + "${LIBRARY_DIR}/lib/smb.c" + "${LIBRARY_DIR}/lib/smtp.c" + "${LIBRARY_DIR}/lib/socketpair.c" "${LIBRARY_DIR}/lib/socks.c" - "${LIBRARY_DIR}/lib/curl_addrinfo.c" "${LIBRARY_DIR}/lib/socks_gssapi.c" "${LIBRARY_DIR}/lib/socks_sspi.c" - "${LIBRARY_DIR}/lib/curl_sspi.c" - "${LIBRARY_DIR}/lib/slist.c" - "${LIBRARY_DIR}/lib/nonblock.c" - "${LIBRARY_DIR}/lib/curl_memrchr.c" - "${LIBRARY_DIR}/lib/imap.c" - "${LIBRARY_DIR}/lib/pop3.c" - "${LIBRARY_DIR}/lib/smtp.c" - "${LIBRARY_DIR}/lib/pingpong.c" - "${LIBRARY_DIR}/lib/rtsp.c" - "${LIBRARY_DIR}/lib/curl_threads.c" - "${LIBRARY_DIR}/lib/warnless.c" - "${LIBRARY_DIR}/lib/hmac.c" - "${LIBRARY_DIR}/lib/curl_rtmp.c" - "${LIBRARY_DIR}/lib/openldap.c" - "${LIBRARY_DIR}/lib/curl_gethostname.c" - "${LIBRARY_DIR}/lib/gopher.c" - "${LIBRARY_DIR}/lib/http_proxy.c" - "${LIBRARY_DIR}/lib/asyn-thread.c" - "${LIBRARY_DIR}/lib/curl_gssapi.c" - "${LIBRARY_DIR}/lib/http_ntlm.c" - "${LIBRARY_DIR}/lib/curl_ntlm_wb.c" - "${LIBRARY_DIR}/lib/curl_ntlm_core.c" - "${LIBRARY_DIR}/lib/curl_sasl.c" - "${LIBRARY_DIR}/lib/rand.c" - "${LIBRARY_DIR}/lib/curl_multibyte.c" - "${LIBRARY_DIR}/lib/conncache.c" - "${LIBRARY_DIR}/lib/cf-h1-proxy.c" - "${LIBRARY_DIR}/lib/http2.c" - "${LIBRARY_DIR}/lib/smb.c" - "${LIBRARY_DIR}/lib/curl_endian.c" - "${LIBRARY_DIR}/lib/curl_des.c" + "${LIBRARY_DIR}/lib/speedcheck.c" + "${LIBRARY_DIR}/lib/splay.c" + "${LIBRARY_DIR}/lib/strcase.c" + "${LIBRARY_DIR}/lib/strdup.c" + "${LIBRARY_DIR}/lib/strerror.c" + "${LIBRARY_DIR}/lib/strtok.c" + "${LIBRARY_DIR}/lib/strtoofft.c" "${LIBRARY_DIR}/lib/system_win32.c" - "${LIBRARY_DIR}/lib/mime.c" - "${LIBRARY_DIR}/lib/sha256.c" - "${LIBRARY_DIR}/lib/setopt.c" - "${LIBRARY_DIR}/lib/curl_path.c" - "${LIBRARY_DIR}/lib/curl_range.c" - "${LIBRARY_DIR}/lib/psl.c" - "${LIBRARY_DIR}/lib/doh.c" - "${LIBRARY_DIR}/lib/urlapi.c" - "${LIBRARY_DIR}/lib/curl_trc.c" - "${LIBRARY_DIR}/lib/curl_get_line.c" - "${LIBRARY_DIR}/lib/altsvc.c" - "${LIBRARY_DIR}/lib/socketpair.c" - "${LIBRARY_DIR}/lib/bufref.c" - "${LIBRARY_DIR}/lib/bufq.c" - "${LIBRARY_DIR}/lib/dynbuf.c" - "${LIBRARY_DIR}/lib/dynhds.c" - "${LIBRARY_DIR}/lib/hsts.c" - "${LIBRARY_DIR}/lib/http_aws_sigv4.c" - "${LIBRARY_DIR}/lib/mqtt.c" - "${LIBRARY_DIR}/lib/rename.c" - "${LIBRARY_DIR}/lib/headers.c" + "${LIBRARY_DIR}/lib/telnet.c" "${LIBRARY_DIR}/lib/timediff.c" - "${LIBRARY_DIR}/lib/vauth/vauth.c" + "${LIBRARY_DIR}/lib/timeval.c" + "${LIBRARY_DIR}/lib/transfer.c" + "${LIBRARY_DIR}/lib/url.c" + "${LIBRARY_DIR}/lib/urlapi.c" "${LIBRARY_DIR}/lib/vauth/cleartext.c" "${LIBRARY_DIR}/lib/vauth/cram.c" "${LIBRARY_DIR}/lib/vauth/digest.c" @@ -138,21 +135,24 @@ set (SRCS "${LIBRARY_DIR}/lib/vauth/oauth2.c" "${LIBRARY_DIR}/lib/vauth/spnego_gssapi.c" "${LIBRARY_DIR}/lib/vauth/spnego_sspi.c" + "${LIBRARY_DIR}/lib/vauth/vauth.c" + "${LIBRARY_DIR}/lib/version.c" "${LIBRARY_DIR}/lib/vquic/vquic.c" - "${LIBRARY_DIR}/lib/vtls/openssl.c" + "${LIBRARY_DIR}/lib/vssh/libssh.c" + "${LIBRARY_DIR}/lib/vssh/libssh2.c" + "${LIBRARY_DIR}/lib/vtls/bearssl.c" "${LIBRARY_DIR}/lib/vtls/gtls.c" - "${LIBRARY_DIR}/lib/vtls/vtls.c" - "${LIBRARY_DIR}/lib/vtls/wolfssl.c" + "${LIBRARY_DIR}/lib/vtls/hostcheck.c" + "${LIBRARY_DIR}/lib/vtls/keylog.c" + "${LIBRARY_DIR}/lib/vtls/mbedtls.c" + "${LIBRARY_DIR}/lib/vtls/openssl.c" "${LIBRARY_DIR}/lib/vtls/schannel.c" "${LIBRARY_DIR}/lib/vtls/schannel_verify.c" "${LIBRARY_DIR}/lib/vtls/sectransp.c" - "${LIBRARY_DIR}/lib/vtls/mbedtls.c" - "${LIBRARY_DIR}/lib/vtls/bearssl.c" - "${LIBRARY_DIR}/lib/vtls/keylog.c" + "${LIBRARY_DIR}/lib/vtls/vtls.c" + "${LIBRARY_DIR}/lib/vtls/wolfssl.c" "${LIBRARY_DIR}/lib/vtls/x509asn1.c" - "${LIBRARY_DIR}/lib/vtls/hostcheck.c" - "${LIBRARY_DIR}/lib/vssh/libssh2.c" - "${LIBRARY_DIR}/lib/vssh/libssh.c" + "${LIBRARY_DIR}/lib/warnless.c" ) add_library (_curl ${SRCS}) From d5ed014ec4e4a2a0c49ac95a193aa0c15a511f4c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 16 Aug 2023 22:56:32 +0300 Subject: [PATCH 620/777] Fix flaky test `02443_detach_attach_partition` (#53478) * fix flaky test * empty commit --- .../02443_detach_attach_partition.reference | 4 ++-- .../02443_detach_attach_partition.sh | 18 ++++++++++++++---- 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02443_detach_attach_partition.reference b/tests/queries/0_stateless/02443_detach_attach_partition.reference index 70930ea6d9a..77cfb77479d 100644 --- a/tests/queries/0_stateless/02443_detach_attach_partition.reference +++ b/tests/queries/0_stateless/02443_detach_attach_partition.reference @@ -1,4 +1,4 @@ default begin inserts default end inserts -20 210 -20 210 +30 465 +30 465 diff --git a/tests/queries/0_stateless/02443_detach_attach_partition.sh b/tests/queries/0_stateless/02443_detach_attach_partition.sh index 5a3f1b64065..ae104b833e3 100755 --- a/tests/queries/0_stateless/02443_detach_attach_partition.sh +++ b/tests/queries/0_stateless/02443_detach_attach_partition.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, zookeeper, no-parallel +# Tags: race, zookeeper, long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -29,9 +29,19 @@ function thread_attach() done } +insert_type=$(($RANDOM % 3)) +$CLICKHOUSE_CLIENT -q "SELECT '$CLICKHOUSE_DATABASE', 'insert_type $insert_type' FORMAT Null" + function insert() { - $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $RANDOM, $i" 2>/dev/null + # Fault injection may lead to duplicates + if [[ "$insert_type" -eq 0 ]]; then + $CLICKHOUSE_CLIENT --insert_deduplication_token=$1 -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $RANDOM, $1" 2>/dev/null + elif [[ "$insert_type" -eq 1 ]]; then + $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $1, $1" 2>/dev/null + else + $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $RANDOM, $1" 2>/dev/null + fi } thread_detach & PID_1=$! @@ -41,8 +51,8 @@ thread_attach & PID_4=$! function do_inserts() { - for i in {1..20}; do - while ! insert; do $CLICKHOUSE_CLIENT -q "SELECT '$CLICKHOUSE_DATABASE', 'retrying insert $i' FORMAT Null"; done + for i in {1..30}; do + while ! insert $i; do $CLICKHOUSE_CLIENT -q "SELECT '$CLICKHOUSE_DATABASE', 'retrying insert $i' FORMAT Null"; done done } From 30ccfa1638356dafaa7b84c8bdb11077874aa99a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 16 Aug 2023 20:23:10 +0000 Subject: [PATCH 621/777] Fix wrong columns order for queries with parallel FINAL. --- src/Processors/QueryPlan/PartsSplitter.cpp | 27 +++++++++++++++++++ ...rallel_final_wrong_columns_order.reference | 1 + ...841_parallel_final_wrong_columns_order.sql | 5 ++++ 3 files changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.reference create mode 100644 tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql diff --git a/src/Processors/QueryPlan/PartsSplitter.cpp b/src/Processors/QueryPlan/PartsSplitter.cpp index 533fbde1e13..61c6422de5a 100644 --- a/src/Processors/QueryPlan/PartsSplitter.cpp +++ b/src/Processors/QueryPlan/PartsSplitter.cpp @@ -254,6 +254,32 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +static void reorderColumns(ActionsDAG & dag, const Block & header, const std::string & filter_column) +{ + std::unordered_map inputs_map; + for (const auto * input : dag.getInputs()) + inputs_map[input->result_name] = input; + + for (const auto & col : header) + { + auto & input = inputs_map[col.name]; + if (!input) + input = &dag.addInput(col); + } + + ActionsDAG::NodeRawConstPtrs new_outputs; + new_outputs.reserve(header.columns() + 1); + + new_outputs.push_back(&dag.findInOutputs(filter_column)); + for (const auto & col : header) + { + auto & input = inputs_map[col.name]; + new_outputs.push_back(input); + } + + dag.getOutputs() = std::move(new_outputs); +} + Pipes buildPipesForReadingByPKRanges( const KeyDescription & primary_key, ExpressionActionsPtr sorting_expr, @@ -279,6 +305,7 @@ Pipes buildPipesForReadingByPKRanges( continue; auto syntax_result = TreeRewriter(context).analyze(filter_function, primary_key.expression->getRequiredColumnsWithTypes()); auto actions = ExpressionAnalyzer(filter_function, syntax_result, context).getActionsDAG(false); + reorderColumns(*actions, pipes[i].getHeader(), filter_function->getColumnName()); ExpressionActionsPtr expression_actions = std::make_shared(std::move(actions)); auto description = fmt::format( "filter values in [{}, {})", i ? ::toString(borders[i - 1]) : "-inf", i < borders.size() ? ::toString(borders[i]) : "+inf"); diff --git a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.reference b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.reference new file mode 100644 index 00000000000..749fce669df --- /dev/null +++ b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.reference @@ -0,0 +1 @@ +1000000 diff --git a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql new file mode 100644 index 00000000000..12ff8972761 --- /dev/null +++ b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql @@ -0,0 +1,5 @@ +drop table if exists tab2; +create table tab2 (id String, version Int64, l String, accountCode String, z Int32) engine = ReplacingMergeTree(z) PRIMARY KEY (accountCode, id) ORDER BY (accountCode, id, version, l); +insert into tab2 select toString(number), number, toString(number), toString(number), 0 from numbers(1e6); +set max_threads=2; +select count() from tab2 final; From 91c241532fefcab27f0277c8ebc0538645aeaa69 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 16 Aug 2023 20:26:23 +0000 Subject: [PATCH 622/777] Add a check for header structure. --- src/Processors/Transforms/FilterSortedStreamByRange.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Processors/Transforms/FilterSortedStreamByRange.h b/src/Processors/Transforms/FilterSortedStreamByRange.h index e1141ebd299..7f38c10d691 100644 --- a/src/Processors/Transforms/FilterSortedStreamByRange.h +++ b/src/Processors/Transforms/FilterSortedStreamByRange.h @@ -27,6 +27,9 @@ public: true) , filter_transform(header_, expression_, filter_column_name_, remove_filter_column_, on_totals_) { + assertBlocksHaveEqualStructure( + header_, getOutputPort().getHeader(), + "Expression for FilterSortedStreamByRange shoud not change header"); } String getName() const override { return "FilterSortedStreamByRange"; } From 08e4e2bb0b1f2ce790f0f1b6cf8dbad45db330f2 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 16 Aug 2023 22:35:45 +0200 Subject: [PATCH 623/777] fix --- src/Client/ClientBase.cpp | 8 ++- .../ReplaceQueryParameterVisitor.cpp | 7 ++- .../ReplaceQueryParameterVisitor.h | 4 ++ src/Interpreters/executeQuery.cpp | 5 +- .../01018_dictionaries_from_dictionaries.sql | 59 ++++++++++--------- ...rfluous_dict_reload_on_create_database.sql | 38 ++++++------ 6 files changed, 71 insertions(+), 50 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 9092f12b62b..495e3bdfd4e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -847,7 +847,9 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa visitor.visit(parsed_query); /// Get new query after substitutions. - query = serializeAST(*parsed_query); + if (visitor.getNumberOfReplacedParameters()) + query = serializeAST(*parsed_query); + chassert(!query.empty()); } if (allow_merge_tree_settings && parsed_query->as()) @@ -1332,7 +1334,9 @@ void ClientBase::processInsertQuery(const String & query_to_execute, ASTPtr pars visitor.visit(parsed_query); /// Get new query after substitutions. - query = serializeAST(*parsed_query); + if (visitor.getNumberOfReplacedParameters()) + query = serializeAST(*parsed_query); + chassert(!query.empty()); } /// Process the query that requires transferring data blocks to the server. diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.cpp b/src/Interpreters/ReplaceQueryParameterVisitor.cpp index 89a40baaff9..46dcc6129bc 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.cpp +++ b/src/Interpreters/ReplaceQueryParameterVisitor.cpp @@ -66,10 +66,11 @@ void ReplaceQueryParameterVisitor::visitChildren(ASTPtr & ast) const String & ReplaceQueryParameterVisitor::getParamValue(const String & name) { auto search = query_parameters.find(name); - if (search != query_parameters.end()) - return search->second; - else + if (search == query_parameters.end()) throw Exception(ErrorCodes::UNKNOWN_QUERY_PARAMETER, "Substitution {} is not set", backQuote(name)); + + ++num_replaced_parameters; + return search->second; } void ReplaceQueryParameterVisitor::visitQueryParameter(ASTPtr & ast) diff --git a/src/Interpreters/ReplaceQueryParameterVisitor.h b/src/Interpreters/ReplaceQueryParameterVisitor.h index dd785cd768e..7d5da7ea85b 100644 --- a/src/Interpreters/ReplaceQueryParameterVisitor.h +++ b/src/Interpreters/ReplaceQueryParameterVisitor.h @@ -20,8 +20,12 @@ public: void visit(ASTPtr & ast); + size_t getNumberOfReplacedParameters() const { return num_replaced_parameters; } + private: const NameToNameMap & query_parameters; + size_t num_replaced_parameters = 0; + const String & getParamValue(const String & name); void visitIdentifier(ASTPtr & ast); void visitQueryParameter(ASTPtr & ast); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 1e6843f65e6..bac6807b682 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -735,7 +735,10 @@ static std::tuple executeQueryImpl( { ReplaceQueryParameterVisitor visitor(context->getQueryParameters()); visitor.visit(ast); - query = serializeAST(*ast); + if (visitor.getNumberOfReplacedParameters()) + query = serializeAST(*ast); + else + query.assign(begin, query_end); } else { diff --git a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql index d958d7d58ef..e72e113f859 100644 --- a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql +++ b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql @@ -1,7 +1,12 @@ +-- Tags: no-parallel SET send_logs_level = 'fatal'; -CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict +DROP DATABASE IF EXISTS database_for_dict; + +CREATE DATABASE database_for_dict; + +CREATE TABLE database_for_dict.table_for_dict ( key_column UInt64, second_column UInt8, @@ -11,9 +16,9 @@ CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.table_for_dict ENGINE = MergeTree() ORDER BY key_column; -INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100); +INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100); -CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1 +CREATE DICTIONARY database_for_dict.dict1 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -21,13 +26,13 @@ CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict1 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase())) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; +SELECT count(*) from database_for_dict.dict1; -CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict2 +CREATE DICTIONARY database_for_dict.dict2 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -35,20 +40,20 @@ CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict2 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict1' DB currentDatabase())) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict1' DB 'database_for_dict')) LIFETIME(MIN 1 MAX 10) LAYOUT(HASHED()); -SELECT count(*) FROM {CLICKHOUSE_DATABASE:Identifier}.dict2; +SELECT count(*) FROM database_for_dict.dict2; -INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100, 100); +INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(100, 100); SYSTEM RELOAD DICTIONARIES; -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict2; -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; +SELECT count(*) from database_for_dict.dict2; +SELECT count(*) from database_for_dict.dict1; -CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict3 +CREATE DICTIONARY database_for_dict.dict3 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -56,22 +61,22 @@ CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict3 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict2' DB currentDatabase())) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict2' DB 'database_for_dict')) LIFETIME(MIN 1 MAX 10) LAYOUT(HASHED()); -SELECT count(*) FROM {CLICKHOUSE_DATABASE:Identifier}.dict3; +SELECT count(*) FROM database_for_dict.dict3; -INSERT INTO {CLICKHOUSE_DATABASE:Identifier}.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(200, 100); +INSERT INTO database_for_dict.table_for_dict SELECT number, number % 17, toString(number * number), number / 2.0 from numbers(200, 100); SYSTEM RELOAD DICTIONARIES; -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict3; -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict2; -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; +SELECT count(*) from database_for_dict.dict3; +SELECT count(*) from database_for_dict.dict2; +SELECT count(*) from database_for_dict.dict1; -CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict4 +CREATE DICTIONARY database_for_dict.dict4 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, @@ -79,17 +84,17 @@ CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict4 fourth_column Float64 DEFAULT 42.0 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'non_existing_table' DB currentDatabase())) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'non_existing_table' DB 'database_for_dict')) LIFETIME(MIN 1 MAX 10) LAYOUT(HASHED()); -SELECT count(*) FROM {CLICKHOUSE_DATABASE:Identifier}.dict4; -- {serverError 60} +SELECT count(*) FROM database_for_dict.dict4; -- {serverError 60} -SELECT name from system.tables WHERE database = currentDatabase() ORDER BY name; -SELECT name from system.dictionaries WHERE database = currentDatabase() ORDER BY name; +SELECT name from system.tables WHERE database = 'database_for_dict' ORDER BY name; +SELECT name from system.dictionaries WHERE database = 'database_for_dict' ORDER BY name; -DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}; +DROP DATABASE IF EXISTS database_for_dict; -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict3; --{serverError 81} -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict2; --{serverError 81} -SELECT count(*) from {CLICKHOUSE_DATABASE:Identifier}.dict1; --{serverError 81} +SELECT count(*) from database_for_dict.dict3; --{serverError 81} +SELECT count(*) from database_for_dict.dict2; --{serverError 81} +SELECT count(*) from database_for_dict.dict1; --{serverError 81} diff --git a/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql b/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql index bbd848f81d2..9040d7b3231 100644 --- a/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql +++ b/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql @@ -1,31 +1,35 @@ +-- Tags: no-parallel -CREATE TABLE {CLICKHOUSE_DATABASE:Identifier}.dict_data (key UInt64, val UInt64) Engine=Memory(); -CREATE DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict +DROP DATABASE IF EXISTS dict_db_01036; +CREATE DATABASE dict_db_01036; + +CREATE TABLE dict_db_01036.dict_data (key UInt64, val UInt64) Engine=Memory(); +CREATE DICTIONARY dict_db_01036.dict ( key UInt64 DEFAULT 0, val UInt64 DEFAULT 10 ) PRIMARY KEY key -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB currentDatabase())) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'dict_data' PASSWORD '' DB 'dict_db_01036')) LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()); -SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; -SELECT dictGetUInt64({CLICKHOUSE_DATABASE:String}||'.dict', 'val', toUInt64(0)); -SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; +SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; +SELECT dictGetUInt64('dict_db_01036.dict', 'val', toUInt64(0)); +SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; SELECT 'SYSTEM RELOAD DICTIONARY'; -SYSTEM RELOAD DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict; -SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; -SELECT dictGetUInt64({CLICKHOUSE_DATABASE:String}||'.dict', 'val', toUInt64(0)); -SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; +SYSTEM RELOAD DICTIONARY dict_db_01036.dict; +SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; +SELECT dictGetUInt64('dict_db_01036.dict', 'val', toUInt64(0)); +SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; SELECT 'CREATE DATABASE'; -DROP DATABASE IF EXISTS {CLICKHOUSE_DATABASE_1:Identifier}; -CREATE DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; -SELECT query_count FROM system.dictionaries WHERE database = currentDatabase() AND name = 'dict'; +DROP DATABASE IF EXISTS empty_db_01036; +CREATE DATABASE empty_db_01036; +SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; -DROP DICTIONARY {CLICKHOUSE_DATABASE:Identifier}.dict; -DROP TABLE {CLICKHOUSE_DATABASE:Identifier}.dict_data; -DROP DATABASE {CLICKHOUSE_DATABASE:Identifier}; -DROP DATABASE {CLICKHOUSE_DATABASE_1:Identifier}; +DROP DICTIONARY dict_db_01036.dict; +DROP TABLE dict_db_01036.dict_data; +DROP DATABASE dict_db_01036; +DROP DATABASE empty_db_01036; From 3a6a3d491efaca938ef6ba709ae9d5136b22f057 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 16 Aug 2023 20:40:28 +0000 Subject: [PATCH 624/777] Fix typo --- src/Processors/Transforms/FilterSortedStreamByRange.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/FilterSortedStreamByRange.h b/src/Processors/Transforms/FilterSortedStreamByRange.h index 7f38c10d691..e3d3f6f10ef 100644 --- a/src/Processors/Transforms/FilterSortedStreamByRange.h +++ b/src/Processors/Transforms/FilterSortedStreamByRange.h @@ -29,7 +29,7 @@ public: { assertBlocksHaveEqualStructure( header_, getOutputPort().getHeader(), - "Expression for FilterSortedStreamByRange shoud not change header"); + "Expression for FilterSortedStreamByRange should not change header"); } String getName() const override { return "FilterSortedStreamByRange"; } From 970eb24067ac54ed709a6ca4823b37e8d891a578 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 16 Aug 2023 22:57:49 +0200 Subject: [PATCH 625/777] fix --- tests/integration/test_grpc_protocol/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 5b56c3da8dc..c149e1a4096 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -352,13 +352,13 @@ def test_authentication(): def test_logs(): - query = "SELECT has(groupArray(number), 42) FROM numbers(1000)" + query = "SELECT has(groupArray(number), 42) FROM numbers(100000)" logs = query_and_get_logs( query, settings={"send_logs_level": "debug"}, ) assert query in logs - assert "Read 1 rows" in logs + assert "Read 100000 rows" in logs assert "Peak memory usage" in logs From b711802c43290ce059a374057a9df73881ac42b2 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 16 Aug 2023 21:23:11 +0000 Subject: [PATCH 626/777] Fix spelling --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2e231120e41..02358580b16 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -315,6 +315,7 @@ Greenwald HDDs HHMM HMAC +HNSW HSTS HTTPConnection HTTPThreads @@ -697,6 +698,7 @@ Promtail Protobuf ProtobufSingle ProxySQL +PyArrow PyCharm QEMU QTCreator @@ -921,6 +923,7 @@ URL's URLHash URLHierarchy URLPathHierarchy +USearch UUIDNumToString UUIDStringToNum UUIDs @@ -1086,8 +1089,8 @@ authenticators autocompletion autodetect autodetected -autogenerated autogenerate +autogenerated autogeneration autostart avgWeighted @@ -2000,7 +2003,6 @@ ptrs pushdown pwrite py -PyArrow qryn quantile quantileBFloat @@ -2499,6 +2501,7 @@ uring url urlCluster urls +usearch userspace userver utils From 3c36af111d559a883d1c85662a9eacd02309531f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 16 Aug 2023 23:47:46 +0200 Subject: [PATCH 627/777] Update style checker --- docker/test/style/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index bd1c26855b7..a4feae27c67 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -1,5 +1,5 @@ # docker build -t clickhouse/style-test . -FROM ubuntu:20.04 +FROM ubuntu:22.04 ARG ACT_VERSION=0.2.33 ARG ACTIONLINT_VERSION=1.6.22 From a0a6ec18095db249b124414e6e7d524018de6205 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Aug 2023 01:15:40 +0300 Subject: [PATCH 628/777] Update 02841_parallel_final_wrong_columns_order.sql --- .../0_stateless/02841_parallel_final_wrong_columns_order.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql index 12ff8972761..3f4658e6abe 100644 --- a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql +++ b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql @@ -3,3 +3,4 @@ create table tab2 (id String, version Int64, l String, accountCode String, z Int insert into tab2 select toString(number), number, toString(number), toString(number), 0 from numbers(1e6); set max_threads=2; select count() from tab2 final; +DROP TABLE tab2; From 53fdbac7b039fba7aa6f68fb88c73e350d00a0fc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Aug 2023 01:16:58 +0300 Subject: [PATCH 629/777] Update 02841_parallel_final_wrong_columns_order.sql --- .../0_stateless/02841_parallel_final_wrong_columns_order.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql index 3f4658e6abe..8a28388cf15 100644 --- a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql +++ b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql @@ -1,3 +1,6 @@ +-- Tags: no-random-settings +-- Because we insert one million rows, it shouldn't choose too low index granularity. + drop table if exists tab2; create table tab2 (id String, version Int64, l String, accountCode String, z Int32) engine = ReplacingMergeTree(z) PRIMARY KEY (accountCode, id) ORDER BY (accountCode, id, version, l); insert into tab2 select toString(number), number, toString(number), toString(number), 0 from numbers(1e6); From 368191af2fc55e528691765a4a564fa46fc14e76 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Aug 2023 01:18:00 +0300 Subject: [PATCH 630/777] Update 02841_parallel_final_wrong_columns_order.sql --- .../0_stateless/02841_parallel_final_wrong_columns_order.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql index 8a28388cf15..db15abb28cb 100644 --- a/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql +++ b/tests/queries/0_stateless/02841_parallel_final_wrong_columns_order.sql @@ -1,4 +1,4 @@ --- Tags: no-random-settings +-- Tags: no-random-merge-tree-settings -- Because we insert one million rows, it shouldn't choose too low index granularity. drop table if exists tab2; From d7272410bdc8d9f19830a6acb01e165471cdd2ff Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 17 Aug 2023 00:42:51 +0200 Subject: [PATCH 631/777] the worst fix for a deadlock ever --- src/Interpreters/InterpreterDropQuery.cpp | 3 ++- src/Interpreters/InterpreterDropQuery.h | 2 +- src/Storages/StorageMaterializedView.cpp | 7 +++++-- tests/clickhouse-test | 8 ++++---- 4 files changed, 12 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index ed927d550a8..55256e8f9f7 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -432,8 +432,9 @@ AccessRightsElements InterpreterDropQuery::getRequiredAccessForDDLOnCluster() co } void InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, - const StorageID & target_table_id, bool sync, bool ignore_sync_setting) + const StorageID & target_table_id, bool sync, bool ignore_sync_setting, bool need_ddl_guard) { + auto ddl_guard = (need_ddl_guard ? DatabaseCatalog::instance().getDDLGuard(target_table_id.database_name, target_table_id.table_name) : nullptr); if (DatabaseCatalog::instance().tryGetTable(target_table_id, current_context)) { /// We create and execute `drop` query for internal table. diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index af7a4ddef25..7ae544a7356 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -25,7 +25,7 @@ public: BlockIO execute() override; static void executeDropQuery(ASTDropQuery::Kind kind, ContextPtr global_context, ContextPtr current_context, - const StorageID & target_table_id, bool sync, bool ignore_sync_setting = false); + const StorageID & target_table_id, bool sync, bool ignore_sync_setting = false, bool need_ddl_guard = false); bool supportsTransactions() const override; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 16d724d54d8..7354dd56552 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -228,10 +228,13 @@ void StorageMaterializedView::dropInnerTableIfAny(bool sync, ContextPtr local_co { /// We will use `sync` argument wneh this function is called from a DROP query /// and will ignore database_atomic_wait_for_drop_and_detach_synchronously when it's called from drop task. - /// See the comment in StorageMaterializedView::drop + /// See the comment in StorageMaterializedView::drop. + /// DDL queries with StorageMaterializedView are fundamentally broken. + /// Best-effort to make them work: the inner table name is almost always less than the MV name (so it's safe to lock DDLGuard) + bool may_lock_ddl_guard = getStorageID().getQualifiedName() < target_table_id.getQualifiedName(); if (has_inner_table && tryGetTargetTable()) InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, getContext(), local_context, target_table_id, - sync, /* ignore_sync_setting */ true); + sync, /* ignore_sync_setting */ true, may_lock_ddl_guard); } void StorageMaterializedView::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 1ce5ad981ad..49a56476a41 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -383,19 +383,19 @@ def get_stacktraces_from_clickhouse(args): ) replicated_msg = ( f"{args.client} {settings_str} --query " - '"SELECT materialize((hostName(), tcpPort())) as host, thread_id, ' + '"SELECT materialize((hostName(), tcpPort())) as host, thread_name, thread_id, query_id, trace, ' "arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), " "arrayMap(x -> addressToLine(x), trace), " - "arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace " + "arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace_str " "FROM clusterAllReplicas('test_cluster_database_replicated', 'system.stack_trace') " 'ORDER BY host, thread_id FORMAT Vertical"' ) msg = ( f"{args.client} {settings_str} --query " - "\"SELECT arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), " + "\"SELECT thread_name, thread_id, query_id, trace, arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), " "arrayMap(x -> addressToLine(x), trace), " - "arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace " + "arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace_str " 'FROM system.stack_trace FORMAT Vertical"' ) From c6fc31c1e36cd981c370fd98baf83cc670c7d584 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 16 Aug 2023 23:06:42 +0000 Subject: [PATCH 632/777] Review suggestions --- src/Server/ServerType.cpp | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/src/Server/ServerType.cpp b/src/Server/ServerType.cpp index 883be4b0892..fb052e7d6e6 100644 --- a/src/Server/ServerType.cpp +++ b/src/Server/ServerType.cpp @@ -63,18 +63,13 @@ bool ServerType::shouldStart(Type server_type, const std::string & server_custom } }; - auto is_type_custom = [](Type current_type) - { - return current_type == Type::CUSTOM; - }; - if (exclude_types.contains(Type::QUERIES_ALL)) return false; if (exclude_types.contains(Type::QUERIES_DEFAULT) && is_type_default(server_type)) return false; - if (exclude_types.contains(Type::QUERIES_CUSTOM) && is_type_custom(server_type)) + if (exclude_types.contains(Type::QUERIES_CUSTOM) && server_type == Type::CUSTOM) return false; if (exclude_types.contains(server_type)) @@ -93,7 +88,7 @@ bool ServerType::shouldStart(Type server_type, const std::string & server_custom return is_type_default(server_type); if (type == Type::QUERIES_CUSTOM) - return is_type_custom(server_type); + return server_type == Type::CUSTOM; if (type == Type::CUSTOM) return server_type == type && server_custom_name == custom_name; From 87685a7c0c824ba9cdcd64115b491302f1ead5a4 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 17 Aug 2023 00:49:00 +0200 Subject: [PATCH 633/777] Address shellcheck issues --- tests/queries/0_stateless/00301_csv.sh | 10 +++++----- .../0_stateless/00900_long_parquet_load.sh | 6 +++--- .../0_stateless/01747_system_session_log_long.sh | 16 ++++++++++------ .../0_stateless/02833_concurrrent_sessions.sh | 15 ++++++++++----- tests/queries/0_stateless/replication.lib | 4 +++- 5 files changed, 31 insertions(+), 20 deletions(-) diff --git a/tests/queries/0_stateless/00301_csv.sh b/tests/queries/0_stateless/00301_csv.sh index 80053c99a17..2e2e1384534 100755 --- a/tests/queries/0_stateless/00301_csv.sh +++ b/tests/queries/0_stateless/00301_csv.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -echo === Test input_format_csv_empty_as_default +echo '=== Test input_format_csv_empty_as_default' $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS csv"; $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (s String, n UInt64 DEFAULT 1, d Date DEFAULT '2019-06-19') ENGINE = Memory"; @@ -19,7 +19,7 @@ Hello "world", 789 ,2016-01-03 $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY d, s"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; -echo === Test datetime +echo '=== Test datetime' $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (t DateTime('Asia/Istanbul'), s String) ENGINE = Memory"; echo '"2016-01-01 01:02:03","1" @@ -30,7 +30,7 @@ echo '"2016-01-01 01:02:03","1" $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; -echo === Test nullable datetime +echo '=== Test nullable datetime' $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (t Nullable(DateTime('Asia/Istanbul')), s Nullable(String)) ENGINE = Memory"; echo 'NULL, NULL @@ -41,7 +41,7 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s NULLS LAST"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; -echo === Test ignore extra columns +echo '=== Test ignore extra columns' $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (s String, n UInt64 DEFAULT 3, d String DEFAULT 'String4') ENGINE = Memory"; echo '"Hello", 1, "String1" @@ -55,7 +55,7 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s, n"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; -echo === Test missing as default +echo '=== Test missing as default' $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (f1 String, f2 UInt64, f3 UInt256, f4 UInt64 Default 33, f5 Nullable(UInt64), f6 Nullable(UInt64) Default 55, f7 String DEFAULT 'Default') ENGINE = Memory"; echo ' diff --git a/tests/queries/0_stateless/00900_long_parquet_load.sh b/tests/queries/0_stateless/00900_long_parquet_load.sh index 60317e1671c..2f19cce9107 100755 --- a/tests/queries/0_stateless/00900_long_parquet_load.sh +++ b/tests/queries/0_stateless/00900_long_parquet_load.sh @@ -43,9 +43,9 @@ for NAME in $(find "$DATA_DIR"/*.parquet -print0 | xargs -0 -n 1 basename | LC_A JSON=$DATA_DIR/$NAME.json COLUMNS_FILE=$DATA_DIR/$NAME.columns - ([ -z "$PARQUET_READER" ] || [ ! -s "$PARQUET_READER" ]) && [ ! -s "$COLUMNS_FILE" ] && continue + { [ -z "$PARQUET_READER" ] || [ ! -s "$PARQUET_READER" ]; } && [ ! -s "$COLUMNS_FILE" ] && continue - echo === Try load data from "$NAME" + echo "=== Try load data from $NAME" # If you want change or add .parquet file - rm data_parquet/*.json data_parquet/*.columns [ -n "$PARQUET_READER" ] && [ ! -s "$COLUMNS_FILE" ] && [ ! -s "$JSON" ] && "$PARQUET_READER" --json "$DATA_DIR"/"$NAME" > "$JSON" @@ -64,7 +64,7 @@ CREATE TABLE parquet_load ($COLUMNS) ENGINE = Memory; EOF # Some files contain unsupported data structures, exception is ok. - cat "$DATA_DIR"/"$NAME" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_load FORMAT Parquet" 2>&1 | sed 's/Exception/Ex---tion/' + ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_load FORMAT Parquet" < "$DATA_DIR"/"$NAME" 2>&1 | sed 's/Exception/Ex---tion/' ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load ORDER BY tuple(*) LIMIT 100" ${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_load" diff --git a/tests/queries/0_stateless/01747_system_session_log_long.sh b/tests/queries/0_stateless/01747_system_session_log_long.sh index c6e93f4abd7..6bd58facb51 100755 --- a/tests/queries/0_stateless/01747_system_session_log_long.sh +++ b/tests/queries/0_stateless/01747_system_session_log_long.sh @@ -33,8 +33,10 @@ set -eu # Since there is no way to cleanup system.session_log table, # make sure that we can identify log entries from this test by a random user name. -readonly BASE_USERNAME="session_log_test_user_$(cat /dev/urandom | tr -cd 'a-f0-9' | head -c 32)" -readonly TMP_QUERY_FILE=$(mktemp /tmp/tmp_query.log.XXXXXX) +BASE_USERNAME="session_log_test_user_$(tr -cd 'a-f0-9' < /dev/urandom | head -c 32)" +readonly BASE_USERNAME +TMP_QUERY_FILE=$(mktemp /tmp/tmp_query.log.XXXXXX) +readonly TMP_QUERY_FILE declare -a ALL_USERNAMES ALL_USERNAMES+=("${BASE_USERNAME}") @@ -80,7 +82,7 @@ trap "cleanup" EXIT function executeQueryExpectError() { cat - > "${TMP_QUERY_FILE}" - ! ${CLICKHOUSE_CLIENT} --multiquery --queries-file "${TMP_QUERY_FILE}" "${@}" 2>&1 | tee -a ${TMP_QUERY_FILE} + ! ${CLICKHOUSE_CLIENT} --multiquery --queries-file "${TMP_QUERY_FILE}" "${@}" 2>&1 | tee -a "${TMP_QUERY_FILE}" } function createUser() @@ -95,7 +97,8 @@ function createUser() elif [[ "${auth_type}" == "plaintext_password" ]] then - password="${password}" + # password="${password}" + : elif [[ "${auth_type}" == "sha256_password" ]] then @@ -198,7 +201,7 @@ function testHTTPNamedSession() { echo "HTTP endpoint with named session" local HTTP_SESSION_ID - HTTP_SESSION_ID="session_id_$(cat /dev/urandom | tr -cd 'a-f0-9' | head -c 32)" + HTTP_SESSION_ID="session_id_$(tr -cd 'a-f0-9' < /dev/urandom | head -c 32)" if [ -v CLICKHOUSE_URL_PARAMS ] then CLICKHOUSE_URL_WITH_SESSION_ID="${CLICKHOUSE_URL}&session_id=${HTTP_SESSION_ID}" @@ -343,7 +346,8 @@ SET DEFAULT ROLE session_log_test_role, session_log_test_role2 TO ${username}; } # to cut off previous runs -readonly start_time="$(executeQuery <<< 'SELECT now64(6);')" +start_time="$(executeQuery <<< 'SELECT now64(6);')" +readonly start_time # Special case: user and profile are both defined in XML runEndpointTests "User with profile from XML" "no_password" "session_log_test_xml_user" '' diff --git a/tests/queries/0_stateless/02833_concurrrent_sessions.sh b/tests/queries/0_stateless/02833_concurrrent_sessions.sh index 26b48462a76..c5b6204529b 100755 --- a/tests/queries/0_stateless/02833_concurrrent_sessions.sh +++ b/tests/queries/0_stateless/02833_concurrrent_sessions.sh @@ -14,11 +14,16 @@ readonly HTTP_WITH_SESSION_ID_SESSION_USERS=( "02833_HTTP_WITH_SESSION_ID_USER_$ readonly MYSQL_USERS=( "02833_MYSQL_USER_${PID}") readonly ALL_USERS=( "${TCP_USERS[@]}" "${HTTP_USERS[@]}" "${HTTP_WITH_SESSION_ID_SESSION_USERS[@]}" "${MYSQL_USERS[@]}" ) -readonly TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" -readonly ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +TCP_USERS_SQL_COLLECTION_STRING="$( echo "${TCP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly TCP_USERS_SQL_COLLECTION_STRING +HTTP_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_USERS_SQL_COLLECTION_STRING +HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING="$( echo "${HTTP_WITH_SESSION_ID_SESSION_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly HTTP_WITH_SESSION_ID_USERS_SQL_COLLECTION_STRING +MYSQL_USERS_SQL_COLLECTION_STRING="$( echo "${MYSQL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly MYSQL_USERS_SQL_COLLECTION_STRING +ALL_USERS_SQL_COLLECTION_STRING="$( echo "${ALL_USERS[*]}" | sed "s/[^[:space:]]\+/'&'/g" | sed 's/[[:space:]]/,/g' )" +readonly ALL_USERS_SQL_COLLECTION_STRING readonly SESSION_LOG_MATCHING_FIELDS="auth_id, auth_type, client_version_major, client_version_minor, client_version_patch, interface" diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 1805b56f8dc..e0e11990d83 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -37,7 +37,7 @@ function try_sync_replicas() pids[${i}]=$! i=$((i + 1)) done - for pid in ${pids[*]}; do + for pid in "${pids[@]}"; do wait $pid || (echo "Failed to sync some replicas" && exit 1) done echo "Replication did not hang: synced all replicas of $table_name_prefix" @@ -114,3 +114,5 @@ function check_replication_consistency() fi } + +# vi: ft=bash From 21ef1f1d1c9ba399948239dd166e2d08bb32db0b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 16 Aug 2023 23:24:01 +0000 Subject: [PATCH 634/777] Remove unnecessary Scalars usage for cluster_for_parallel_replicas --- src/Processors/QueryPlan/ReadFromRemote.cpp | 15 +++++++++++++-- src/Storages/StorageMergeTree.cpp | 12 ------------ src/Storages/StorageReplicatedMergeTree.cpp | 11 ----------- 3 files changed, 13 insertions(+), 25 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index bf9a87a692e..0c3d106b658 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -239,9 +239,20 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact if (context->getParallelReplicasMode() == Context::ParallelReplicasMode::READ_TASKS) { + if (context->getSettingsRef().cluster_for_parallel_replicas.changed) + { + const String cluster_for_parallel_replicas = context->getSettingsRef().cluster_for_parallel_replicas; + if (cluster_for_parallel_replicas != cluster_name) + { + LOG_INFO(log, "cluster_for_parallel_replicas was set for the query but has no effect: {}. Distributed table cluster is used: {}", cluster_for_parallel_replicas, cluster_name); + } + } + + context->setSetting("cluster_for_parallel_replicas", cluster_name); + /// the cluster is defined by Distributed table and passed to shards via `_cluster_for_parallel_replicas` scalar value - scalars["_cluster_for_parallel_replicas"] = - Block{{DataTypeString().createColumnConst(1, cluster_name), std::make_shared(), "_cluster_for_parallel_replicas"}}; + // scalars["_cluster_for_parallel_replicas"] = + // Block{{DataTypeString().createColumnConst(1, cluster_name), std::make_shared(), "_cluster_for_parallel_replicas"}}; } std::shared_ptr remote_query_executor; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9d46b0a07aa..c7bedd3aa4a 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -221,18 +221,6 @@ void StorageMergeTree::read( table_id.database_name, table_id.table_name, /*remote_table_function_ptr*/nullptr); String cluster_for_parallel_replicas = local_context->getSettingsRef().cluster_for_parallel_replicas; - { - /// if parallel replicas query executed over Distributed table, - /// the cluster is defined by Distributed table and passed to shards via `_cluster_for_parallel_replicas` scalar value - const auto scalars = local_context->hasQueryContext() ? local_context->getQueryContext()->getScalars() : Scalars{}; - const auto it = scalars.find("_cluster_for_parallel_replicas"); - if (it != scalars.end()) - { - const Block & block = it->second; - const auto & column = block.safeGetByPosition(0).column; - cluster_for_parallel_replicas = column->getDataAt(0).toString(); - } - } auto cluster = local_context->getCluster(cluster_for_parallel_replicas); Block header; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ba0ce4b76a1..e3698df945a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5155,17 +5155,6 @@ void StorageReplicatedMergeTree::readParallelReplicasImpl( auto scalars = local_context->hasQueryContext() ? local_context->getQueryContext()->getScalars() : Scalars{}; String cluster_for_parallel_replicas = local_context->getSettingsRef().cluster_for_parallel_replicas; - { - /// if parallel replicas query executed over Distributed table, - /// the cluster is defined by Distributed table and passed to shards via `_cluster_for_parallel_replicas` scalar value - const auto it = scalars.find("_cluster_for_parallel_replicas"); - if (it != scalars.end()) - { - const Block & block = it->second; - const auto & column = block.safeGetByPosition(0).column; - cluster_for_parallel_replicas = column->getDataAt(0).toString(); - } - } auto parallel_replicas_cluster = local_context->getCluster(cluster_for_parallel_replicas); ASTPtr modified_query_ast; From 2e4d346e445cf8dae1d5b79bb8ab62ad311324d3 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 16 Aug 2023 23:31:20 +0000 Subject: [PATCH 635/777] Fixes --- src/Interpreters/ClusterProxy/executeQuery.cpp | 2 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 10 ++-------- 2 files changed, 3 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index a42ec25cda8..0c2ad35515f 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -296,7 +296,7 @@ void executeQueryWithParallelReplicas( else new_cluster = not_optimized_cluster->getClusterWithReplicasAsShards(settings); - auto all_replicas_count = std::min(static_cast(settings.max_parallel_replicas), new_cluster->getShardCount()); + auto all_replicas_count = std::min(static_cast(settings.max_parallel_replicas), not_optimized_cluster->getShardCount()); auto coordinator = std::make_shared(all_replicas_count); /// This is a little bit weird, but we construct an "empty" coordinator without diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 0c3d106b658..2ec6c82898f 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -243,16 +243,10 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact { const String cluster_for_parallel_replicas = context->getSettingsRef().cluster_for_parallel_replicas; if (cluster_for_parallel_replicas != cluster_name) - { - LOG_INFO(log, "cluster_for_parallel_replicas was set for the query but has no effect: {}. Distributed table cluster is used: {}", cluster_for_parallel_replicas, cluster_name); - } + LOG_INFO(log, "cluster_for_parallel_replicas has been set for the query but has no effect: {}. Distributed table cluster is used: {}", + cluster_for_parallel_replicas, cluster_name); } - context->setSetting("cluster_for_parallel_replicas", cluster_name); - - /// the cluster is defined by Distributed table and passed to shards via `_cluster_for_parallel_replicas` scalar value - // scalars["_cluster_for_parallel_replicas"] = - // Block{{DataTypeString().createColumnConst(1, cluster_name), std::make_shared(), "_cluster_for_parallel_replicas"}}; } std::shared_ptr remote_query_executor; From 857856b8b674c46e4c768780efdc9631a1fdcc87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Aug 2023 03:58:32 +0200 Subject: [PATCH 636/777] Leave only simplifications --- docker/test/performance-comparison/compare.sh | 38 ------------------- tests/ci/performance_comparison_check.py | 18 --------- 2 files changed, 56 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index ce8c4903c00..4b1b5c13b9b 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -63,25 +63,6 @@ function left_or_right() function configure { - # Setup a cluster for logs export to ClickHouse Cloud - # Note: these variables are provided to the Docker run command by the Python script in tests/ci - if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] - then - set +x - echo " -remote_servers: - system_logs_export: - shard: - replica: - secure: 1 - user: ci - host: '${CLICKHOUSE_CI_LOGS_HOST}' - port: 9440 - password: '${CLICKHOUSE_CI_LOGS_PASSWORD}' -" > right/config/config.d/system_logs_export.yaml - set -x - fi - # Use the new config for both servers, so that we can change it in a PR. rm right/config/config.d/text_log.xml ||: cp -rv right/config left ||: @@ -111,25 +92,6 @@ remote_servers: wait_for_server $LEFT_SERVER_PORT $left_pid echo "Server for setup started" - # Initialize export of system logs to ClickHouse Cloud - # Note: it is set up for the "left" server, and its database is then cloned to the "right" server. - if [ -n "${CLICKHOUSE_CI_LOGS_HOST}" ] - then - ( - set +x - export EXTRA_COLUMNS_EXPRESSION="$PR_TO_TEST AS pull_request_number, '$SHA_TO_TEST' AS commit_sha, '$CHECK_START_TIME' AS check_start_time, '$CHECK_NAME' AS check_name, '$INSTANCE_TYPE' AS instance_type" - export CONNECTION_PARAMETERS="--secure --user ci --host ${CLICKHOUSE_CI_LOGS_HOST} --password ${CLICKHOUSE_CI_LOGS_PASSWORD}" - - /setup_export_logs.sh "--port $LEFT_SERVER_PORT" - - # Unset variables after use - export CONNECTION_PARAMETERS='' - export CLICKHOUSE_CI_LOGS_HOST='' - export CLICKHOUSE_CI_LOGS_PASSWORD='' - set -x - ) - fi - clickhouse-client --port $LEFT_SERVER_PORT --query "create database test" ||: clickhouse-client --port $LEFT_SERVER_PORT --query "rename table datasets.hits_v1 to test.hits" ||: diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index 70369f9881e..27a67e2ae0e 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -39,8 +39,6 @@ def get_run_command( instance_type = get_instance_type() envs = [ - "-e CLICKHOUSE_CI_LOGS_HOST", - "-e CLICKHOUSE_CI_LOGS_PASSWORD", f"-e CHECK_START_TIME='{check_start_time}'", f"-e CHECK_NAME='{check_name}'", f"-e INSTANCE_TYPE='{instance_type}'", @@ -203,22 +201,6 @@ if __name__ == "__main__": subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) - # Cleanup run log from the credentials of CI logs database. - # Note: a malicious user can still print them by splitting the value into parts. - # But we will be warned when a malicious user modifies CI script. - # Although they can also print them from inside tests. - # Nevertheless, the credentials of the CI logs have limited scope - # and does not provide access to sensitive info. - - ci_logs_host = os.getenv("CLICKHOUSE_CI_LOGS_HOST", "CLICKHOUSE_CI_LOGS_HOST") - ci_logs_password = os.getenv( - "CLICKHOUSE_CI_LOGS_PASSWORD", "CLICKHOUSE_CI_LOGS_PASSWORD" - ) - subprocess.check_call( - f"sed -i -r -e 's!{ci_logs_host}!CLICKHOUSE_CI_LOGS_HOST!g; s!{ci_logs_password}!CLICKHOUSE_CI_LOGS_PASSWORD!g;' '{run_log_path}' '{compare_log_path}'", - shell=True, - ) - paths = { "compare.log": compare_log_path, "output.7z": os.path.join(result_path, "output.7z"), From c3e6f7e9ae792b54ef713beb8a5513307af119f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Aug 2023 03:59:15 +0200 Subject: [PATCH 637/777] Leave only simplifications --- docker/test/base/setup_export_logs.sh | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index ef510552d2f..12fae855b03 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -1,5 +1,4 @@ #!/bin/bash -# shellcheck disable=SC2086 # This script sets up export of system log tables to a remote server. # Remote tables are created if not exist, and augmented with extra columns, @@ -8,7 +7,6 @@ # Pre-configured destination cluster, where to export the data CLUSTER=${CLUSTER:=system_logs_export} -LOCAL_PARAMETERS=$1 EXTRA_COLUMNS=${EXTRA_COLUMNS:="pull_request_number UInt32, commit_sha String, check_start_time DateTime, check_name LowCardinality(String), instance_type LowCardinality(String), "} EXTRA_COLUMNS_EXPRESSION=${EXTRA_COLUMNS_EXPRESSION:="0 AS pull_request_number, '' AS commit_sha, now() AS check_start_time, '' AS check_name, '' AS instance_type"} @@ -17,13 +15,13 @@ EXTRA_ORDER_BY_COLUMNS=${EXTRA_ORDER_BY_COLUMNS:="check_name, "} CONNECTION_PARAMETERS=${CONNECTION_PARAMETERS:=""} # Create all configured system logs: -clickhouse-client $LOCAL_PARAMETERS --query "SYSTEM FLUSH LOGS" +clickhouse-client --query "SYSTEM FLUSH LOGS" # For each system log table: -clickhouse-client $LOCAL_PARAMETERS --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table +clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table do # Calculate hash of its structure: - hash=$(clickhouse-client $LOCAL_PARAMETERS --query " + hash=$(clickhouse-client --query " SELECT sipHash64(groupArray((name, type))) FROM (SELECT name, type FROM system.columns WHERE database = 'system' AND table = '$table' @@ -31,7 +29,7 @@ do ") # Create the destination table with adapted name and structure: - statement=$(clickhouse-client $LOCAL_PARAMETERS --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e ' + statement=$(clickhouse-client --format TSVRaw --query "SHOW CREATE TABLE system.${table}" | sed -r -e ' s/^\($/('"$EXTRA_COLUMNS"'/; s/ORDER BY \(/ORDER BY ('"$EXTRA_ORDER_BY_COLUMNS"'/; s/^CREATE TABLE system\.\w+_log$/CREATE TABLE IF NOT EXISTS '"$table"'_'"$hash"'/; @@ -45,7 +43,7 @@ do echo "Creating table system.${table}_sender" >&2 # Create Distributed table and materialized view to watch on the original table: - clickhouse-client $LOCAL_PARAMETERS --query " + clickhouse-client --query " CREATE TABLE system.${table}_sender ENGINE = Distributed(${CLUSTER}, default, ${table}_${hash}) EMPTY AS @@ -55,7 +53,7 @@ do echo "Creating materialized view system.${table}_watcher" >&2 - clickhouse-client $LOCAL_PARAMETERS --query " + clickhouse-client --query " CREATE MATERIALIZED VIEW system.${table}_watcher TO system.${table}_sender AS SELECT ${EXTRA_COLUMNS_EXPRESSION}, * FROM system.${table} From c4777397f78212224bd92912d84ebbf0f86eed04 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 17 Aug 2023 14:42:51 +0800 Subject: [PATCH 638/777] fix integration test test_kafka_formats --- src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 45183a91f43..bc8b508f083 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -302,6 +302,7 @@ void ORCBlockInputFormat::resetParser() file_reader.reset(); stripe_reader.reset(); include_indices.clear(); + batch.reset(); block_missing_values.clear(); } From 17ca2661a1ce0498883d1fc801c0ddc9528efd66 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 17 Aug 2023 08:47:47 +0200 Subject: [PATCH 639/777] Add ability to turn off flush of Distributed on DETACH/DROP/server shutdown Sometimes you can have tons of data there, i.e. few TiBs, and sending them on server shutdown does not looks sane (maybe there is a bug and you need to update/restart to fix flushing). Signed-off-by: Azat Khuzhin --- .../Distributed/DistributedSettings.h | 1 + src/Storages/StorageDistributed.cpp | 6 ++++ ...2860_distributed_flush_on_detach.reference | 27 +++++++++++++++ .../02860_distributed_flush_on_detach.sql | 33 +++++++++++++++++++ 4 files changed, 67 insertions(+) create mode 100644 tests/queries/0_stateless/02860_distributed_flush_on_detach.reference create mode 100644 tests/queries/0_stateless/02860_distributed_flush_on_detach.sql diff --git a/src/Storages/Distributed/DistributedSettings.h b/src/Storages/Distributed/DistributedSettings.h index 86b77707dee..85e64cabfd7 100644 --- a/src/Storages/Distributed/DistributedSettings.h +++ b/src/Storages/Distributed/DistributedSettings.h @@ -26,6 +26,7 @@ class ASTStorage; M(UInt64, monitor_split_batch_on_failure, 0, "Default - distributed_directory_monitor_split_batch_on_failure", 0) \ M(Milliseconds, monitor_sleep_time_ms, 0, "Default - distributed_directory_monitor_sleep_time_ms", 0) \ M(Milliseconds, monitor_max_sleep_time_ms, 0, "Default - distributed_directory_monitor_max_sleep_time_ms", 0) \ + M(Bool, flush_on_detach, true, "Flush data to remote nodes on DETACH/DROP/server shutdown", 0) \ DECLARE_SETTINGS_TRAITS(DistributedSettingsTraits, LIST_OF_DISTRIBUTED_SETTINGS) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index f80e498efa8..6f0072c4560 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1438,6 +1438,12 @@ ActionLock StorageDistributed::getActionLock(StorageActionBlockType type) void StorageDistributed::flushAndPrepareForShutdown() { + if (!getDistributedSettingsRef().flush_on_detach) + { + LOG_INFO(log, "Skip flushing data (due to flush_on_detach=0)"); + return; + } + try { flushClusterNodesAllData(getContext()); diff --git a/tests/queries/0_stateless/02860_distributed_flush_on_detach.reference b/tests/queries/0_stateless/02860_distributed_flush_on_detach.reference new file mode 100644 index 00000000000..8e5815188e7 --- /dev/null +++ b/tests/queries/0_stateless/02860_distributed_flush_on_detach.reference @@ -0,0 +1,27 @@ +-- { echoOn } + +create table data (key Int) engine=Memory(); +create table dist (key Int) engine=Distributed(default, currentDatabase(), data); +system stop distributed sends dist; +-- check that FLUSH DISTRIBUTED does flushing anyway +insert into dist values (1); +select * from data; +system flush distributed dist; +select * from data; +1 +truncate table data; +-- check that flush_on_detach=1 by default +insert into dist values (1); +detach table dist; +select * from data; +1 +attach table dist; +truncate table data; +-- check flush_on_detach=0 +drop table dist; +create table dist (key Int) engine=Distributed(default, currentDatabase(), data) settings flush_on_detach=0; +system stop distributed sends dist; +insert into dist values (1); +detach table dist; +select * from data; +attach table dist; diff --git a/tests/queries/0_stateless/02860_distributed_flush_on_detach.sql b/tests/queries/0_stateless/02860_distributed_flush_on_detach.sql new file mode 100644 index 00000000000..5ba45d72c08 --- /dev/null +++ b/tests/queries/0_stateless/02860_distributed_flush_on_detach.sql @@ -0,0 +1,33 @@ +set prefer_localhost_replica=0; + +drop table if exists data; +drop table if exists dist; + +-- { echoOn } + +create table data (key Int) engine=Memory(); +create table dist (key Int) engine=Distributed(default, currentDatabase(), data); +system stop distributed sends dist; + +-- check that FLUSH DISTRIBUTED does flushing anyway +insert into dist values (1); +select * from data; +system flush distributed dist; +select * from data; +truncate table data; + +-- check that flush_on_detach=1 by default +insert into dist values (1); +detach table dist; +select * from data; +attach table dist; +truncate table data; + +-- check flush_on_detach=0 +drop table dist; +create table dist (key Int) engine=Distributed(default, currentDatabase(), data) settings flush_on_detach=0; +system stop distributed sends dist; +insert into dist values (1); +detach table dist; +select * from data; +attach table dist; From fd3e6c93ee58c40d303055ca1bc508e322e8920a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 17 Aug 2023 09:14:47 +0200 Subject: [PATCH 640/777] Produce standard report for cases when AST failed --- tests/ci/ast_fuzzer_check.py | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index fecf207589e..a8c7f2c3a9c 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -21,7 +21,6 @@ from commit_status_helper import ( ) from docker_pull_helper import get_image_with_version from env_helper import ( - GITHUB_RUN_URL, REPORTS_PATH, TEMP_PATH, ) @@ -30,6 +29,7 @@ from pr_info import PRInfo from report import TestResult from s3_helper import S3Helper from stopwatch import Stopwatch +from upload_result_helper import upload_results IMAGE_NAME = "clickhouse/fuzzer" @@ -183,10 +183,6 @@ def main(): logging.info("Exception uploading file %s text %s", f, ex) paths[f] = "" - report_url = GITHUB_RUN_URL - if paths["report.html"]: - report_url = paths["report.html"] - # Try to get status message saved by the fuzzer try: with open( @@ -208,6 +204,19 @@ def main(): if "fail" in status: test_result.status = "FAIL" + if paths["report.html"]: + report_url = paths["report.html"] + else: + report_url = upload_results( + s3_helper, + pr_info.number, + pr_info.sha, + [test_result], + [], + check_name, + [url for url in paths.values() if url], + ) + ch_helper = ClickHouseHelper() prepared_events = prepare_tests_results_for_clickhouse( From 6b44088a8be3d85c59439b939907f0ca1a68b8c6 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 17 Aug 2023 07:51:44 +0000 Subject: [PATCH 641/777] Added waiting for PostgreSQL port open --- tests/integration/test_profile_max_sessions_for_user/test.py | 4 ++++ tests/integration/test_session_log/test.py | 5 ++++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_profile_max_sessions_for_user/test.py b/tests/integration/test_profile_max_sessions_for_user/test.py index 5eaef09bf6d..38f77b9d375 100755 --- a/tests/integration/test_profile_max_sessions_for_user/test.py +++ b/tests/integration/test_profile_max_sessions_for_user/test.py @@ -108,6 +108,10 @@ def threaded_run_test(sessions): def started_cluster(): try: cluster.start() + # Wait for the PostgreSQL handler to start. + # Cluster.start waits until port 9000 becomes accessible. + # Server opens the PostgreSQL compatibility port a bit later. + instance.wait_for_log_line("PostgreSQL compatibility protocol") yield cluster finally: cluster.shutdown() diff --git a/tests/integration/test_session_log/test.py b/tests/integration/test_session_log/test.py index bb7cafa4ee6..0c350e6c2c9 100644 --- a/tests/integration/test_session_log/test.py +++ b/tests/integration/test_session_log/test.py @@ -1,5 +1,4 @@ import os - import grpc import pymysql.connections import pytest @@ -130,6 +129,10 @@ def mysql_query(query, user_, pass_, raise_exception): def started_cluster(): try: cluster.start() + # Wait for the PostgreSQL handler to start. + # Cluster.start waits until port 9000 becomes accessible. + # Server opens the PostgreSQL compatibility port a bit later. + instance.wait_for_log_line("PostgreSQL compatibility protocol") yield cluster finally: cluster.shutdown() From adc73beb933c8c5965eaaf6520b322053b172a79 Mon Sep 17 00:00:00 2001 From: Alexey Gerasimchuck Date: Thu, 17 Aug 2023 08:24:17 +0000 Subject: [PATCH 642/777] Added waiting to other PostgreSQL tests --- tests/integration/test_postgresql_protocol/test.py | 5 ++++- tests/integration/test_storage_postgresql/test.py | 4 ++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index de01bba6862..9da9ec4d0b7 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -40,7 +40,10 @@ server_port = 5433 def started_cluster(): try: cluster.start() - + # Wait for the PostgreSQL handler to start. + # Cluster.start waits until port 9000 becomes accessible. + # Server opens the PostgreSQL compatibility port a bit later. + cluster.instances["node"].wait_for_log_line("PostgreSQL compatibility protocol") yield cluster except Exception as ex: logging.exception(ex) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index d4f8fab3a82..11729a5ab18 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -26,6 +26,10 @@ def started_cluster(): cluster.start() node1.query("CREATE DATABASE test") node2.query("CREATE DATABASE test") + # Wait for the PostgreSQL handler to start. + # cluster.start waits until port 9000 becomes accessible. + # Server opens the PostgreSQL compatibility port a bit later. + node1.wait_for_log_line("PostgreSQL compatibility protocol") yield cluster finally: From e609a9f3d11c43cbadf91f3d122be1ddc3d87c88 Mon Sep 17 00:00:00 2001 From: Filipp Ozinov Date: Thu, 17 Aug 2023 12:50:44 +0400 Subject: [PATCH 643/777] Test for system.data_skipping_indices with MaterializedMySQL --- .../materialized_with_ddl.py | 34 +++++++++++++++++++ 1 file changed, 34 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 36be2aa1672..ce5adac35bd 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -2637,3 +2637,37 @@ def create_table_as_select(clickhouse_node, mysql_node, service_name): clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + + +def table_with_indexes(clickhouse_node, mysql_node, service_name): + db = "table_with_indexes" + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"CREATE DATABASE {db}") + + mysql_node.query( + f"CREATE TABLE {db}.t1(id INT NOT NULL PRIMARY KEY," + f"data varchar(255) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin NOT NULL) ENGINE = InnoDB" + ) + + mysql_node.query(f"INSERT INTO {db}.t1 VALUES(1, 'some test string 1')") + mysql_node.query(f"INSERT INTO {db}.t1 VALUES(2, 'some test string 2')") + + clickhouse_node.query( + f""" + CREATE DATABASE {db} ENGINE = MaterializeMySQL('{service_name}:3306', '{db}', 'root', 'clickhouse') + TABLE OVERRIDE t1 (COLUMNS ( + INDEX data_idx data TYPE ngrambf_v1(5, 65536, 4, 0) GRANULARITY 1 + )) + """ + ) + + check_query( + clickhouse_node, + "SELECT data_uncompressed_bytes FROM system.data_skipping_indices WHERE " + "database = 'table_with_indexes' and table = 't1' and name = 'data_idx'", + '65536\n', + ) + + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") From f402d80bcea3ba421b603ad071b04c2e6921f1dd Mon Sep 17 00:00:00 2001 From: Filipp Ozinov Date: Thu, 17 Aug 2023 12:51:41 +0400 Subject: [PATCH 644/777] Test for system.data_skipping_indices with MaterializedMySQL --- tests/integration/test_materialized_mysql_database/test.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 5096e0a03b7..64c4fb45e9b 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -546,3 +546,9 @@ def test_create_table_as_select(started_cluster, started_mysql_8_0, clickhouse_n materialized_with_ddl.create_table_as_select( clickhouse_node, started_mysql_8_0, "mysql80" ) + + +def test_table_with_indexes(started_cluster, started_mysql_8_0, clickhouse_node): + materialized_with_ddl.table_with_indexes( + clickhouse_node, started_mysql_8_0, "mysql80" + ) From cfea6db997f4f1b979d56f04cd22b1e1cdc9a93f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Thu, 17 Aug 2023 16:57:41 +0800 Subject: [PATCH 645/777] Update native_orc.cpp --- src/Processors/examples/native_orc.cpp | 35 ++------------------------ 1 file changed, 2 insertions(+), 33 deletions(-) diff --git a/src/Processors/examples/native_orc.cpp b/src/Processors/examples/native_orc.cpp index 69a72dc7485..67e8bcf433b 100644 --- a/src/Processors/examples/native_orc.cpp +++ b/src/Processors/examples/native_orc.cpp @@ -8,8 +8,8 @@ using namespace DB; int main() { /// Read schema from orc file - // String path = "/path/to/orc/file"; - String path = "/data1/clickhouse_official/data/user_files/bigolive_audience_stats_orc.orc"; + String path = "/path/to/orc/file"; + // String path = "/data1/clickhouse_official/data/user_files/bigolive_audience_stats_orc.orc"; { ReadBufferFromFile in(path); ORCSchemaReader schema_reader(in, {}); @@ -32,36 +32,5 @@ int main() auto schema = schema_reader.readSchema(); std::cout << "schema:" << schema.toString() << std::endl; } - - { - String content - = "\x4f\x52\x43\x0a\x0b\x0a\x03\x00\x00\x00\x12\x04\x08\x01\x50\x00\x0a\x15\x0a\x05\x00\x00\x00\x00\x00\x12\x0c\x08\x01\x12\x06" - "\x08\x00\x10\x00\x18\x00\x50\x00\x0a\x12\x0a\x06\x00\x00\x00\x00\x00\x00\x12\x08\x08\x01\x42\x02\x08\x06\x50\x00\x0a\x12\x0a" - "\x06\x00\x00\x00\x00\x00\x00\x12\x08\x08\x01\x42\x02\x08\x04\x50\x00\x0a\x29\x0a\x04\x00\x00\x00\x00\x12\x21\x08\x01\x1a\x1b" - "\x09\x00\x00\x00\x00\x00\x00\xe0\x3f\x11\x00\x00\x00\x00\x00\x00\xe0\x3f\x19\x00\x00\x00\x00\x00\x00\xe0\x3f\x50\x00\x0a\x15" - "\x0a\x05\x00\x00\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\xff\x80\xff\x80\xff\x00\xff\x80\xff\x03" - "\x42\x41\x44\xff\x80\xff\x02\x41\x4d\xff\x80\x00\x00\x00\x3f\xff\x80\xff\x01\x0a\x06\x08\x06\x10\x00\x18\x0d\x0a\x06\x08\x06" - "\x10\x01\x18\x17\x0a\x06\x08\x06\x10\x02\x18\x14\x0a\x06\x08\x06\x10\x03\x18\x14\x0a\x06\x08\x06\x10\x04\x18\x2b\x0a\x06\x08" - "\x06\x10\x05\x18\x17\x0a\x06\x08\x00\x10\x00\x18\x02\x0a\x06\x08\x00\x10\x01\x18\x02\x0a\x06\x08\x01\x10\x01\x18\x02\x0a\x06" - "\x08\x00\x10\x02\x18\x02\x0a\x06\x08\x02\x10\x02\x18\x02\x0a\x06\x08\x01\x10\x02\x18\x03\x0a\x06\x08\x00\x10\x03\x18\x02\x0a" - "\x06\x08\x02\x10\x03\x18\x02\x0a\x06\x08\x01\x10\x03\x18\x02\x0a\x06\x08\x00\x10\x04\x18\x02\x0a\x06\x08\x01\x10\x04\x18\x04" - "\x0a\x06\x08\x00\x10\x05\x18\x02\x0a\x06\x08\x01\x10\x05\x18\x02\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08" - "\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x12\x04\x08\x00\x10\x00\x1a\x03\x47\x4d\x54\x0a\x59\x0a\x04\x08" - "\x01\x50\x00\x0a\x0c\x08\x01\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x0a\x08\x08\x01\x42\x02\x08\x06\x50\x00\x0a\x08\x08\x01" - "\x42\x02\x08\x04\x50\x00\x0a\x21\x08\x01\x1a\x1b\x09\x00\x00\x00\x00\x00\x00\xe0\x3f\x11\x00\x00\x00\x00\x00\x00\xe0\x3f\x19" - "\x00\x00\x00\x00\x00\x00\xe0\x3f\x50\x00\x0a\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\x08\x03\x10\xec\x02\x1a\x0c" - "\x08\x03\x10\x8e\x01\x18\x1d\x20\xc1\x01\x28\x01\x22\x2e\x08\x0c\x12\x05\x01\x02\x03\x04\x05\x1a\x02\x69\x64\x1a\x07\x62\x6c" - "\x6f\x63\x6b\x4e\x6f\x1a\x04\x76\x61\x6c\x31\x1a\x04\x76\x61\x6c\x32\x1a\x04\x76\x61\x6c\x33\x20\x00\x28\x00\x30\x00\x22\x08" - "\x08\x04\x20\x00\x28\x00\x30\x00\x22\x08\x08\x08\x20\x00\x28\x00\x30\x00\x22\x08\x08\x08\x20\x00\x28\x00\x30\x00\x22\x08\x08" - "\x05\x20\x00\x28\x00\x30\x00\x22\x08\x08\x01\x20\x00\x28\x00\x30\x00\x30\x01\x3a\x04\x08\x01\x50\x00\x3a\x0c\x08\x01\x12\x06" - "\x08\x00\x10\x00\x18\x00\x50\x00\x3a\x08\x08\x01\x42\x02\x08\x06\x50\x00\x3a\x08\x08\x01\x42\x02\x08\x04\x50\x00\x3a\x21\x08" - "\x01\x1a\x1b\x09\x00\x00\x00\x00\x00\x00\xe0\x3f\x11\x00\x00\x00\x00\x00\x00\xe0\x3f\x19\x00\x00\x00\x00\x00\x00\xe0\x3f\x50" - "\x00\x3a\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\x40\x90\x4e\x48\x01\x08\xd5\x01\x10\x00\x18\x80\x80\x04\x22\x02" - "\x00\x0b\x28\x5b\x30\x06\x82\xf4\x03\x03\x4f\x52\x43\x18"; - ReadBufferFromString in(content); - ORCSchemaReader schema_reader(in, {}); - auto schema = schema_reader.readSchema(); - std::cout << "schema:" << schema.toString() << std::endl; - } return 0; } From 5253a7f650fc726098a9515734fe9553cf408c70 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 15 Aug 2023 21:04:40 +0000 Subject: [PATCH 646/777] Respect cgroup memory limit when reloading configuration Cgroups allows to change the amount of memory available to a process while it runs. The previous logic calculated the amount of available memory only once at server startup. As a result, memory thresholds set via cgroups were not picked up when the settings changed. We now always incorporate the current limits during re-configuraton. Note 1: getMemoryAmount() opens/reads a file which is potentially expensive. Should be fine though since that happens only when the server configuration changes. Note 2: An better approach would be to treat cgroup limit changes as another trigger for ClickHouse server re-configuration (which currently only happens when the config files change). Shied away from that for now because of the case that when the cgroup limit is lowered, there is no guarantee that ClickHouse can shrink the memory amount accordingly in time (afaik, it does so only lazily by denying new allocations). As a result, the OOM killer would kill the server. The same will happen with this PR but at a lower implementation complexity. --- programs/server/Server.cpp | 21 +++++++++++---------- src/Core/ServerSettings.h | 18 +++++++++--------- 2 files changed, 20 insertions(+), 19 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b823cbbe367..a193b132f72 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -659,10 +659,10 @@ try global_context->addWarningMessage("Server was built with sanitizer. It will work slowly."); #endif - const size_t memory_amount = getMemoryAmount(); + const size_t physical_server_memory = getMemoryAmount(); LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.", - formatReadableSizeWithBinarySuffix(memory_amount), + formatReadableSizeWithBinarySuffix(physical_server_memory), getNumberOfPhysicalCPUCores(), // on ARM processors it can show only enabled at current moment cores std::thread::hardware_concurrency()); @@ -1137,9 +1137,10 @@ try server_settings_.loadSettingsFromConfig(*config); size_t max_server_memory_usage = server_settings_.max_server_memory_usage; - double max_server_memory_usage_to_ram_ratio = server_settings_.max_server_memory_usage_to_ram_ratio; - size_t default_max_server_memory_usage = static_cast(memory_amount * max_server_memory_usage_to_ram_ratio); + + size_t current_physical_server_memory = getMemoryAmount(); /// With cgroups, the amount of memory available to the server can be changed dynamically. + size_t default_max_server_memory_usage = static_cast(current_physical_server_memory * max_server_memory_usage_to_ram_ratio); if (max_server_memory_usage == 0) { @@ -1147,7 +1148,7 @@ try LOG_INFO(log, "Setting max_server_memory_usage was set to {}" " ({} available * {:.2f} max_server_memory_usage_to_ram_ratio)", formatReadableSizeWithBinarySuffix(max_server_memory_usage), - formatReadableSizeWithBinarySuffix(memory_amount), + formatReadableSizeWithBinarySuffix(current_physical_server_memory), max_server_memory_usage_to_ram_ratio); } else if (max_server_memory_usage > default_max_server_memory_usage) @@ -1158,7 +1159,7 @@ try " calculated as {} available" " * {:.2f} max_server_memory_usage_to_ram_ratio", formatReadableSizeWithBinarySuffix(max_server_memory_usage), - formatReadableSizeWithBinarySuffix(memory_amount), + formatReadableSizeWithBinarySuffix(current_physical_server_memory), max_server_memory_usage_to_ram_ratio); } @@ -1168,14 +1169,14 @@ try size_t merges_mutations_memory_usage_soft_limit = server_settings_.merges_mutations_memory_usage_soft_limit; - size_t default_merges_mutations_server_memory_usage = static_cast(memory_amount * server_settings_.merges_mutations_memory_usage_to_ram_ratio); + size_t default_merges_mutations_server_memory_usage = static_cast(current_physical_server_memory * server_settings_.merges_mutations_memory_usage_to_ram_ratio); if (merges_mutations_memory_usage_soft_limit == 0) { merges_mutations_memory_usage_soft_limit = default_merges_mutations_server_memory_usage; LOG_INFO(log, "Setting merges_mutations_memory_usage_soft_limit was set to {}" " ({} available * {:.2f} merges_mutations_memory_usage_to_ram_ratio)", formatReadableSizeWithBinarySuffix(merges_mutations_memory_usage_soft_limit), - formatReadableSizeWithBinarySuffix(memory_amount), + formatReadableSizeWithBinarySuffix(current_physical_server_memory), server_settings_.merges_mutations_memory_usage_to_ram_ratio); } else if (merges_mutations_memory_usage_soft_limit > default_merges_mutations_server_memory_usage) @@ -1184,7 +1185,7 @@ try LOG_WARNING(log, "Setting merges_mutations_memory_usage_soft_limit was set to {}" " ({} available * {:.2f} merges_mutations_memory_usage_to_ram_ratio)", formatReadableSizeWithBinarySuffix(merges_mutations_memory_usage_soft_limit), - formatReadableSizeWithBinarySuffix(memory_amount), + formatReadableSizeWithBinarySuffix(current_physical_server_memory), server_settings_.merges_mutations_memory_usage_to_ram_ratio); } @@ -1486,7 +1487,7 @@ try /// Set up caches. - const size_t max_cache_size = static_cast(memory_amount * server_settings.cache_size_to_ram_max_ratio); + const size_t max_cache_size = static_cast(physical_server_memory * server_settings.cache_size_to_ram_max_ratio); String uncompressed_cache_policy = server_settings.uncompressed_cache_policy; size_t uncompressed_cache_size = server_settings.uncompressed_cache_size; diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index f759cd150a8..3740929b491 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -44,19 +44,19 @@ namespace DB M(String, tmp_policy, "", "Policy for storage with temporary data.", 0) \ M(UInt64, max_temporary_data_on_disk_size, 0, "The maximum amount of storage that could be used for external aggregation, joins or sorting., ", 0) \ M(String, temporary_data_in_cache, "", "Cache disk name for temporary data.", 0) \ - M(UInt64, max_server_memory_usage, 0, "Limit on total memory usage. Zero means Unlimited.", 0) \ - M(Double, max_server_memory_usage_to_ram_ratio, 0.9, "Same as max_server_memory_usage but in to ram ratio. Allows to lower max memory on low-memory systems.", 0) \ - M(UInt64, merges_mutations_memory_usage_soft_limit, 0, "Limit on total memory usage for merges and mutations. Zero means Unlimited.", 0) \ - M(Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to ram ratio. Allows to lower memory limit on low-memory systems.", 0) \ + M(UInt64, max_server_memory_usage, 0, "Maximum total memory usage of the server in bytes. Zero means unlimited.", 0) \ + M(Double, max_server_memory_usage_to_ram_ratio, 0.9, "Same as max_server_memory_usage but in to RAM ratio. Allows to lower max memory on low-memory systems.", 0) \ + M(UInt64, merges_mutations_memory_usage_soft_limit, 0, "Maximum total memory usage for merges and mutations in bytes. Zero means unlimited.", 0) \ + M(Double, merges_mutations_memory_usage_to_ram_ratio, 0.5, "Same as merges_mutations_memory_usage_soft_limit but in to RAM ratio. Allows to lower memory limit on low-memory systems.", 0) \ M(Bool, allow_use_jemalloc_memory, true, "Allows to use jemalloc memory.", 0) \ M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \ M(Bool, async_insert_queue_flush_on_shutdown, true, "If true queue of asynchronous inserts is flushed on graceful shutdown", 0) \ \ - M(UInt64, max_concurrent_queries, 0, "Limit on total number of concurrently executed queries. Zero means Unlimited.", 0) \ - M(UInt64, max_concurrent_insert_queries, 0, "Limit on total number of concurrently insert queries. Zero means Unlimited.", 0) \ - M(UInt64, max_concurrent_select_queries, 0, "Limit on total number of concurrently select queries. Zero means Unlimited.", 0) \ + M(UInt64, max_concurrent_queries, 0, "Maximum number of concurrently executed queries. Zero means unlimited.", 0) \ + M(UInt64, max_concurrent_insert_queries, 0, "Maximum number of concurrently INSERT queries. Zero means unlimited.", 0) \ + M(UInt64, max_concurrent_select_queries, 0, "Maximum number of concurrently SELECT queries. Zero means unlimited.", 0) \ \ - M(Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro ram max ratio. Allows to lower cache size on low-memory systems.", 0) \ + M(Double, cache_size_to_ram_max_ratio, 0.5, "Set cache size ro RAM max ratio. Allows to lower cache size on low-memory systems.", 0) \ M(String, uncompressed_cache_policy, DEFAULT_UNCOMPRESSED_CACHE_POLICY, "Uncompressed cache policy name.", 0) \ M(UInt64, uncompressed_cache_size, DEFAULT_UNCOMPRESSED_CACHE_MAX_SIZE, "Size of cache for uncompressed blocks. Zero means disabled.", 0) \ M(UInt64, mark_cache_size, DEFAULT_MARK_CACHE_MAX_SIZE, "Size of cache for marks (index of MergeTree family of tables).", 0) \ @@ -71,7 +71,7 @@ namespace DB \ M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \ M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \ - M(UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means Unlimited.", 0) \ + M(UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0) \ M(UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0) \ \ M(UInt64, background_pool_size, 16, "The maximum number of threads what will be used for merging or mutating data parts for *MergeTree-engine tables in a background.", 0) \ From f420006e093041574084970735043dd397dcf101 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 17 Aug 2023 09:39:19 +0000 Subject: [PATCH 647/777] Make the code consistent with Annoy index and fix build --- .../ApproximateNearestNeighborIndexesCommon.h | 3 +++ .../MergeTree/MergeTreeIndexAnnoy.cpp | 8 ++----- src/Storages/MergeTree/MergeTreeIndexHnsw.cpp | 22 +++++++++---------- 3 files changed, 16 insertions(+), 17 deletions(-) diff --git a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h index 310890eba1e..5092fbdd864 100644 --- a/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h +++ b/src/Storages/MergeTree/ApproximateNearestNeighborIndexesCommon.h @@ -9,6 +9,9 @@ namespace DB { +static constexpr auto DISTANCE_FUNCTION_L2 = "L2Distance"; +static constexpr auto DISTANCE_FUNCTION_COSINE = "cosineDistance"; + /// Approximate Nearest Neighbour queries have a similar structure: /// - reference vector from which all distances are calculated /// - metric name (e.g L2Distance, LpDistance, etc.) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 1c92645dbfa..13577229a75 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -25,12 +25,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static constexpr auto DISTANCE_FUNCTION_L2 = "L2Distance"; -static constexpr auto DISTANCE_FUNCTION_COSINE = "cosineDistance"; - -static constexpr auto DEFAULT_TREES = 100uz; -static constexpr auto DEFAULT_DISTANCE_FUNCTION = DISTANCE_FUNCTION_L2; - template AnnoyIndexWithSerialization::AnnoyIndexWithSerialization(size_t dimensions) : Base::AnnoyIndex(dimensions) @@ -318,10 +312,12 @@ MergeTreeIndexConditionPtr MergeTreeIndexAnnoy::createIndexCondition(const Selec MergeTreeIndexPtr annoyIndexCreator(const IndexDescription & index) { + static constexpr auto DEFAULT_DISTANCE_FUNCTION = DISTANCE_FUNCTION_L2; String distance_function = DEFAULT_DISTANCE_FUNCTION; if (!index.arguments.empty()) distance_function = index.arguments[0].get(); + static constexpr auto DEFAULT_TREES = 100uz; UInt64 trees = DEFAULT_TREES; if (index.arguments.size() > 1) trees = index.arguments[1].get(); diff --git a/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp b/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp index 369779e97fd..b49a65c1895 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp @@ -159,7 +159,7 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t throw Exception(ErrorCodes::INCORRECT_DATA, rc.error.release()); for (size_t current_row = 1; current_row < num_rows; ++current_row) if (auto rc = index->add(index->size(), &array[offsets[current_row - 1]]); !rc) - throw Exception(ErrorCodes::INCORRECT_DATA, add_result.error.release()); + throw Exception(ErrorCodes::INCORRECT_DATA, rc.error.release()); } else if (const auto & column_tuple = typeid_cast(column_cut.get())) @@ -183,7 +183,7 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t for (const auto & item : data) if (auto rc = index->add(index->size(), item.data()); !rc) - throw Exception(ErrorCodes::INCORRECT_DATA, add_result.error.release()); + throw Exception(ErrorCodes::INCORRECT_DATA, rc.error.release()); } else throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected Array or Tuple column"); @@ -210,9 +210,9 @@ bool MergeTreeIndexConditionUSearch::alwaysUnknownOrTrue() const std::vector MergeTreeIndexConditionUSearch::getUsefulRanges(MergeTreeIndexGranulePtr idx_granule) const { - if (distance_function == "L2Distance") + if (distance_function == DISTANCE_FUNCTION_L2) return getUsefulRangesImpl(idx_granule); - else if (distance_function == "cosineDistance") + else if (distance_function == DISTANCE_FUNCTION_COSINE) return getUsefulRangesImpl(idx_granule); std::unreachable(); } @@ -272,18 +272,18 @@ MergeTreeIndexUSearch::MergeTreeIndexUSearch(const IndexDescription & index_, co MergeTreeIndexGranulePtr MergeTreeIndexUSearch::createIndexGranule() const { - if (distance_function == "L2Distance") + if (distance_function == DISTANCE_FUNCTION_L2) return std::make_shared>(index.name, index.sample_block); - else if (distance_function == "cosineDistance") + else if (distance_function == DISTANCE_FUNCTION_COSINE) return std::make_shared>(index.name, index.sample_block); std::unreachable(); } MergeTreeIndexAggregatorPtr MergeTreeIndexUSearch::createIndexAggregator() const { - if (distance_function == "L2Distance") + if (distance_function == DISTANCE_FUNCTION_L2) return std::make_shared>(index.name, index.sample_block); - else if (distance_function == "cosineDistance") + else if (distance_function == DISTANCE_FUNCTION_COSINE) return std::make_shared>(index.name, index.sample_block); std::unreachable(); } @@ -295,7 +295,7 @@ MergeTreeIndexConditionPtr MergeTreeIndexUSearch::createIndexCondition(const Sel MergeTreeIndexPtr usearchIndexCreator(const IndexDescription & index) { - static constexpr auto default_distance_function = "L2Distance"; + static constexpr auto default_distance_function = DISTANCE_FUNCTION_L2; String distance_function = default_distance_function; if (!index.arguments.empty()) distance_function = index.arguments[0].get(); @@ -323,8 +323,8 @@ void usearchIndexValidator(const IndexDescription & index, bool /* attach */) if (!index.arguments.empty()) { String distance_name = index.arguments[0].get(); - if (distance_name != "L2Distance" && distance_name != "cosineDistance") - throw Exception(ErrorCodes::INCORRECT_DATA, "USearch index only supports distance functions 'L2Distance' and 'cosineDistance'"); + if (distance_name != DISTANCE_FUNCTION_L2 && distance_name != DISTANCE_FUNCTION_COSINE) + throw Exception(ErrorCodes::INCORRECT_DATA, "USearch index only supports distance functions '{}' and '{}'", DISTANCE_FUNCTION_L2, DISTANCE_FUNCTION_COSINE); } /// Check data type of indexed column: From 5003406ef75a38f0e8240090f863bdd4e21e1738 Mon Sep 17 00:00:00 2001 From: Filipp Ozinov Date: Thu, 17 Aug 2023 13:52:21 +0400 Subject: [PATCH 648/777] Style --- .../test_materialized_mysql_database/materialized_with_ddl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index ce5adac35bd..7fe439a8be4 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -2666,7 +2666,7 @@ def table_with_indexes(clickhouse_node, mysql_node, service_name): clickhouse_node, "SELECT data_uncompressed_bytes FROM system.data_skipping_indices WHERE " "database = 'table_with_indexes' and table = 't1' and name = 'data_idx'", - '65536\n', + "65536\n", ) mysql_node.query(f"DROP DATABASE IF EXISTS {db}") From 42644bfbf579bc9a2c79bd92df371340f3e02f5f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 17 Aug 2023 09:44:10 +0000 Subject: [PATCH 649/777] Rename file to MergeTreeIndexUSearch.h/cpp --- ...ndexHnsw.cpp => MergeTreeIndexUSearch.cpp} | 41 ++++++++++++------- ...reeIndexHnsw.h => MergeTreeIndexUSearch.h} | 7 ++-- 2 files changed, 30 insertions(+), 18 deletions(-) rename src/Storages/MergeTree/{MergeTreeIndexHnsw.cpp => MergeTreeIndexUSearch.cpp} (93%) rename src/Storages/MergeTree/{MergeTreeIndexHnsw.h => MergeTreeIndexUSearch.h} (90%) diff --git a/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp similarity index 93% rename from src/Storages/MergeTree/MergeTreeIndexHnsw.cpp rename to src/Storages/MergeTree/MergeTreeIndexUSearch.cpp index b49a65c1895..eddbc61b11b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHnsw.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.cpp @@ -1,6 +1,6 @@ #ifdef ENABLE_USEARCH -#include +#include #include #include @@ -25,7 +25,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - template USearchIndexWithSerialization::USearchIndexWithSerialization(size_t dimensions) : Base(Base::make(unum::usearch::metric_punned_t(dimensions, Metric))) @@ -64,17 +63,24 @@ size_t USearchIndexWithSerialization::getDimensions() const return Base::dimensions(); } - template -MergeTreeIndexGranuleUSearch::MergeTreeIndexGranuleUSearch(const String & index_name_, const Block & index_sample_block_) - : index_name(index_name_), index_sample_block(index_sample_block_), index(nullptr) +MergeTreeIndexGranuleUSearch::MergeTreeIndexGranuleUSearch( + const String & index_name_, + const Block & index_sample_block_) + : index_name(index_name_) + , index_sample_block(index_sample_block_) + , index(nullptr) { } template MergeTreeIndexGranuleUSearch::MergeTreeIndexGranuleUSearch( - const String & index_name_, const Block & index_sample_block_, USearchIndexWithSerializationPtr index_) - : index_name(index_name_), index_sample_block(index_sample_block_), index(std::move(index_)) + const String & index_name_, + const Block & index_sample_block_, + USearchIndexWithSerializationPtr index_) + : index_name(index_name_) + , index_sample_block(index_sample_block_) + , index(std::move(index_)) { } @@ -97,8 +103,11 @@ void MergeTreeIndexGranuleUSearch::deserializeBinary(ReadBuffer & istr, } template -MergeTreeIndexAggregatorUSearch::MergeTreeIndexAggregatorUSearch(const String & index_name_, const Block & index_sample_block_) - : index_name(index_name_), index_sample_block(index_sample_block_) +MergeTreeIndexAggregatorUSearch::MergeTreeIndexAggregatorUSearch( + const String & index_name_, + const Block & index_sample_block_) + : index_name(index_name_) + , index_sample_block(index_sample_block_) { } @@ -191,10 +200,13 @@ void MergeTreeIndexAggregatorUSearch::update(const Block & block, size_t *pos += rows_read; } - MergeTreeIndexConditionUSearch::MergeTreeIndexConditionUSearch( - const IndexDescription & /*index_description*/, const SelectQueryInfo & query, const String & distance_function_, ContextPtr context) - : ann_condition(query, context), distance_function(distance_function_) + const IndexDescription & /*index_description*/, + const SelectQueryInfo & query, + const String & distance_function_, + ContextPtr context) + : ann_condition(query, context) + , distance_function(distance_function_) { } @@ -265,11 +277,11 @@ std::vector MergeTreeIndexConditionUSearch::getUsefulRangesImpl(MergeTre } MergeTreeIndexUSearch::MergeTreeIndexUSearch(const IndexDescription & index_, const String & distance_function_) - : IMergeTreeIndex(index_), distance_function(distance_function_) + : IMergeTreeIndex(index_) + , distance_function(distance_function_) { } - MergeTreeIndexGranulePtr MergeTreeIndexUSearch::createIndexGranule() const { if (distance_function == DISTANCE_FUNCTION_L2) @@ -356,6 +368,7 @@ void usearchIndexValidator(const IndexDescription & index, bool /* attach */) else throw_unsupported_underlying_column_exception(); } + } #endif diff --git a/src/Storages/MergeTree/MergeTreeIndexHnsw.h b/src/Storages/MergeTree/MergeTreeIndexUSearch.h similarity index 90% rename from src/Storages/MergeTree/MergeTreeIndexHnsw.h rename to src/Storages/MergeTree/MergeTreeIndexUSearch.h index 89f5e3bb8cb..2b8ddf633d4 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHnsw.h +++ b/src/Storages/MergeTree/MergeTreeIndexUSearch.h @@ -28,8 +28,7 @@ template struct MergeTreeIndexGranuleUSearch final : public IMergeTreeIndexGranule { MergeTreeIndexGranuleUSearch(const String & index_name_, const Block & index_sample_block_); - MergeTreeIndexGranuleUSearch( - const String & index_name_, const Block & index_sample_block_, USearchIndexWithSerializationPtr index_); + MergeTreeIndexGranuleUSearch(const String & index_name_, const Block & index_sample_block_, USearchIndexWithSerializationPtr index_); ~MergeTreeIndexGranuleUSearch() override = default; @@ -62,8 +61,7 @@ struct MergeTreeIndexAggregatorUSearch final : IMergeTreeIndexAggregator class MergeTreeIndexConditionUSearch final : public IMergeTreeIndexConditionApproximateNearestNeighbor { public: - MergeTreeIndexConditionUSearch( - const IndexDescription & index_description, const SelectQueryInfo & query, const String & distance_function, ContextPtr context); + MergeTreeIndexConditionUSearch(const IndexDescription & index_description, const SelectQueryInfo & query, const String & distance_function, ContextPtr context); ~MergeTreeIndexConditionUSearch() override = default; @@ -101,3 +99,4 @@ private: } #endif + From 9e69120bee9e0b256b1e4124eec2d31c3bd232fd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 17 Aug 2023 12:03:38 +0200 Subject: [PATCH 650/777] fix --- tests/integration/test_grpc_protocol/test.py | 2 +- .../00002_log_and_exception_messages_formatting.sql | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index c149e1a4096..42d6d2bfda3 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -352,7 +352,7 @@ def test_authentication(): def test_logs(): - query = "SELECT has(groupArray(number), 42) FROM numbers(100000)" + query = "SELECT has(groupArray(number), 42) FROM numbers(100000) SETTINGS max_block_size=10000" logs = query_and_get_logs( query, settings={"send_logs_level": "debug"}, diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index c4a94174125..e2de14a4a8b 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -14,11 +14,10 @@ select 'runtime messages', greatest(coalesce(sum(length(message_format_string) = -- Check the same for exceptions. The value was 0.03 select 'runtime exceptions', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.05) from logs where (message like '%DB::Exception%' or message like '%Coordination::Exception%') - and message not like '%Received from CLICKHOUSE_CI_LOGS_HOST:9440%'; + and message not like '% Received from %clickhouse-staging.com:9440%'; select 'unknown runtime exceptions', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.01) from logs where (message like '%DB::Exception%' or message like '%Coordination::Exception%') - and message not like '%Received from CLICKHOUSE_CI_LOGS_HOST:9440%' and message not like '% Received from %' and message not like '%(SYNTAX_ERROR)%'; -- FIXME some of the following messages are not informative and it has to be fixed From ac7a27b7ffefb7e556535d8ff213db383e31bb53 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 17 Aug 2023 10:03:58 +0000 Subject: [PATCH 651/777] Update docs --- .../mergetree-family/annindexes.md | 23 +++++++++++++++---- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index 76289ee7f60..dbcc76254ca 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -149,8 +149,8 @@ was specified for ANN indexes, the default value is 100 million. Annoy indexes are currently experimental, to use them you first need to `SET allow_experimental_annoy_index = 1`. They are also currently disabled on ARM due to memory safety problems with the algorithm. -This type of ANN index implements [the Annoy algorithm](https://github.com/spotify/annoy) which is based on a recursive division of the -space in random linear surfaces (lines in 2D, planes in 3D etc.). +This type of ANN index is based on the [Annoy library](https://github.com/spotify/annoy) which recursively divides the space into random +linear surfaces (lines in 2D, planes in 3D etc.).
+
Syntax to create an USearch index over an [Array](../../../sql-reference/data-types/array.md) column: @@ -259,4 +272,4 @@ USearch currently supports two distance functions: ([Wikipedia](https://en.wikipedia.org/wiki/Cosine_similarity)). For normalized data, `L2Distance` is usually a better choice, otherwise `cosineDistance` is recommended to compensate for scale. If no -distance function was specified during index creation, `L2Distance` is used as default. \ No newline at end of file +distance function was specified during index creation, `L2Distance` is used as default. From 76702a03bb75e9d94cb4313d0a67da97a3497d27 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 17 Aug 2023 13:08:27 +0300 Subject: [PATCH 652/777] Update materialized_with_ddl.py (#53494) --- .../materialized_with_ddl.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 36be2aa1672..e64884707c1 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -1249,7 +1249,7 @@ def err_sync_user_privs_with_materialized_mysql_database( ) assert "priv_err_db" in clickhouse_node.query("SHOW DATABASES") assert "test_table_1" not in clickhouse_node.query("SHOW TABLES FROM priv_err_db") - clickhouse_node.query_with_retry("DETACH DATABASE priv_err_db") + clickhouse_node.query_with_retry("DETACH DATABASE priv_err_db SYNC") mysql_node.query("REVOKE SELECT ON priv_err_db.* FROM 'test'@'%'") time.sleep(3) @@ -1442,7 +1442,7 @@ def mysql_kill_sync_thread_restore_test(clickhouse_node, mysql_node, service_nam time.sleep(sleep_time) clickhouse_node.query("SELECT * FROM test_database.test_table") - clickhouse_node.query_with_retry("DETACH DATABASE test_database") + clickhouse_node.query_with_retry("DETACH DATABASE test_database SYNC") clickhouse_node.query("ATTACH DATABASE test_database") check_query( clickhouse_node, @@ -1506,7 +1506,7 @@ def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name): mysql_node.alloc_connection() - clickhouse_node.query_with_retry("DETACH DATABASE kill_mysql_while_insert") + clickhouse_node.query_with_retry("DETACH DATABASE kill_mysql_while_insert SYNC") clickhouse_node.query("ATTACH DATABASE kill_mysql_while_insert") result = mysql_node.query_and_get_data( @@ -2593,7 +2593,7 @@ def named_collections(clickhouse_node, mysql_node, service_name): "1\ta\t1\n2\tb\t2\n", ) clickhouse_node.query(f"ALTER NAMED COLLECTION {db} SET port=9999") - clickhouse_node.query(f"DETACH DATABASE {db}") + clickhouse_node.query_with_retry(f"DETACH DATABASE {db} SYNC") mysql_node.query(f"INSERT INTO {db}.t1 VALUES (3, 'c', 3)") assert "ConnectionFailed:" in clickhouse_node.query_and_get_error( f"ATTACH DATABASE {db}" From df4ca322102c00375685ad1df178a505c3662a37 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 17 Aug 2023 11:31:51 +0000 Subject: [PATCH 653/777] Fix: select correct number of replicas for shard on coordinator + more tests --- .../ClusterProxy/executeQuery.cpp | 23 +++++++++- .../configs/remote_servers.xml | 14 ++++-- .../test.py | 43 +++++++++++++++---- 3 files changed, 67 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 0c2ad35515f..4e07c78e7cb 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -284,19 +284,38 @@ void executeQueryWithParallelReplicas( shard_num = column->getUInt(0); } + size_t all_replicas_count = 0; ClusterPtr new_cluster; /// if got valid shard_num from query initiator, then parallel replicas scope is the specified shard /// shards are numbered in order of appearance in the cluster config if (shard_num > 0) { - LOG_DEBUG(&Poco::Logger::get("executeQueryWithParallelReplicas"), "Parallel replicas query in shard scope: shard_num={}", shard_num); + const auto shard_count = not_optimized_cluster->getShardCount(); + if (shard_num > shard_count) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Shard number is greater than shard count: shard_num={} shard_count={} cluster={}", + shard_num, + shard_count, + not_optimized_cluster->getName()); + + chassert(shard_count == not_optimized_cluster->getShardsAddresses().size()); + + LOG_DEBUG( + &Poco::Logger::get("executeQueryWithParallelReplicas"), "Parallel replicas query in shard scope: shard_num={}", shard_num); + + const auto shard_replicas_num = not_optimized_cluster->getShardsAddresses()[shard_num - 1].size(); + all_replicas_count = std::min(static_cast(settings.max_parallel_replicas), shard_replicas_num); + /// shard_num is 1-based, but getClusterWithSingleShard expects 0-based index new_cluster = not_optimized_cluster->getClusterWithSingleShard(shard_num - 1); } else + { new_cluster = not_optimized_cluster->getClusterWithReplicasAsShards(settings); + all_replicas_count = std::min(static_cast(settings.max_parallel_replicas), new_cluster->getShardCount()); + } - auto all_replicas_count = std::min(static_cast(settings.max_parallel_replicas), not_optimized_cluster->getShardCount()); auto coordinator = std::make_shared(all_replicas_count); /// This is a little bit weird, but we construct an "empty" coordinator without diff --git a/tests/integration/test_parallel_replicas_over_distributed/configs/remote_servers.xml b/tests/integration/test_parallel_replicas_over_distributed/configs/remote_servers.xml index d5b027072ef..6d7a45365f7 100644 --- a/tests/integration/test_parallel_replicas_over_distributed/configs/remote_servers.xml +++ b/tests/integration/test_parallel_replicas_over_distributed/configs/remote_servers.xml @@ -11,17 +11,25 @@ n2 9000 -
- - true n3 9000 + + + true n4 9000 + + n5 + 9000 + + + n6 + 9000 + diff --git a/tests/integration/test_parallel_replicas_over_distributed/test.py b/tests/integration/test_parallel_replicas_over_distributed/test.py index a0f9d2b71ac..1c8a3787356 100644 --- a/tests/integration/test_parallel_replicas_over_distributed/test.py +++ b/tests/integration/test_parallel_replicas_over_distributed/test.py @@ -7,7 +7,7 @@ nodes = [ cluster.add_instance( f"n{i}", main_configs=["configs/remote_servers.xml"], with_zookeeper=True ) - for i in (1, 2, 3, 4) + for i in (1, 2, 3, 4, 5, 6) ] @@ -46,13 +46,19 @@ def create_tables(cluster, table_name): nodes[1].query( f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') ORDER BY (key)" ) - # shard 2 nodes[2].query( + f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" + ) + # shard 2 + nodes[3].query( f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard2/{table_name}', 'r1') ORDER BY (key)" ) - nodes[3].query( + nodes[4].query( f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard2/{table_name}', 'r2') ORDER BY (key)" ) + nodes[5].query( + f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard2/{table_name}', 'r3') ORDER BY (key)" + ) else: raise Exception(f"Unexpected cluster: {cluster}") @@ -94,10 +100,31 @@ def create_tables(cluster, table_name): @pytest.mark.parametrize( - "cluster", - ["test_single_shard_multiple_replicas", "test_multiple_shards_multiple_replicas"], + "cluster,max_parallel_replicas,prefer_localhost_replica", + [ + # prefer_localhost_replica=0 + pytest.param("test_single_shard_multiple_replicas", 2, 0), + pytest.param("test_single_shard_multiple_replicas", 3, 0), + pytest.param("test_single_shard_multiple_replicas", 4, 0), + pytest.param("test_single_shard_multiple_replicas", 10, 0), + # prefer_localhost_replica=1 + pytest.param("test_single_shard_multiple_replicas", 2, 1), + pytest.param("test_single_shard_multiple_replicas", 3, 1), + pytest.param("test_single_shard_multiple_replicas", 4, 1), + pytest.param("test_single_shard_multiple_replicas", 10, 1), + # prefer_localhost_replica=0 + pytest.param("test_multiple_shards_multiple_replicas", 2, 0), + pytest.param("test_multiple_shards_multiple_replicas", 3, 0), + pytest.param("test_multiple_shards_multiple_replicas", 4, 0), + pytest.param("test_multiple_shards_multiple_replicas", 10, 0), + # prefer_localhost_replica=1 + pytest.param("test_multiple_shards_multiple_replicas", 2, 1), + pytest.param("test_multiple_shards_multiple_replicas", 3, 1), + pytest.param("test_multiple_shards_multiple_replicas", 4, 1), + pytest.param("test_multiple_shards_multiple_replicas", 10, 1), + ] ) -def test_parallel_replicas_over_distributed(start_cluster, cluster): +def test_parallel_replicas_over_distributed(start_cluster, cluster, max_parallel_replicas, prefer_localhost_replica): table_name = "test_table" create_tables(cluster, table_name) @@ -116,8 +143,8 @@ def test_parallel_replicas_over_distributed(start_cluster, cluster): f"SELECT count(), min(key), max(key), sum(key) FROM {table_name}_d", settings={ "allow_experimental_parallel_reading_from_replicas": 2, - "prefer_localhost_replica": 0, - "max_parallel_replicas": 4, + "prefer_localhost_replica": prefer_localhost_replica, + "max_parallel_replicas": max_parallel_replicas, "use_hedged_requests": 0, }, ) From c36f828f9fd02b360c5f6ff8b41ba0d3a3310bf6 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 17 Aug 2023 11:42:56 +0000 Subject: [PATCH 654/777] Automatic style fix --- .../test_parallel_replicas_over_distributed/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_parallel_replicas_over_distributed/test.py b/tests/integration/test_parallel_replicas_over_distributed/test.py index 1c8a3787356..6525901d366 100644 --- a/tests/integration/test_parallel_replicas_over_distributed/test.py +++ b/tests/integration/test_parallel_replicas_over_distributed/test.py @@ -122,9 +122,11 @@ def create_tables(cluster, table_name): pytest.param("test_multiple_shards_multiple_replicas", 3, 1), pytest.param("test_multiple_shards_multiple_replicas", 4, 1), pytest.param("test_multiple_shards_multiple_replicas", 10, 1), - ] + ], ) -def test_parallel_replicas_over_distributed(start_cluster, cluster, max_parallel_replicas, prefer_localhost_replica): +def test_parallel_replicas_over_distributed( + start_cluster, cluster, max_parallel_replicas, prefer_localhost_replica +): table_name = "test_table" create_tables(cluster, table_name) From fcb66379d1a59a8258ac6a555c8903b29bc5de38 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 16 Aug 2023 17:46:39 +0200 Subject: [PATCH 655/777] Remove outdated code in ReplicatedMergeTreeQueue::initialize() Initially added in #28817 But got cleaned up in #41981 Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 21d5597e614..e44f078326a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -77,15 +77,6 @@ void ReplicatedMergeTreeQueue::initialize(zkutil::ZooKeeperPtr zookeeper) virtual_parts.add(part_name, nullptr); } - /// Drop parts can negatively affect virtual parts. So when we load parts - /// from zookeeper we can break invariant with virtual parts. To fix this we - /// have it here. - for (const LogEntryPtr & entry : queue) - { - if (entry->isDropPart(format_version)) - virtual_parts.removePartAndCoveredParts(*entry->getDropRange(format_version)); - } - LOG_TRACE(log, "Queue initialized"); } From 4af3790f34d191cb54a1c003dd3c3266d4fb9502 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 17 Aug 2023 12:11:28 +0000 Subject: [PATCH 656/777] Fix style --- src/Interpreters/ClusterProxy/executeQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 4e07c78e7cb..9e91febddb3 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -28,6 +28,7 @@ namespace DB namespace ErrorCodes { extern const int TOO_LARGE_DISTRIBUTED_DEPTH; + extern const int LOGICAL_ERROR; } namespace ClusterProxy From bca91548ade7e64d13cfb392b87da2bc1e7f752c Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 17 Aug 2023 12:28:01 +0000 Subject: [PATCH 657/777] Add setting input_format_parquet_local_file_min_bytes_for_seek --- docs/en/interfaces/formats.md | 1 + docs/en/operations/settings/settings-formats.md | 6 ++++++ src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp | 2 +- tests/queries/0_stateless/02725_parquet_preserve_order.sh | 6 +++--- 7 files changed, 14 insertions(+), 4 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 0d1308afc4d..327bde120a6 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -2134,6 +2134,7 @@ To exchange data with Hadoop, you can use [HDFS table engine](/docs/en/engines/t - [input_format_parquet_case_insensitive_column_matching](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_case_insensitive_column_matching) - ignore case when matching Parquet columns with ClickHouse columns. Default value - `false`. - [input_format_parquet_allow_missing_columns](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_allow_missing_columns) - allow missing columns while reading Parquet data. Default value - `false`. - [input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference) - allow skipping columns with unsupported types while schema inference for Parquet format. Default value - `false`. +- [input_format_parquet_local_file_min_bytes_for_seek](/docs/en/operations/settings/settings-formats.md/#input_format_parquet_local_file_min_bytes_for_seek) - min bytes required for local read (file) to do seek, instead of read with ignore in Parquet input format. Default value - `8192`. - [output_format_parquet_fixed_string_as_fixed_byte_array](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_fixed_string_as_fixed_byte_array) - use Parquet FIXED_LENGTH_BYTE_ARRAY type instead of Binary/String for FixedString columns. Default value - `true`. - [output_format_parquet_version](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_version) - The version of Parquet format used in output format. Default value - `2.latest`. - [output_format_parquet_compression_method](/docs/en/operations/settings/settings-formats.md/#output_format_parquet_compression_method) - compression method used in output Parquet format. Default value - `snappy`. diff --git a/docs/en/operations/settings/settings-formats.md b/docs/en/operations/settings/settings-formats.md index beb1d372e08..86aabae187f 100644 --- a/docs/en/operations/settings/settings-formats.md +++ b/docs/en/operations/settings/settings-formats.md @@ -1223,6 +1223,12 @@ Allow skipping columns with unsupported types while schema inference for format Disabled by default. +### input_format_parquet_local_file_min_bytes_for_seek {#input_format_parquet_local_file_min_bytes_for_seek} + +min bytes required for local read (file) to do seek, instead of read with ignore in Parquet input format. + +Default value - `8192`. + ### output_format_parquet_string_as_string {#output_format_parquet_string_as_string} Use Parquet String type instead of Binary for String columns. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6c3d339b4be..0bff3f58fd5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -873,6 +873,7 @@ class IColumn; M(Bool, input_format_allow_seeks, true, "Allow seeks while reading in ORC/Parquet/Arrow input formats", 0) \ M(Bool, input_format_orc_allow_missing_columns, false, "Allow missing columns while reading ORC input formats", 0) \ M(Bool, input_format_parquet_allow_missing_columns, false, "Allow missing columns while reading Parquet input formats", 0) \ + M(UInt64, input_format_parquet_local_file_min_bytes_for_seek, 8192, "Min bytes required for local read (file) to do seek, instead of read with ignore in Parquet input format", 0) \ M(Bool, input_format_arrow_allow_missing_columns, false, "Allow missing columns while reading Arrow input formats", 0) \ M(Char, input_format_hive_text_fields_delimiter, '\x01', "Delimiter between fields in Hive Text File", 0) \ M(Char, input_format_hive_text_collection_items_delimiter, '\x02', "Delimiter between collection(array or map) items in Hive Text File", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index dd07ca7c981..56d27a59315 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -133,6 +133,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.parquet.parallel_encoding = settings.output_format_parquet_parallel_encoding; format_settings.parquet.data_page_size = settings.output_format_parquet_data_page_size; format_settings.parquet.write_batch_size = settings.output_format_parquet_batch_size; + format_settings.parquet.local_read_min_bytes_for_seek = settings.input_format_parquet_local_file_min_bytes_for_seek; format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; format_settings.pretty.color = settings.output_format_pretty_color; format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 2c283dcc2b7..56eb7686f56 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -241,6 +241,7 @@ struct FormatSettings bool output_compliant_nested_types = true; size_t data_page_size = 1024 * 1024; size_t write_batch_size = 1024; + size_t local_read_min_bytes_for_seek = 8192; } parquet; struct Pretty diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index c0bad5ff116..0793a6f570b 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -413,7 +413,7 @@ void registerInputFormatParquet(FormatFactory & factory) size_t /* max_download_threads */, size_t max_parsing_threads) { - size_t min_bytes_for_seek = is_remote_fs ? read_settings.remote_read_min_bytes_for_seek : 8 * 1024; + size_t min_bytes_for_seek = is_remote_fs ? read_settings.remote_read_min_bytes_for_seek : settings.parquet.local_read_min_bytes_for_seek; return std::make_shared( buf, sample, diff --git a/tests/queries/0_stateless/02725_parquet_preserve_order.sh b/tests/queries/0_stateless/02725_parquet_preserve_order.sh index ac29ef3f361..94f2eaaa753 100755 --- a/tests/queries/0_stateless/02725_parquet_preserve_order.sh +++ b/tests/queries/0_stateless/02725_parquet_preserve_order.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # It'll be read into two blocks. The first block will sleep 2x longer than the second. # So reordering is very likely if the order-preservation doesn't work. -$CLICKHOUSE_LOCAL -q "select number + sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=1, function_sleep_max_microseconds_per_block = 6000000" +$CLICKHOUSE_LOCAL -q "select number + sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=1, function_sleep_max_microseconds_per_block = 6000000, input_format_parquet_local_file_min_bytes_for_seek=0" -$CLICKHOUSE_LOCAL -q "explain pipeline select number + sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=1, max_threads=2" -$CLICKHOUSE_LOCAL -q "explain pipeline select number + sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=0, parallelize_output_from_storages=1, max_threads=2" +$CLICKHOUSE_LOCAL -q "explain pipeline select number + sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=1, max_threads=2, input_format_parquet_local_file_min_bytes_for_seek=0" +$CLICKHOUSE_LOCAL -q "explain pipeline select number + sleepEachRow(3) from file('$CURDIR/data_parquet/02725_data.parquet') settings input_format_parquet_preserve_order=0, parallelize_output_from_storages=1, max_threads=2, input_format_parquet_local_file_min_bytes_for_seek=0" From c823d1f6193fc7394dc345b022618d4e83402a66 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 17 Aug 2023 12:20:45 +0000 Subject: [PATCH 658/777] Make usearch dependencies separate submodules --- .gitmodules | 9 +++++++++ contrib/CMakeLists.txt | 12 +++++++++++- contrib/FP16 | 1 + contrib/FP16-cmake/CMakeLists.txt | 1 + contrib/SimSIMD | 1 + contrib/SimSIMD-cmake/CMakeLists.txt | 1 + contrib/robin-map | 1 + contrib/robin-map-cmake/CMakeLists.txt | 1 + contrib/usearch-cmake/CMakeLists.txt | 20 +++++++++++--------- 9 files changed, 37 insertions(+), 10 deletions(-) create mode 160000 contrib/FP16 create mode 100644 contrib/FP16-cmake/CMakeLists.txt create mode 160000 contrib/SimSIMD create mode 100644 contrib/SimSIMD-cmake/CMakeLists.txt create mode 160000 contrib/robin-map create mode 100644 contrib/robin-map-cmake/CMakeLists.txt diff --git a/.gitmodules b/.gitmodules index 36721723371..c3592372b7e 100644 --- a/.gitmodules +++ b/.gitmodules @@ -350,3 +350,12 @@ [submodule "contrib/usearch"] path = contrib/usearch url = https://github.com/unum-cloud/usearch.git +[submodule "contrib/SimSIMD"] + path = contrib/SimSIMD + url = https://github.com/ashvardanian/SimSIMD.git +[submodule "contrib/FP16"] + path = contrib/FP16 + url = https://github.com/Maratyszcza/FP16.git +[submodule "contrib/robin-map"] + path = contrib/robin-map + url = https://github.com/Tessil/robin-map.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index abecfcb30c8..2557ebf78ae 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -196,7 +196,17 @@ if (ARCH_S390X) add_contrib(crc32-s390x-cmake crc32-s390x) endif() add_contrib (annoy-cmake annoy) -add_contrib (usearch-cmake usearch) + +option(ENABLE_USEARCH "Enable USearch (Approximate Neighborhood Search, HNSW) support" ${ENABLE_LIBRARIES}) +if (ENABLE_USEARCH) + add_contrib (FP16-cmake FP16) + add_contrib (robin-map-cmake robin-map) + add_contrib (SimSIMD-cmake SimSIMD) + add_contrib (usearch-cmake usearch) # requires: FP16, robin-map, SimdSIMD +else () + message(STATUS "Not using USearch") +endif () + add_contrib (xxHash-cmake xxHash) add_contrib (libbcrypt-cmake libbcrypt) diff --git a/contrib/FP16 b/contrib/FP16 new file mode 160000 index 00000000000..0a92994d729 --- /dev/null +++ b/contrib/FP16 @@ -0,0 +1 @@ +Subproject commit 0a92994d729ff76a58f692d3028ca1b64b145d91 diff --git a/contrib/FP16-cmake/CMakeLists.txt b/contrib/FP16-cmake/CMakeLists.txt new file mode 100644 index 00000000000..f82ad705dcc --- /dev/null +++ b/contrib/FP16-cmake/CMakeLists.txt @@ -0,0 +1 @@ +# See contrib/usearch-cmake/CMakeLists.txt diff --git a/contrib/SimSIMD b/contrib/SimSIMD new file mode 160000 index 00000000000..de2cb75b9e9 --- /dev/null +++ b/contrib/SimSIMD @@ -0,0 +1 @@ +Subproject commit de2cb75b9e9e3389d5e1e51fd9f8ed151f3c17cf diff --git a/contrib/SimSIMD-cmake/CMakeLists.txt b/contrib/SimSIMD-cmake/CMakeLists.txt new file mode 100644 index 00000000000..f82ad705dcc --- /dev/null +++ b/contrib/SimSIMD-cmake/CMakeLists.txt @@ -0,0 +1 @@ +# See contrib/usearch-cmake/CMakeLists.txt diff --git a/contrib/robin-map b/contrib/robin-map new file mode 160000 index 00000000000..851a59e0e30 --- /dev/null +++ b/contrib/robin-map @@ -0,0 +1 @@ +Subproject commit 851a59e0e3063ee0e23089062090a73fd3de482d diff --git a/contrib/robin-map-cmake/CMakeLists.txt b/contrib/robin-map-cmake/CMakeLists.txt new file mode 100644 index 00000000000..f82ad705dcc --- /dev/null +++ b/contrib/robin-map-cmake/CMakeLists.txt @@ -0,0 +1 @@ +# See contrib/usearch-cmake/CMakeLists.txt diff --git a/contrib/usearch-cmake/CMakeLists.txt b/contrib/usearch-cmake/CMakeLists.txt index f4ed6a9adca..29fbe57106c 100644 --- a/contrib/usearch-cmake/CMakeLists.txt +++ b/contrib/usearch-cmake/CMakeLists.txt @@ -1,15 +1,17 @@ -option(ENABLE_USEARCH "Enable USearch (Approximate Neighborhood Search, HNSW) support" ${ENABLE_LIBRARIES}) - -if (NOT ENABLE_USEARCH) - message (STATUS "Not using usearch") - return() -endif() - set(USEARCH_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/usearch") set(USEARCH_SOURCE_DIR "${USEARCH_PROJECT_DIR}/include") +set(FP16_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/FP16") +set(ROBIN_MAP_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/robin-map") +set(SIMSIMD_PROJECT_DIR "${ClickHouse_SOURCE_DIR}/contrib/SimSIMD-map") + add_library(_usearch INTERFACE) -target_include_directories(_usearch SYSTEM INTERFACE ${USEARCH_PROJECT_DIR}/fp16/include ${USEARCH_PROJECT_DIR}/robin-map/include ${USEARCH_PROJECT_DIR}/simsimd/include ${USEARCH_SOURCE_DIR}) + +target_include_directories(_usearch SYSTEM INTERFACE + ${FP16_PROJECT_DIR}/include + ${ROBIN_MAP_PROJECT_DIR}/include + ${SIMSIMD_PROJECT_DIR}/include + ${USEARCH_SOURCE_DIR}) add_library(ch_contrib::usearch ALIAS _usearch) -target_compile_definitions(_usearch INTERFACE ENABLE_USEARCH) \ No newline at end of file +target_compile_definitions(_usearch INTERFACE ENABLE_USEARCH) From 409dbbe7a4a8e7df0268a9960bb2f348ad9209bd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 17 Aug 2023 12:40:30 +0000 Subject: [PATCH 659/777] Make test name consistent with index name --- .../{02354_hnsw_index.reference => 02354_usearch_index.reference} | 0 .../0_stateless/{02354_hnsw_index.sql => 02354_usearch_index.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02354_hnsw_index.reference => 02354_usearch_index.reference} (100%) rename tests/queries/0_stateless/{02354_hnsw_index.sql => 02354_usearch_index.sql} (100%) diff --git a/tests/queries/0_stateless/02354_hnsw_index.reference b/tests/queries/0_stateless/02354_usearch_index.reference similarity index 100% rename from tests/queries/0_stateless/02354_hnsw_index.reference rename to tests/queries/0_stateless/02354_usearch_index.reference diff --git a/tests/queries/0_stateless/02354_hnsw_index.sql b/tests/queries/0_stateless/02354_usearch_index.sql similarity index 100% rename from tests/queries/0_stateless/02354_hnsw_index.sql rename to tests/queries/0_stateless/02354_usearch_index.sql From c2340d588f1a4cdc085ed2364edd533a5c6695ba Mon Sep 17 00:00:00 2001 From: Salvatore Mesoraca Date: Thu, 17 Aug 2023 14:57:34 +0200 Subject: [PATCH 660/777] Fix issue found by fuzzer --- src/Functions/FunctionsHashing.h | 6 ++++-- tests/queries/0_stateless/02534_keyed_siphash.reference | 2 ++ tests/queries/0_stateless/02534_keyed_siphash.sql | 4 ++++ 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 479f15a9e31..c4dd14768f5 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -126,6 +126,9 @@ namespace impl if (!checkColumn(*ret.key1)) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "second element of the key tuple is not UInt64"); + if (ret.size() == 1) + ret.is_const = true; + return ret; } } @@ -1384,8 +1387,7 @@ private: icolumn->getName(), icolumn->size(), vec_to.size(), getName()); if constexpr (Keyed) - if ((!key_cols.is_const && key_cols.size() != vec_to.size()) - || (key_cols.is_const && key_cols.size() != 1)) + if (key_cols.size() != vec_to.size() && key_cols.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Key column size {} doesn't match result column size {} of function {}", key_cols.size(), vec_to.size(), getName()); if (which.isUInt8()) executeIntType(key_cols, icolumn, vec_to); diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index eea5b6ec1a4..e3fae07333a 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -234,3 +234,5 @@ Check multiple keys as separate ints from a table with constant data F6D93D8FEA6D7DECCDD95A7A0A2AA36D Check asan bug 0 +Check bug found fuzzing +9042C6691B1A75F0EA3314B6F55728BB diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 2495a47671a..112ae15bf46 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -334,3 +334,7 @@ DROP TABLE sipHashKeyed_keys; SELECT 'Check asan bug'; SELECT sipHash128((toUInt64(9223372036854775806), 1)) = sipHash128(1) GROUP BY sipHash128(1::UInt8), toUInt64(9223372036854775806); + +SELECT 'Check bug found fuzzing'; +SELECT [(255, 1048575)], sipHash128ReferenceKeyed((toUInt64(2147483646), toUInt64(9223372036854775807)), ([(NULL, 100), (NULL, NULL), (1024, 10)], toUInt64(2), toUInt64(1024)), ''), hex(sipHash128ReferenceKeyed((-9223372036854775807, 1.), '-1', NULL)), ('', toUInt64(65535), [(9223372036854775807, 9223372036854775806)], toUInt64(65536)), arrayJoin((NULL, 65537, 255), [(NULL, NULL)]) GROUP BY tupleElement((NULL, NULL, NULL, -1), toUInt64(2), 2) = NULL; -- { serverError NOT_IMPLEMENTED } +SELECT hex(sipHash128ReferenceKeyed((0::UInt64, 0::UInt64), ([1, 1]))); From 7115110dad245be1c3507b2d3b61045e1f0afc37 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Thu, 17 Aug 2023 15:59:22 +0200 Subject: [PATCH 661/777] Update merge-tree-settings.md parts_to_throw_insert=3000 --- docs/en/operations/settings/merge-tree-settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 4122b4af40f..8ea599b9861 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -56,11 +56,11 @@ Possible values: - Any positive integer. -Default value: 300. +Default value: 3000. To achieve maximum performance of `SELECT` queries, it is necessary to minimize the number of parts processed, see [Merge Tree](../../development/architecture.md#merge-tree). -You can set a larger value to 600 (1200), this will reduce the probability of the `Too many parts` error, but at the same time `SELECT` performance might degrade. Also in case of a merge issue (for example, due to insufficient disk space) you will notice it later than it could be with the original 300. +Prior to 23.6 this setting was set to 300. You can set a higher different value, it will reduce the probability of the `Too many parts` error, but at the same time `SELECT` performance might degrade. Also in case of a merge issue (for example, due to insufficient disk space) you will notice it later than it could be with the original 300. ## parts_to_delay_insert {#parts-to-delay-insert} From fdca288a73aefe8cb2d591ffe709613b070c630a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 17 Aug 2023 16:49:50 +0200 Subject: [PATCH 662/777] make sending logs less fragile --- docker/test/base/setup_export_logs.sh | 6 +++++- .../0_stateless/02443_detach_attach_partition.sh | 11 +++++++++-- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 12fae855b03..d68fd392d62 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -17,6 +17,9 @@ CONNECTION_PARAMETERS=${CONNECTION_PARAMETERS:=""} # Create all configured system logs: clickhouse-client --query "SYSTEM FLUSH LOGS" +# It's doesn't make sense to try creating tables if SYNC fails +echo "SYSTEM SYNC DATABASE REPLICA default" clickhouse-client --receive_timeout 180 $CONNECTION_PARAMETERS || exit 0 + # For each system log table: clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table do @@ -38,7 +41,7 @@ do echo "Creating destination table ${table}_${hash}" >&2 - echo "$statement" | clickhouse-client $CONNECTION_PARAMETERS + echo "$statement" | clickhouse-client --distributed_ddl_task_timeout=10 $CONNECTION_PARAMETERS || continue echo "Creating table system.${table}_sender" >&2 @@ -46,6 +49,7 @@ do clickhouse-client --query " CREATE TABLE system.${table}_sender ENGINE = Distributed(${CLUSTER}, default, ${table}_${hash}) + SETTINGS flush_on_detach=0 EMPTY AS SELECT ${EXTRA_COLUMNS_EXPRESSION}, * FROM system.${table} diff --git a/tests/queries/0_stateless/02443_detach_attach_partition.sh b/tests/queries/0_stateless/02443_detach_attach_partition.sh index ae104b833e3..2bb7361e9da 100755 --- a/tests/queries/0_stateless/02443_detach_attach_partition.sh +++ b/tests/queries/0_stateless/02443_detach_attach_partition.sh @@ -30,17 +30,24 @@ function thread_attach() } insert_type=$(($RANDOM % 3)) +if [[ "$engine" == "ReplicatedMergeTree" ]]; then + insert_type=$(($RANDOM % 2)) +fi $CLICKHOUSE_CLIENT -q "SELECT '$CLICKHOUSE_DATABASE', 'insert_type $insert_type' FORMAT Null" function insert() { # Fault injection may lead to duplicates if [[ "$insert_type" -eq 0 ]]; then - $CLICKHOUSE_CLIENT --insert_deduplication_token=$1 -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $RANDOM, $1" 2>/dev/null + $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $RANDOM, $1" 2>/dev/null elif [[ "$insert_type" -eq 1 ]]; then $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $1, $1" 2>/dev/null else - $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $RANDOM, $1" 2>/dev/null + # It may reproduce something interesting: if the insert status is unknown (due to fault injection in retries) + # and the part was committed locally but not in zk, then it will be active and DETACH may detach it. + # And we will ATTACH it later. But the next INSERT attempt will not be deduplicated because the first one failed. + # So we will get duplicates. + $CLICKHOUSE_CLIENT --insert_deduplication_token=$1 -q "INSERT INTO alter_table$(($RANDOM % 2)) SELECT $RANDOM, $1" 2>/dev/null fi } From 80b767316dc27131c96ce7b6b1b5c959cca7b721 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 15 Aug 2023 18:55:40 +0200 Subject: [PATCH 663/777] Deduplicate same watch callbacks registered multiple times --- src/Common/ZooKeeper/IKeeper.h | 10 ++-- src/Common/ZooKeeper/TestKeeper.cpp | 24 ++++----- src/Common/ZooKeeper/TestKeeper.h | 10 ++-- src/Common/ZooKeeper/ZooKeeper.cpp | 50 +++++++++++++++---- src/Common/ZooKeeper/ZooKeeper.h | 16 +++++- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 27 ++++++---- src/Common/ZooKeeper/ZooKeeperImpl.h | 10 ++-- .../examples/zkutil_test_commands_new_lib.cpp | 49 ++++++++++-------- 8 files changed, 127 insertions(+), 69 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index e2c9afdd5da..ec23b52ceb1 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -165,6 +165,10 @@ struct WatchResponse : virtual Response }; using WatchCallback = std::function; +/// Passing watch callback as a shared_ptr allows to +/// - avoid copying of the callback +/// - registering the same callback only once per path +using WatchCallbackPtr = std::shared_ptr; struct SetACLRequest : virtual Request { @@ -557,12 +561,12 @@ public: virtual void exists( const String & path, ExistsCallback callback, - WatchCallback watch) = 0; + WatchCallbackPtr watch) = 0; virtual void get( const String & path, GetCallback callback, - WatchCallback watch) = 0; + WatchCallbackPtr watch) = 0; virtual void set( const String & path, @@ -574,7 +578,7 @@ public: const String & path, ListRequestType list_request_type, ListCallback callback, - WatchCallback watch) = 0; + WatchCallbackPtr watch) = 0; virtual void check( const String & path, diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 9e355093c9d..f46c3f814a9 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -42,9 +42,9 @@ static void processWatchesImpl(const String & path, TestKeeper::Watches & watche auto it = watches.find(watch_response.path); if (it != watches.end()) { - for (auto & callback : it->second) + for (const auto & callback : it->second) if (callback) - callback(watch_response); + (*callback)(watch_response); watches.erase(it); } @@ -55,9 +55,9 @@ static void processWatchesImpl(const String & path, TestKeeper::Watches & watche it = list_watches.find(watch_list_response.path); if (it != list_watches.end()) { - for (auto & callback : it->second) + for (const auto & callback : it->second) if (callback) - callback(watch_list_response); + (*callback)(watch_list_response); list_watches.erase(it); } @@ -587,11 +587,11 @@ void TestKeeper::processingThread() ? list_watches : watches; - watches_type[info.request->getPath()].emplace_back(std::move(info.watch)); + watches_type[info.request->getPath()].insert(info.watch); } else if (response->error == Error::ZNONODE && dynamic_cast(info.request.get())) { - watches[info.request->getPath()].emplace_back(std::move(info.watch)); + watches[info.request->getPath()].insert(info.watch); } } @@ -634,13 +634,13 @@ void TestKeeper::finalize(const String &) response.state = EXPIRED_SESSION; response.error = Error::ZSESSIONEXPIRED; - for (auto & callback : path_watch.second) + for (const auto & callback : path_watch.second) { if (callback) { try { - callback(response); + (*callback)(response); } catch (...) { @@ -677,7 +677,7 @@ void TestKeeper::finalize(const String &) response.error = Error::ZSESSIONEXPIRED; try { - info.watch(response); + (*info.watch)(response); } catch (...) { @@ -756,7 +756,7 @@ void TestKeeper::remove( void TestKeeper::exists( const String & path, ExistsCallback callback, - WatchCallback watch) + WatchCallbackPtr watch) { TestKeeperExistsRequest request; request.path = path; @@ -771,7 +771,7 @@ void TestKeeper::exists( void TestKeeper::get( const String & path, GetCallback callback, - WatchCallback watch) + WatchCallbackPtr watch) { TestKeeperGetRequest request; request.path = path; @@ -804,7 +804,7 @@ void TestKeeper::list( const String & path, ListRequestType list_request_type, ListCallback callback, - WatchCallback watch) + WatchCallbackPtr watch) { TestKeeperFilteredListRequest request; request.path = path; diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 991f689394e..3a760270207 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -59,12 +59,12 @@ public: void exists( const String & path, ExistsCallback callback, - WatchCallback watch) override; + WatchCallbackPtr watch) override; void get( const String & path, GetCallback callback, - WatchCallback watch) override; + WatchCallbackPtr watch) override; void set( const String & path, @@ -76,7 +76,7 @@ public: const String & path, ListRequestType list_request_type, ListCallback callback, - WatchCallback watch) override; + WatchCallbackPtr watch) override; void check( const String & path, @@ -117,7 +117,7 @@ public: using Container = std::map; - using WatchCallbacks = std::vector; + using WatchCallbacks = std::unordered_set; using Watches = std::map; private: @@ -127,7 +127,7 @@ private: { TestKeeperRequestPtr request; ResponseCallback callback; - WatchCallback watch; + WatchCallbackPtr watch; clock::time_point time; }; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index b7f85293e94..cc65acdc98e 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -212,7 +212,7 @@ static Coordination::WatchCallback callbackForEvent(const EventPtr & watch) Coordination::Error ZooKeeper::getChildrenImpl(const std::string & path, Strings & res, Coordination::Stat * stat, - Coordination::WatchCallback watch_callback, + Coordination::WatchCallbackPtr watch_callback, Coordination::ListRequestType list_request_type) { auto future_result = asyncTryGetChildrenNoThrow(path, watch_callback, list_request_type); @@ -250,6 +250,13 @@ Strings ZooKeeper::getChildrenWatch(const std::string & path, Coordination::Stat return res; } +Strings ZooKeeper::getChildrenWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallbackPtr watch_callback, Coordination::ListRequestType list_request_type) +{ + Strings res; + check(tryGetChildrenWatch(path, res, stat, watch_callback, list_request_type), path); + return res; +} + Coordination::Error ZooKeeper::tryGetChildren( const std::string & path, Strings & res, @@ -257,7 +264,9 @@ Coordination::Error ZooKeeper::tryGetChildren( const EventPtr & watch, Coordination::ListRequestType list_request_type) { - Coordination::Error code = getChildrenImpl(path, res, stat, callbackForEvent(watch), list_request_type); + Coordination::Error code = getChildrenImpl(path, res, stat, + watch ? std::make_shared(callbackForEvent(watch)) : Coordination::WatchCallbackPtr{}, + list_request_type); if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) throw KeeperException::fromPath(code, path); @@ -271,6 +280,22 @@ Coordination::Error ZooKeeper::tryGetChildrenWatch( Coordination::Stat * stat, Coordination::WatchCallback watch_callback, Coordination::ListRequestType list_request_type) +{ + Coordination::Error code = getChildrenImpl(path, res, stat, + watch_callback ? std::make_shared(watch_callback) : Coordination::WatchCallbackPtr{}, + list_request_type); + + if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) + throw KeeperException(code, path); + + return code; +} +Coordination::Error ZooKeeper::tryGetChildrenWatch( + const std::string & path, + Strings & res, + Coordination::Stat * stat, + Coordination::WatchCallbackPtr watch_callback, + Coordination::ListRequestType list_request_type) { Coordination::Error code = getChildrenImpl(path, res, stat, watch_callback, list_request_type); @@ -814,7 +839,7 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & do { /// Use getData insteand of exists to avoid watch leak. - impl->get(path, callback, watch); + impl->get(path, callback, std::make_shared(watch)); if (!state->event.tryWait(1000)) continue; @@ -929,7 +954,8 @@ std::future ZooKeeper::asyncGet(const std::string & p promise->set_value(response); }; - impl->get(path, std::move(callback), watch_callback); + impl->get(path, std::move(callback), + watch_callback ? std::make_shared(watch_callback) : Coordination::WatchCallbackPtr{}); return future; } @@ -943,7 +969,8 @@ std::future ZooKeeper::asyncTryGetNoThrow(const std:: promise->set_value(response); }; - impl->get(path, std::move(callback), watch_callback); + impl->get(path, std::move(callback), + watch_callback ? std::make_shared(watch_callback) : Coordination::WatchCallbackPtr{}); return future; } @@ -978,7 +1005,8 @@ std::future ZooKeeper::asyncExists(const std::stri promise->set_value(response); }; - impl->exists(path, std::move(callback), watch_callback); + impl->exists(path, std::move(callback), + watch_callback ? std::make_shared(watch_callback) : Coordination::WatchCallbackPtr{}); return future; } @@ -992,7 +1020,8 @@ std::future ZooKeeper::asyncTryExistsNoThrow(const promise->set_value(response); }; - impl->exists(path, std::move(callback), watch_callback); + impl->exists(path, std::move(callback), + watch_callback ? std::make_shared(watch_callback) : Coordination::WatchCallbackPtr{}); return future; } @@ -1042,12 +1071,13 @@ std::future ZooKeeper::asyncGetChildren( promise->set_value(response); }; - impl->list(path, list_request_type, std::move(callback), watch_callback); + impl->list(path, list_request_type, std::move(callback), + watch_callback ? std::make_shared(watch_callback) : Coordination::WatchCallbackPtr{}); return future; } std::future ZooKeeper::asyncTryGetChildrenNoThrow( - const std::string & path, Coordination::WatchCallback watch_callback, Coordination::ListRequestType list_request_type) + const std::string & path, Coordination::WatchCallbackPtr watch_callback, Coordination::ListRequestType list_request_type) { auto promise = std::make_shared>(); auto future = promise->get_future(); @@ -1057,7 +1087,7 @@ std::future ZooKeeper::asyncTryGetChildrenNoThrow( promise->set_value(response); }; - impl->list(path, list_request_type, std::move(callback), watch_callback); + impl->list(path, list_request_type, std::move(callback), std::move(watch_callback)); return future; } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index c2cba7ef401..38c81369c73 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -333,6 +333,11 @@ public: Coordination::WatchCallback watch_callback, Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL); + Strings getChildrenWatch(const std::string & path, + Coordination::Stat * stat, + Coordination::WatchCallbackPtr watch_callback, + Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL); + using MultiGetChildrenResponse = MultiReadResponses; using MultiTryGetChildrenResponse = MultiReadResponses; @@ -369,6 +374,13 @@ public: Coordination::WatchCallback watch_callback, Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL); + Coordination::Error tryGetChildrenWatch( + const std::string & path, + Strings & res, + Coordination::Stat * stat, + Coordination::WatchCallbackPtr watch_callback, + Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL); + template MultiTryGetChildrenResponse tryGetChildren(TIter start, TIter end, Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL) @@ -474,7 +486,7 @@ public: /// Like the previous one but don't throw any exceptions on future.get() FutureGetChildren asyncTryGetChildrenNoThrow( const std::string & path, - Coordination::WatchCallback watch_callback = {}, + Coordination::WatchCallbackPtr watch_callback = {}, Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL); using FutureSet = std::future; @@ -545,7 +557,7 @@ private: const std::string & path, Strings & res, Coordination::Stat * stat, - Coordination::WatchCallback watch_callback, + Coordination::WatchCallbackPtr watch_callback, Coordination::ListRequestType list_request_type); Coordination::Error multiImpl(const Coordination::Requests & requests, Coordination::Responses & responses); Coordination::Error existsImpl(const std::string & path, Coordination::Stat * stat_, Coordination::WatchCallback watch_callback); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 4dbdcf51b24..886522687bd 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -782,9 +782,9 @@ void ZooKeeper::receiveEvent() } else { - for (auto & callback : it->second) + for (const auto & callback : it->second) if (callback) - callback(watch_response); /// NOTE We may process callbacks not under mutex. + (*callback)(watch_response); /// NOTE We may process callbacks not under mutex. CurrentMetrics::sub(CurrentMetrics::ZooKeeperWatch, it->second.size()); watches.erase(it); @@ -846,13 +846,17 @@ void ZooKeeper::receiveEvent() if (add_watch) { - CurrentMetrics::add(CurrentMetrics::ZooKeeperWatch); /// The key of wathces should exclude the args.chroot String req_path = request_info.request->getPath(); removeRootPath(req_path, args.chroot); std::lock_guard lock(watches_mutex); - watches[req_path].emplace_back(std::move(request_info.watch)); + auto & callbacks = watches[req_path]; + if (request_info.watch && *request_info.watch) + { + if (callbacks.insert(request_info.watch).second) + CurrentMetrics::add(CurrentMetrics::ZooKeeperWatch); + } } } @@ -1002,14 +1006,14 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea response.state = EXPIRED_SESSION; response.error = Error::ZSESSIONEXPIRED; - for (auto & callback : path_watches.second) + for (const auto & callback : path_watches.second) { watch_callback_count += 1; if (callback) { try { - callback(response); + (*callback)(response); } catch (...) { @@ -1054,7 +1058,7 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea response.error = Error::ZSESSIONEXPIRED; try { - info.watch(response); + (*info.watch)(response); } catch (...) { @@ -1232,7 +1236,7 @@ void ZooKeeper::remove( void ZooKeeper::exists( const String & path, ExistsCallback callback, - WatchCallback watch) + WatchCallbackPtr watch) { ZooKeeperExistsRequest request; request.path = path; @@ -1250,7 +1254,7 @@ void ZooKeeper::exists( void ZooKeeper::get( const String & path, GetCallback callback, - WatchCallback watch) + WatchCallbackPtr watch) { ZooKeeperGetRequest request; request.path = path; @@ -1289,7 +1293,7 @@ void ZooKeeper::list( const String & path, ListRequestType list_request_type, ListCallback callback, - WatchCallback watch) + WatchCallbackPtr watch) { std::shared_ptr request{nullptr}; if (!isFeatureEnabled(KeeperFeatureFlag::FILTERED_LIST)) @@ -1310,7 +1314,8 @@ void ZooKeeper::list( RequestInfo request_info; request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; - request_info.watch = watch; + if (watch) + request_info.watch = std::move(watch); request_info.request = std::move(request); pushRequest(std::move(request_info)); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 6a6f34b8b3d..56e199352e9 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -154,12 +154,12 @@ public: void exists( const String & path, ExistsCallback callback, - WatchCallback watch) override; + WatchCallbackPtr watch) override; void get( const String & path, GetCallback callback, - WatchCallback watch) override; + WatchCallbackPtr watch) override; void set( const String & path, @@ -171,7 +171,7 @@ public: const String & path, ListRequestType list_request_type, ListCallback callback, - WatchCallback watch) override; + WatchCallbackPtr watch) override; void check( const String & path, @@ -252,7 +252,7 @@ private: { ZooKeeperRequestPtr request; ResponseCallback callback; - WatchCallback watch; + WatchCallbackPtr watch; clock::time_point time; }; @@ -267,7 +267,7 @@ private: Operations operations TSA_GUARDED_BY(operations_mutex); std::mutex operations_mutex; - using WatchCallbacks = std::vector; + using WatchCallbacks = std::unordered_set; using Watches = std::map; Watches watches TSA_GUARDED_BY(watches_mutex); diff --git a/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp b/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp index 021f444386a..fe38b486ada 100644 --- a/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp +++ b/src/Common/ZooKeeper/examples/zkutil_test_commands_new_lib.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -72,13 +73,15 @@ try //event.set(); }, - [](const WatchResponse & response) - { - if (response.error != Coordination::Error::ZOK) - std::cerr << "Watch (get) on /test, Error: " << errorMessage(response.error) << '\n'; - else - std::cerr << "Watch (get) on /test, path: " << response.path << ", type: " << response.type << '\n'; - }); + std::make_shared( + [](const WatchResponse & response) + { + if (response.error != Coordination::Error::ZOK) + std::cerr << "Watch (get) on /test, Error: " << errorMessage(response.error) << '\n'; + else + std::cerr << "Watch (get) on /test, path: " << response.path << ", type: " << response.type << '\n'; + }) + ); //event.wait(); @@ -114,13 +117,15 @@ try //event.set(); }, - [](const WatchResponse & response) - { - if (response.error != Coordination::Error::ZOK) - std::cerr << "Watch (list) on /, Error: " << errorMessage(response.error) << '\n'; - else - std::cerr << "Watch (list) on /, path: " << response.path << ", type: " << response.type << '\n'; - }); + std::make_shared( + [](const WatchResponse & response) + { + if (response.error != Coordination::Error::ZOK) + std::cerr << "Watch (list) on /, Error: " << errorMessage(response.error) << '\n'; + else + std::cerr << "Watch (list) on /, path: " << response.path << ", type: " << response.type << '\n'; + }) + ); //event.wait(); @@ -136,13 +141,15 @@ try //event.set(); }, - [](const WatchResponse & response) - { - if (response.error != Coordination::Error::ZOK) - std::cerr << "Watch (exists) on /test, Error: " << errorMessage(response.error) << '\n'; - else - std::cerr << "Watch (exists) on /test, path: " << response.path << ", type: " << response.type << '\n'; - }); + std::make_shared( + [](const WatchResponse & response) + { + if (response.error != Coordination::Error::ZOK) + std::cerr << "Watch (exists) on /test, Error: " << errorMessage(response.error) << '\n'; + else + std::cerr << "Watch (exists) on /test, path: " << response.path << ", type: " << response.type << '\n'; + }) + ); //event.wait(); From 54e2be4e327d1bbbc68d250a1cf2ae9065d2bc50 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 15 Aug 2023 22:30:50 +0200 Subject: [PATCH 664/777] Use WatchCallbackPtr --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeQueue.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 4 +++- src/Storages/StorageReplicatedMergeTree.h | 1 + 4 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 84307a3ca7a..334d47288d5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -866,7 +866,7 @@ ActiveDataPartSet getPartNamesToMutate( } -void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback) +void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallbackPtr watch_callback) { std::lock_guard lock(update_mutations_mutex); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 611866877d8..d5d85e58cb5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -335,7 +335,7 @@ public: /// Load new mutation entries. If something new is loaded, schedule storage.merge_selecting_task. /// If watch_callback is not empty, will call it when new mutations appear in ZK. - void updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallback watch_callback = {}); + void updateMutations(zkutil::ZooKeeperPtr zookeeper, Coordination::WatchCallbackPtr watch_callback = {}); /// Remove a mutation from ZooKeeper and from the local set. Returns the removed entry or nullptr /// if it could not be found. Called during KILL MUTATION query execution. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 62db6d2d7b7..316723d4013 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -338,6 +338,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( mutations_updating_task->deactivate(); + mutations_watch_callback = std::make_shared(mutations_updating_task->getWatchCallback()); + merge_selecting_task = getContext()->getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::mergeSelectingTask)", [this] { mergeSelectingTask(); }); @@ -3217,7 +3219,7 @@ void StorageReplicatedMergeTree::mutationsUpdatingTask() { try { - queue.updateMutations(getZooKeeper(), mutations_updating_task->getWatchCallback()); + queue.updateMutations(getZooKeeper(), mutations_watch_callback); } catch (const Coordination::Exception & e) { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 78ef39f032f..8e9eed678c8 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -497,6 +497,7 @@ private: BackgroundSchedulePool::TaskHolder queue_updating_task; BackgroundSchedulePool::TaskHolder mutations_updating_task; + Coordination::WatchCallbackPtr mutations_watch_callback; /// A task that selects parts to merge. BackgroundSchedulePool::TaskHolder merge_selecting_task; From 827ae49b954a2bce5d1e5e818b14060ad96e8fcf Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 17 Aug 2023 13:14:40 +0200 Subject: [PATCH 665/777] Cleanups --- src/Common/ZooKeeper/ZooKeeper.cpp | 17 ++++------------- 1 file changed, 4 insertions(+), 13 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index cc65acdc98e..85db0dcd2fe 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -264,14 +264,9 @@ Coordination::Error ZooKeeper::tryGetChildren( const EventPtr & watch, Coordination::ListRequestType list_request_type) { - Coordination::Error code = getChildrenImpl(path, res, stat, + return tryGetChildrenWatch(path, res, stat, watch ? std::make_shared(callbackForEvent(watch)) : Coordination::WatchCallbackPtr{}, list_request_type); - - if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) - throw KeeperException::fromPath(code, path); - - return code; } Coordination::Error ZooKeeper::tryGetChildrenWatch( @@ -281,15 +276,11 @@ Coordination::Error ZooKeeper::tryGetChildrenWatch( Coordination::WatchCallback watch_callback, Coordination::ListRequestType list_request_type) { - Coordination::Error code = getChildrenImpl(path, res, stat, + return tryGetChildrenWatch(path, res, stat, watch_callback ? std::make_shared(watch_callback) : Coordination::WatchCallbackPtr{}, list_request_type); - - if (!(code == Coordination::Error::ZOK || code == Coordination::Error::ZNONODE)) - throw KeeperException(code, path); - - return code; } + Coordination::Error ZooKeeper::tryGetChildrenWatch( const std::string & path, Strings & res, @@ -1087,7 +1078,7 @@ std::future ZooKeeper::asyncTryGetChildrenNoThrow( promise->set_value(response); }; - impl->list(path, list_request_type, std::move(callback), std::move(watch_callback)); + impl->list(path, list_request_type, std::move(callback), watch_callback); return future; } From f20dd27ba6385b7432556360d8397f700811e832 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 17 Aug 2023 18:45:26 +0000 Subject: [PATCH 666/777] Clean header mess up --- docs/en/sql-reference/dictionaries/index.md | 128 ++++++++++---------- 1 file changed, 64 insertions(+), 64 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index c95ff5758f4..dd8031461e0 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -1092,7 +1092,7 @@ Types of sources (`source_type`): - [Local file](#local_file) - [Executable File](#executable) - [Executable Pool](#executable_pool) -- [HTTP(s)](#http) +- [HTTP(S)](#http) - DBMS - [ODBC](#odbc) - [MySQL](#mysql) @@ -1102,7 +1102,7 @@ Types of sources (`source_type`): - [Cassandra](#cassandra) - [PostgreSQL](#postgresql) -## Local File {#local_file} +### Local File {#local_file} Example of settings: @@ -1132,7 +1132,7 @@ When a dictionary with source `FILE` is created via DDL command (`CREATE DICTION - [Dictionary function](../../sql-reference/table-functions/dictionary.md#dictionary-function) -## Executable File {#executable} +### Executable File {#executable} Working with executable files depends on [how the dictionary is stored in memory](#storig-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request to the executable file’s STDIN. Otherwise, ClickHouse starts the executable file and treats its output as dictionary data. @@ -1161,7 +1161,7 @@ Setting fields: That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled; otherwise, the DB user would be able to execute arbitrary binaries on the ClickHouse node. -## Executable Pool {#executable_pool} +### Executable Pool {#executable_pool} Executable pool allows loading data from pool of processes. This source does not work with dictionary layouts that need to load all data from source. Executable pool works if the dictionary [is stored](#ways-to-store-dictionaries-in-memory) using `cache`, `complex_key_cache`, `ssd_cache`, `complex_key_ssd_cache`, `direct`, or `complex_key_direct` layouts. @@ -1196,9 +1196,9 @@ Setting fields: That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled, otherwise, the DB user would be able to execute arbitrary binary on ClickHouse node. -## Http(s) {#https} +### HTTP(S) {#https} -Working with an HTTP(s) server depends on [how the dictionary is stored in memory](#storig-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. +Working with an HTTP(S) server depends on [how the dictionary is stored in memory](#storig-dictionaries-in-memory). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. Example of settings: @@ -1248,7 +1248,55 @@ Setting fields: When creating a dictionary using the DDL command (`CREATE DICTIONARY ...`) remote hosts for HTTP dictionaries are checked against the contents of `remote_url_allow_hosts` section from config to prevent database users to access arbitrary HTTP server. -### Known Vulnerability of the ODBC Dictionary Functionality +### DBMS + +#### ODBC + +You can use this method to connect any database that has an ODBC driver. + +Example of settings: + +``` xml + + + DatabaseName + ShemaName.TableName
+ DSN=some_parameters + SQL_QUERY + SELECT id, value_1, value_2 FROM ShemaName.TableName +
+ +``` + +or + +``` sql +SOURCE(ODBC( + db 'DatabaseName' + table 'SchemaName.TableName' + connection_string 'DSN=some_parameters' + invalidate_query 'SQL_QUERY' + query 'SELECT id, value_1, value_2 FROM db_name.table_name' +)) +``` + +Setting fields: + +- `db` – Name of the database. Omit it if the database name is set in the `` parameters. +- `table` – Name of the table and schema if exists. +- `connection_string` – Connection string. +- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). +- `query` – The custom query. Optional parameter. + +:::note +The `table` and `query` fields cannot be used together. And either one of the `table` or `query` fields must be declared. +::: + +ClickHouse receives quoting symbols from ODBC-driver and quote all settings in queries to driver, so it’s necessary to set table name accordingly to table name case in database. + +If you have a problems with encodings when using Oracle, see the corresponding [FAQ](/knowledgebase/oracle-odbc) item. + +##### Known Vulnerability of the ODBC Dictionary Functionality :::note When connecting to the database through the ODBC driver connection parameter `Servername` can be substituted. In this case values of `USERNAME` and `PASSWORD` from `odbc.ini` are sent to the remote server and can be compromised. @@ -1277,7 +1325,7 @@ SELECT * FROM odbc('DSN=gregtest;Servername=some-server.com', 'test_db'); ODBC driver will send values of `USERNAME` and `PASSWORD` from `odbc.ini` to `some-server.com`. -### Example of Connecting Postgresql +##### Example of Connecting Postgresql Ubuntu OS. @@ -1358,7 +1406,7 @@ LIFETIME(MIN 300 MAX 360) You may need to edit `odbc.ini` to specify the full path to the library with the driver `DRIVER=/usr/local/lib/psqlodbcw.so`. -### Example of Connecting MS SQL Server +##### Example of Connecting MS SQL Server Ubuntu OS. @@ -1462,55 +1510,7 @@ LAYOUT(FLAT()) LIFETIME(MIN 300 MAX 360) ``` -## DBMS - -### ODBC - -You can use this method to connect any database that has an ODBC driver. - -Example of settings: - -``` xml - - - DatabaseName - ShemaName.TableName
- DSN=some_parameters - SQL_QUERY - SELECT id, value_1, value_2 FROM ShemaName.TableName -
- -``` - -or - -``` sql -SOURCE(ODBC( - db 'DatabaseName' - table 'SchemaName.TableName' - connection_string 'DSN=some_parameters' - invalidate_query 'SQL_QUERY' - query 'SELECT id, value_1, value_2 FROM db_name.table_name' -)) -``` - -Setting fields: - -- `db` – Name of the database. Omit it if the database name is set in the `` parameters. -- `table` – Name of the table and schema if exists. -- `connection_string` – Connection string. -- `invalidate_query` – Query for checking the dictionary status. Optional parameter. Read more in the section [Updating dictionaries](#dictionary-updates). -- `query` – The custom query. Optional parameter. - -:::note -The `table` and `query` fields cannot be used together. And either one of the `table` or `query` fields must be declared. -::: - -ClickHouse receives quoting symbols from ODBC-driver and quote all settings in queries to driver, so it’s necessary to set table name accordingly to table name case in database. - -If you have a problems with encodings when using Oracle, see the corresponding [FAQ](/knowledgebase/oracle-odbc) item. - -### Mysql +#### Mysql Example of settings: @@ -1627,7 +1627,7 @@ SOURCE(MYSQL( )) ``` -### ClickHouse +#### ClickHouse Example of settings: @@ -1680,7 +1680,7 @@ Setting fields: The `table` or `where` fields cannot be used together with the `query` field. And either one of the `table` or `query` fields must be declared. ::: -### Mongodb +#### Mongodb Example of settings: @@ -1723,7 +1723,7 @@ Setting fields: - `options` - MongoDB connection string options (optional parameter). -### Redis +#### Redis Example of settings: @@ -1756,7 +1756,7 @@ Setting fields: - `storage_type` – The structure of internal Redis storage using for work with keys. `simple` is for simple sources and for hashed single key sources, `hash_map` is for hashed sources with two keys. Ranged sources and cache sources with complex key are unsupported. May be omitted, default value is `simple`. - `db_index` – The specific numeric index of Redis logical database. May be omitted, default value is 0. -### Cassandra +#### Cassandra Example of settings: @@ -1798,7 +1798,7 @@ Setting fields: The `column_family` or `where` fields cannot be used together with the `query` field. And either one of the `column_family` or `query` fields must be declared. ::: -### PostgreSQL +#### PostgreSQL Example of settings: @@ -1855,7 +1855,7 @@ Setting fields: The `table` or `where` fields cannot be used together with the `query` field. And either one of the `table` or `query` fields must be declared. ::: -## Null +### Null A special source that can be used to create dummy (empty) dictionaries. Such dictionaries can useful for tests or with setups with separated data and query nodes at nodes with Distributed tables. From 6b37595e1164ef28e2541bd6f30f753b4b2ad10a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 16 Aug 2023 20:43:09 +0000 Subject: [PATCH 667/777] Update boost to 1.79 --- contrib/boost | 2 +- contrib/boost-cmake/CMakeLists.txt | 6 +++--- docs/en/sql-reference/data-types/geo.md | 4 ++-- .../0_stateless/01037_polygon_dicts_correctness_fast.sh | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/contrib/boost b/contrib/boost index aec12eea7fc..3a4a63b4a2b 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit aec12eea7fc762721ae16943d1361340c66c9c17 +Subproject commit 3a4a63b4a2b2612712ebe5c8869162166eb34360 diff --git a/contrib/boost-cmake/CMakeLists.txt b/contrib/boost-cmake/CMakeLists.txt index 6f9dce0b042..ef3a1758522 100644 --- a/contrib/boost-cmake/CMakeLists.txt +++ b/contrib/boost-cmake/CMakeLists.txt @@ -172,9 +172,9 @@ endif() # coroutine set (SRCS_COROUTINE - "${LIBRARY_DIR}/libs/coroutine/detail/coroutine_context.cpp" - "${LIBRARY_DIR}/libs/coroutine/exceptions.cpp" - "${LIBRARY_DIR}/libs/coroutine/posix/stack_traits.cpp" + "${LIBRARY_DIR}/libs/coroutine/src/detail/coroutine_context.cpp" + "${LIBRARY_DIR}/libs/coroutine/src/exceptions.cpp" + "${LIBRARY_DIR}/libs/coroutine/src/posix/stack_traits.cpp" ) add_library (_boost_coroutine ${SRCS_COROUTINE}) add_library (boost::coroutine ALIAS _boost_coroutine) diff --git a/docs/en/sql-reference/data-types/geo.md b/docs/en/sql-reference/data-types/geo.md index 3b2787008d2..1d37b829dd5 100644 --- a/docs/en/sql-reference/data-types/geo.md +++ b/docs/en/sql-reference/data-types/geo.md @@ -26,9 +26,9 @@ SELECT p, toTypeName(p) FROM geo_point; Result: ``` text -┌─p─────┬─toTypeName(p)─┐ +┌─p───────┬─toTypeName(p)─┐ │ (10,10) │ Point │ -└───────┴───────────────┘ +└─────────┴───────────────┘ ``` ## Ring diff --git a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh index f6880ae5009..fe10b6112c2 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-debug, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -8,6 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) TMP_DIR="/tmp" declare -a SearchTypes=("POLYGON_INDEX_EACH" "POLYGON_INDEX_CELL") +# declare -a SearchTypes=("POLYGON_INDEX_EACH") tar -xf "${CURDIR}"/01037_test_data_perf.tar.gz -C "${CURDIR}" From 557bfea4d812310363eb3785517d62af143523c5 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Thu, 17 Aug 2023 19:36:40 +0000 Subject: [PATCH 668/777] Use long timeout for S3 copy requests --- src/Backups/BackupIO_S3.cpp | 4 ++- .../ObjectStorages/S3/S3ObjectStorage.cpp | 12 ++++----- src/IO/S3/copyS3File.cpp | 26 ++++++++++++------- src/IO/S3/copyS3File.h | 7 +++++ 4 files changed, 33 insertions(+), 16 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index d32047efae4..7926d0b2564 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -159,6 +159,7 @@ void BackupReaderS3::copyFileToDisk(const String & path_in_backup, size_t file_s blob_path.size(), mode); copyS3File( + client, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, @@ -218,6 +219,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src { LOG_TRACE(log, "Copying file {} from disk {} to S3", src_path, src_disk->getName()); copyS3File( + client, client, /* src_bucket */ blob_path[1], /* src_key= */ blob_path[0], @@ -238,7 +240,7 @@ void BackupWriterS3::copyFileFromDisk(const String & path_in_backup, DiskPtr src void BackupWriterS3::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) { - copyDataToS3File(create_read_buffer, start_pos, length, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, request_settings, {}, + copyDataToS3File(create_read_buffer, start_pos, length, client, client, s3_uri.bucket, fs::path(s3_uri.key) / path_in_backup, request_settings, {}, threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWriterS3")); } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 4da7b3e892f..bbfc6609079 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -431,11 +431,11 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT /// Shortcut for S3 if (auto * dest_s3 = dynamic_cast(&object_storage_to); dest_s3 != nullptr) { - auto client_ptr = clients.get()->client; + auto clients_ = clients.get(); auto settings_ptr = s3_settings.get(); - auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + auto size = S3::getObjectSize(*clients_->client, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, + copyS3File(clients_->client, clients_->client_with_long_timeout, bucket, object_from.remote_path, 0, size, dest_s3->bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } else @@ -447,11 +447,11 @@ void S3ObjectStorage::copyObjectToAnotherObjectStorage( // NOLINT void S3ObjectStorage::copyObject( // NOLINT const StoredObject & object_from, const StoredObject & object_to, std::optional object_to_attributes) { - auto client_ptr = clients.get()->client; + auto clients_ = clients.get(); auto settings_ptr = s3_settings.get(); - auto size = S3::getObjectSize(*client_ptr, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); + auto size = S3::getObjectSize(*clients_->client, bucket, object_from.remote_path, {}, settings_ptr->request_settings, /* for_disk_s3= */ true); auto scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "S3ObjStor_copy"); - copyS3File(client_ptr, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, + copyS3File(clients_->client, clients_->client_with_long_timeout, bucket, object_from.remote_path, 0, size, bucket, object_to.remote_path, settings_ptr->request_settings, object_to_attributes, scheduler, /* for_disk_s3= */ true); } diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index dda107840cb..002b8dde566 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -53,6 +53,7 @@ namespace public: UploadHelper( const std::shared_ptr & client_ptr_, + const std::shared_ptr & client_with_long_timeout_ptr_, const String & dest_bucket_, const String & dest_key_, const S3Settings::RequestSettings & request_settings_, @@ -61,6 +62,7 @@ namespace bool for_disk_s3_, const Poco::Logger * log_) : client_ptr(client_ptr_) + , client_with_long_timeout_ptr(client_with_long_timeout_ptr_) , dest_bucket(dest_bucket_) , dest_key(dest_key_) , request_settings(request_settings_) @@ -76,6 +78,7 @@ namespace protected: std::shared_ptr client_ptr; + std::shared_ptr client_with_long_timeout_ptr; const String & dest_bucket; const String & dest_key; const S3Settings::RequestSettings & request_settings; @@ -176,7 +179,7 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3CompleteMultipartUpload); - auto outcome = client_ptr->CompleteMultipartUpload(request); + auto outcome = client_with_long_timeout_ptr->CompleteMultipartUpload(request); if (outcome.IsSuccess()) { @@ -430,13 +433,14 @@ namespace size_t offset_, size_t size_, const std::shared_ptr & client_ptr_, + const std::shared_ptr & client_with_long_timeout_ptr_, const String & dest_bucket_, const String & dest_key_, const S3Settings::RequestSettings & request_settings_, const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, bool for_disk_s3_) - : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyDataToS3File")) + : UploadHelper(client_ptr_, client_with_long_timeout_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyDataToS3File")) , create_read_buffer(create_read_buffer_) , offset(offset_) , size(size_) @@ -598,6 +602,7 @@ namespace public: CopyFileHelper( const std::shared_ptr & client_ptr_, + const std::shared_ptr & client_with_long_timeout_ptr_, const String & src_bucket_, const String & src_key_, size_t src_offset_, @@ -608,7 +613,7 @@ namespace const std::optional> & object_metadata_, ThreadPoolCallbackRunner schedule_, bool for_disk_s3_) - : UploadHelper(client_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyS3File")) + : UploadHelper(client_ptr_, client_with_long_timeout_ptr_, dest_bucket_, dest_key_, request_settings_, object_metadata_, schedule_, for_disk_s3_, &Poco::Logger::get("copyS3File")) , src_bucket(src_bucket_) , src_key(src_key_) , offset(src_offset_) @@ -669,7 +674,7 @@ namespace /// If we don't do it, AWS SDK can mistakenly set it to application/xml, see https://github.com/aws/aws-sdk-cpp/issues/1840 request.SetContentType("binary/octet-stream"); - client_ptr->setKMSHeaders(request); + client_with_long_timeout_ptr->setKMSHeaders(request); } void processCopyRequest(const S3::CopyObjectRequest & request) @@ -681,7 +686,7 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3CopyObject); - auto outcome = client_ptr->CopyObject(request); + auto outcome = client_with_long_timeout_ptr->CopyObject(request); if (outcome.IsSuccess()) { LOG_TRACE( @@ -706,6 +711,7 @@ namespace offset, size, client_ptr, + client_with_long_timeout_ptr, dest_bucket, dest_key, request_settings, @@ -779,7 +785,7 @@ namespace if (for_disk_s3) ProfileEvents::increment(ProfileEvents::DiskS3UploadPartCopy); - auto outcome = client_ptr->UploadPartCopy(req); + auto outcome = client_with_long_timeout_ptr->UploadPartCopy(req); if (!outcome.IsSuccess()) { abortMultipartUpload(); @@ -797,6 +803,7 @@ void copyDataToS3File( size_t offset, size_t size, const std::shared_ptr & dest_s3_client, + const std::shared_ptr & dest_s3_client_with_long_timeout, const String & dest_bucket, const String & dest_key, const S3Settings::RequestSettings & settings, @@ -804,13 +811,14 @@ void copyDataToS3File( ThreadPoolCallbackRunner schedule, bool for_disk_s3) { - CopyDataToFileHelper helper{create_read_buffer, offset, size, dest_s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; + CopyDataToFileHelper helper{create_read_buffer, offset, size, dest_s3_client, dest_s3_client_with_long_timeout, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; helper.performCopy(); } void copyS3File( const std::shared_ptr & s3_client, + const std::shared_ptr & s3_client_with_long_timeout, const String & src_bucket, const String & src_key, size_t src_offset, @@ -824,7 +832,7 @@ void copyS3File( { if (settings.allow_native_copy) { - CopyFileHelper helper{s3_client, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; + CopyFileHelper helper{s3_client, s3_client_with_long_timeout, src_bucket, src_key, src_offset, src_size, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3}; helper.performCopy(); } else @@ -833,7 +841,7 @@ void copyS3File( { return std::make_unique(s3_client, src_bucket, src_key, "", settings, Context::getGlobalContextInstance()->getReadSettings()); }; - copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); + copyDataToS3File(create_read_buffer, src_offset, src_size, s3_client, s3_client_with_long_timeout, dest_bucket, dest_key, settings, object_metadata, schedule, for_disk_s3); } } diff --git a/src/IO/S3/copyS3File.h b/src/IO/S3/copyS3File.h index 5d35e5ebe2d..3477f5a20ab 100644 --- a/src/IO/S3/copyS3File.h +++ b/src/IO/S3/copyS3File.h @@ -26,8 +26,14 @@ using CreateReadBuffer = std::function()>; /// has been disabled (with settings.allow_native_copy) or request failed /// because it is a known issue, it is fallbacks to read-write copy /// (copyDataToS3File()). +/// +/// s3_client_with_long_timeout (may be equal to s3_client) is used for native copy and +/// CompleteMultipartUpload requests. These requests need longer timeout because S3 servers often +/// block on them for multiple seconds without sending or receiving data from us (maybe the servers +/// are copying data internally, or maybe throttling, idk). void copyS3File( const std::shared_ptr & s3_client, + const std::shared_ptr & s3_client_with_long_timeout, const String & src_bucket, const String & src_key, size_t src_offset, @@ -49,6 +55,7 @@ void copyDataToS3File( size_t offset, size_t size, const std::shared_ptr & dest_s3_client, + const std::shared_ptr & dest_s3_client_with_long_timeout, const String & dest_bucket, const String & dest_key, const S3Settings::RequestSettings & settings, From 7838ea6896579123e588c3dc7182739ec5ecf70e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 17 Aug 2023 19:37:54 +0000 Subject: [PATCH 669/777] Another try with geometry rescaling re-enabled --- contrib/boost | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boost b/contrib/boost index 3a4a63b4a2b..f864050c9df 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit 3a4a63b4a2b2612712ebe5c8869162166eb34360 +Subproject commit f864050c9df38af268ec22c769fcadfed4959e15 From 9252edf82084a33b6c8d2172d62586c8f93d6123 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 17 Aug 2023 19:42:15 +0000 Subject: [PATCH 670/777] Dis-disfigure test --- .../queries/0_stateless/01037_polygon_dicts_correctness_fast.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh index fe10b6112c2..f6880ae5009 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-debug, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -7,7 +8,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) TMP_DIR="/tmp" declare -a SearchTypes=("POLYGON_INDEX_EACH" "POLYGON_INDEX_CELL") -# declare -a SearchTypes=("POLYGON_INDEX_EACH") tar -xf "${CURDIR}"/01037_test_data_perf.tar.gz -C "${CURDIR}" From 9c9941ca17cfa69d8cba1a596fc3006af56eb0cb Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 17 Aug 2023 22:58:23 +0300 Subject: [PATCH 671/777] Update test.py --- tests/integration/test_grpc_protocol/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 42d6d2bfda3..efc7d98e820 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -352,13 +352,13 @@ def test_authentication(): def test_logs(): - query = "SELECT has(groupArray(number), 42) FROM numbers(100000) SETTINGS max_block_size=10000" + query = "SELECT has(groupArray(number), 42) FROM numbers(1000000) SETTINGS max_block_size=100000" logs = query_and_get_logs( query, settings={"send_logs_level": "debug"}, ) assert query in logs - assert "Read 100000 rows" in logs + assert "Read 1000000 rows" in logs assert "Peak memory usage" in logs From 0c81546960a2f95e5dbcad211df3eb432af749ea Mon Sep 17 00:00:00 2001 From: Jianfei Hu Date: Thu, 17 Aug 2023 20:13:33 +0000 Subject: [PATCH 672/777] remove the comments. Signed-off-by: Jianfei Hu --- src/Storages/MergeTree/MergeTreePartition.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 72017e823b0..f153405744d 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -371,7 +371,7 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe const auto & partition_key_sample = metadata_snapshot->getPartitionKey().sample_block; size_t key_size = partition_key_sample.columns(); - // In some cases we create empty parts and value is empty. + // In some cases we create empty parts and then value is empty. if (value.empty()) { writeCString("tuple()", out); @@ -385,7 +385,6 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe { const DataTypePtr & type = partition_key_sample.getByPosition(0).type; auto column = type->createColumn(); - const auto & all_column_names = metadata_snapshot->getColumns().getAll(); column->insert(value[0]); type->getDefaultSerialization()->serializeText(*column, 0, out, format_settings); } From 9f771b10e96088891d0a6cd4be4ca9419057e123 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 17 Aug 2023 20:42:22 +0000 Subject: [PATCH 673/777] Cleanup --- src/Interpreters/ClusterProxy/executeQuery.cpp | 4 ++-- src/Processors/QueryPlan/ReadFromRemote.cpp | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 9e91febddb3..ccbce0b3dd4 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -302,8 +302,8 @@ void executeQueryWithParallelReplicas( chassert(shard_count == not_optimized_cluster->getShardsAddresses().size()); - LOG_DEBUG( - &Poco::Logger::get("executeQueryWithParallelReplicas"), "Parallel replicas query in shard scope: shard_num={}", shard_num); + LOG_DEBUG(&Poco::Logger::get("executeQueryWithParallelReplicas"), "Parallel replicas query in shard scope: shard_num={} cluster={}", + shard_num, not_optimized_cluster->getName()); const auto shard_replicas_num = not_optimized_cluster->getShardsAddresses()[shard_num - 1].size(); all_replicas_count = std::min(static_cast(settings.max_parallel_replicas), shard_replicas_num); diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 2ec6c82898f..1e2f5adf2b3 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include #include From f0fe3e10464fb3d3699834e304c23d954534c41a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 17 Aug 2023 20:59:37 +0000 Subject: [PATCH 674/777] Exclude risc-v (gives compile errors in FP16) --- contrib/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 2557ebf78ae..e75a7900f7d 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -198,7 +198,7 @@ endif() add_contrib (annoy-cmake annoy) option(ENABLE_USEARCH "Enable USearch (Approximate Neighborhood Search, HNSW) support" ${ENABLE_LIBRARIES}) -if (ENABLE_USEARCH) +if (ENABLE_USEARCH AND NOT ARCH_RISCV64) add_contrib (FP16-cmake FP16) add_contrib (robin-map-cmake robin-map) add_contrib (SimSIMD-cmake SimSIMD) From abcf01b809708e92437973bd583bd43d3b080054 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 17 Aug 2023 23:51:58 +0200 Subject: [PATCH 675/777] impl --- tests/integration/helpers/client.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/client.py b/tests/integration/helpers/client.py index fdeedb9a80d..c5f0584f828 100644 --- a/tests/integration/helpers/client.py +++ b/tests/integration/helpers/client.py @@ -182,7 +182,8 @@ class CommandRequest: # we suppress stderror on client becase sometimes thread sanitizer # can print some debug information there env = {} - env["TSAN_OPTIONS"] = "verbosity=0" + env["ASAN_OPTIONS"] = "use_sigaltstack=0" + env["TSAN_OPTIONS"] = "use_sigaltstack=0 verbosity=0" self.process = sp.Popen( command, stdin=stdin_file, From 2e69a1d3a21e2a8572917a1e251cb72e17c2ccb0 Mon Sep 17 00:00:00 2001 From: pufit Date: Thu, 17 Aug 2023 22:07:39 -0400 Subject: [PATCH 676/777] Fix keeper default path check --- programs/keeper/Keeper.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 4791f42312e..22f0b2c2ac6 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -290,12 +290,6 @@ try { path = config().getString("keeper_server.storage_path"); } - else if (std::filesystem::is_directory(std::filesystem::path{config().getString("path", DBMS_DEFAULT_PATH)} / "coordination")) - { - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, - "By default 'keeper.storage_path' could be assigned to {}, but the directory {} already exists. Please specify 'keeper.storage_path' in the keeper configuration explicitly", - KEEPER_DEFAULT_PATH, String{std::filesystem::path{config().getString("path", DBMS_DEFAULT_PATH)} / "coordination"}); - } else if (config().has("keeper_server.log_storage_path")) { path = std::filesystem::path(config().getString("keeper_server.log_storage_path")).parent_path(); @@ -304,6 +298,12 @@ try { path = std::filesystem::path(config().getString("keeper_server.snapshot_storage_path")).parent_path(); } + else if (std::filesystem::is_directory(std::filesystem::path{config().getString("path", DBMS_DEFAULT_PATH)} / "coordination")) + { + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, + "By default 'keeper.storage_path' could be assigned to {}, but the directory {} already exists. Please specify 'keeper.storage_path' in the keeper configuration explicitly", + KEEPER_DEFAULT_PATH, String{std::filesystem::path{config().getString("path", DBMS_DEFAULT_PATH)} / "coordination"}); + } else { path = KEEPER_DEFAULT_PATH; From 8e0d5b7ee04183eb612f7d682286bf392c140ecf Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 18 Aug 2023 13:31:26 +0800 Subject: [PATCH 677/777] fix bugs --- src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index bc8b508f083..793bbf90d24 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -71,7 +71,7 @@ std::unique_ptr asORCInputStream(ReadBuffer & in, const Format bool has_file_size = isBufferWithFileSize(in); auto * seekable_in = dynamic_cast(&in); - if (has_file_size && seekable_in && settings.seekable_read) + if (has_file_size && seekable_in && settings.seekable_read && seekable_in->checkIfActuallySeekable()) return std::make_unique(*seekable_in, getFileSizeFromReadBuffer(in)); // fallback to loading the entire file in memory From 15720d9cefe4b2144f257b715edf45090c90c85a Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 18 Aug 2023 15:10:25 +0800 Subject: [PATCH 678/777] fix ut tests/queries/0_stateless/02518_parquet_arrow_orc_boolean_value.sh --- .../Impl/NativeORCBlockInputFormat.cpp | 33 ++++++++++--------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 793bbf90d24..b05418bbf35 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -226,9 +226,6 @@ void ORCBlockInputFormat::prepareFileReader() if (getPort().getHeader().has(name, ignore_case) || nested_table_names.contains(ignore_case ? boost::to_lower_copy(name) : name)) include_indices.push_back(static_cast(i)); } - - // std::cout << "schema:" << schema.dumpStructure() << std::endl; - // std::cout << "header:" << getPort().getHeader().dumpStructure() << std::endl; } bool ORCBlockInputFormat::prepareStripeReader() @@ -243,7 +240,6 @@ bool ORCBlockInputFormat::prepareStripeReader() if (current_stripe >= total_stripes) return false; - /// Seek to current stripe current_stripe_info = file_reader->getStripe(current_stripe); if (!current_stripe_info->getNumberOfRows()) return false; @@ -401,6 +397,21 @@ static ColumnPtr readOffsetsFromORCListColumn(const BatchType * orc_column) return offsets_column; } +static ColumnWithTypeAndName +readColumnWithBooleanData(const orc::ColumnVectorBatch * orc_column, const orc::Type *, const String & column_name) +{ + const auto * orc_bool_column = dynamic_cast(orc_column); + auto internal_type = DataTypeFactory::instance().get("Bool"); + auto internal_column = internal_type->createColumn(); + auto & column_data = assert_cast &>(*internal_column).getData(); + column_data.reserve(orc_bool_column->numElements); + + for (size_t i = 0; i < orc_bool_column->numElements; ++i) + column_data.push_back(static_cast(orc_bool_column->data[i])); + + return {std::move(internal_column), internal_type, column_name}; +} + /// Inserts numeric data right into internal column data to reduce an overhead template > static ColumnWithTypeAndName @@ -417,7 +428,6 @@ readColumnWithNumericData(const orc::ColumnVectorBatch * orc_column, const orc:: return {std::move(internal_column), std::move(internal_type), column_name}; } -/// Inserts numeric data right into internal column data to reduce an overhead template > static ColumnWithTypeAndName readColumnWithNumericDataCast(const orc::ColumnVectorBatch * orc_column, const orc::Type *, const String & column_name) @@ -434,7 +444,6 @@ readColumnWithNumericDataCast(const orc::ColumnVectorBatch * orc_column, const o return {std::move(internal_column), std::move(internal_type), column_name}; } -/// Inserts chars and offsets right into internal column data to reduce an overhead. static ColumnWithTypeAndName readColumnWithStringData(const orc::ColumnVectorBatch * orc_column, const orc::Type *, const String & column_name) { @@ -491,7 +500,6 @@ readColumnWithFixedStringData(const orc::ColumnVectorBatch * orc_column, const o } -/// Inserts decimal data right into internal column data to reduce an overhead template > static ColumnWithTypeAndName readColumnWithDecimalDataCast( const orc::ColumnVectorBatch * orc_column, const orc::Type *, const String & column_name, DataTypePtr internal_type) @@ -518,16 +526,11 @@ static ColumnWithTypeAndName readColumnWithDecimalDataCast( decimal_value.value = static_cast(orc_decimal_column->values[i]); column_data.push_back(std::move(decimal_value)); - // std::cout << "i:" << i << "size:" << column_data.size() << "size2:" << internal_column->size() - // << ", value:" << static_cast(column_data.back().value) << std::endl; } - // std::cout << "orc rows:" << orc_decimal_column->numElements << std::endl; - // std::cout << "ch rows:" << internal_column->size() << std::endl; return {std::move(internal_column), internal_type, column_name}; } - static ColumnWithTypeAndName readIPv6ColumnFromBinaryData(const orc::ColumnVectorBatch * orc_column, const orc::Type * orc_type, const String & column_name) { @@ -556,7 +559,6 @@ readIPv6ColumnFromBinaryData(const orc::ColumnVectorBatch * orc_column, const or return {std::move(internal_column), std::move(internal_type), column_name}; } - static ColumnWithTypeAndName readIPv4ColumnWithInt32Data(const orc::ColumnVectorBatch * orc_column, const orc::Type *, const String & column_name) { @@ -593,7 +595,6 @@ static ColumnWithTypeAndName readColumnWithBigNumberFromBinaryData( return {std::move(internal_column), column_type, column_name}; } - static ColumnWithTypeAndName readColumnWithDateData( const orc::ColumnVectorBatch * orc_column, const orc::Type *, const String & column_name, const DataTypePtr & type_hint) { @@ -651,7 +652,6 @@ readColumnWithTimestampData(const orc::ColumnVectorBatch * orc_column, const orc return {std::move(internal_column), std::move(internal_type), column_name}; } - static ColumnWithTypeAndName readColumnFromORCColumn( const orc::ColumnVectorBatch * orc_column, const orc::Type * orc_type, @@ -727,6 +727,8 @@ static ColumnWithTypeAndName readColumnFromORCColumn( } return readColumnWithFixedStringData(orc_column, orc_type, column_name); } + case orc::BOOLEAN: + return readColumnWithBooleanData(orc_column, orc_type, column_name); case orc::BYTE: return readColumnWithNumericDataCast(orc_column, orc_type, column_name); case orc::SHORT: @@ -1056,7 +1058,6 @@ void ORCColumnToCHColumn::orcColumnsToCHChunk( res.setColumns(columns_list, num_rows); } - void registerInputFormatORC(FormatFactory & factory) { factory.registerInputFormat( From 476f3cedc1d44c24253fd0c62eff33036338d936 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 14 Nov 2022 16:59:53 +0800 Subject: [PATCH 679/777] Various reactions when executable stderr has data --- src/Client/ConnectionEstablisher.cpp | 2 +- src/Client/HedgedConnections.cpp | 2 +- src/Common/Epoll.cpp | 27 +++- src/Common/Epoll.h | 9 +- src/Common/ShellCommandSettings.cpp | 23 +++ src/Common/ShellCommandSettings.h | 17 +++ src/Core/SettingsEnums.cpp | 5 + src/Core/SettingsEnums.h | 3 + .../ExecutableDictionarySource.cpp | 1 + .../ExecutablePoolDictionarySource.cpp | 1 + ...alUserDefinedExecutableFunctionsLoader.cpp | 3 + src/Processors/Executors/PollingQueue.cpp | 2 +- src/Processors/Sources/ShellCommandSource.cpp | 138 ++++++++++-------- src/Processors/Sources/ShellCommandSource.h | 4 + .../RemoteQueryExecutorReadContext.cpp | 2 +- src/Storages/ExecutableSettings.h | 3 +- src/Storages/StorageExecutable.cpp | 1 + .../functions/test_function_config.xml | 23 +++ .../test.py | 21 +++ .../user_scripts/input_always_error.py | 10 ++ 20 files changed, 225 insertions(+), 72 deletions(-) create mode 100644 src/Common/ShellCommandSettings.cpp create mode 100644 src/Common/ShellCommandSettings.h create mode 100755 tests/integration/test_executable_user_defined_function/user_scripts/input_always_error.py diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 439025447ca..9805838a311 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -170,7 +170,7 @@ bool ConnectionEstablisherAsync::checkTimeout() epoll_event events[2]; events[0].data.fd = events[1].data.fd = -1; - size_t ready_count = epoll.getManyReady(2, events, false); + size_t ready_count = epoll.getManyReady(2, events, 0); for (size_t i = 0; i != ready_count; ++i) { if (events[i].data.fd == socket_fd) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 0efad1188fa..7d723d02347 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -388,7 +388,7 @@ int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) bool blocking = !static_cast(async_callback); while (events_count == 0) { - events_count = epoll.getManyReady(1, &event, blocking); + events_count = epoll.getManyReady(1, &event, blocking ? -1 : 0); if (!events_count && async_callback) async_callback(epoll.getFileDescriptor(), 0, AsyncEventTimeoutType::NONE, epoll.getDescription(), AsyncTaskExecutor::Event::READ | AsyncTaskExecutor::Event::ERROR); } diff --git a/src/Common/Epoll.cpp b/src/Common/Epoll.cpp index 182981aca27..ac06f044beb 100644 --- a/src/Common/Epoll.cpp +++ b/src/Common/Epoll.cpp @@ -2,6 +2,7 @@ #include "Epoll.h" #include +#include #include #include @@ -57,21 +58,35 @@ void Epoll::remove(int fd) throwFromErrno("Cannot remove descriptor from epoll", DB::ErrorCodes::EPOLL_ERROR); } -size_t Epoll::getManyReady(int max_events, epoll_event * events_out, bool blocking) const +size_t Epoll::getManyReady(int max_events, epoll_event * events_out, int timeout) const { if (events_count == 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "There are no events in epoll"); + Stopwatch watch; int ready_size; - int timeout = blocking ? -1 : 0; - do + while (true) { ready_size = epoll_wait(epoll_fd, events_out, max_events, timeout); - if (ready_size == -1 && errno != EINTR) - throwFromErrno("Error in epoll_wait", DB::ErrorCodes::EPOLL_ERROR); + /// If `ready_size` = 0, it's timeout. + if (ready_size < 0) + { + if (errno == EINTR) + { + if (timeout >= 0) + { + timeout = std::max(0, static_cast(timeout - watch.elapsedMilliseconds())); + watch.restart(); + } + continue; + } + else + throwFromErrno("Error in epoll_wait", DB::ErrorCodes::EPOLL_ERROR); + } + else + break; } - while (ready_size <= 0 && (ready_size != 0 || blocking)); return ready_size; } diff --git a/src/Common/Epoll.h b/src/Common/Epoll.h index ac42a8fc56d..5eadd5a7a65 100644 --- a/src/Common/Epoll.h +++ b/src/Common/Epoll.h @@ -30,10 +30,11 @@ public: /// Remove file descriptor to epoll. void remove(int fd); - /// Get events from epoll. Events are written in events_out, this function returns an amount of ready events. - /// If blocking is false and there are no ready events, - /// return empty vector, otherwise wait for ready events. - size_t getManyReady(int max_events, epoll_event * events_out, bool blocking) const; + /// Get events from epoll. Events are written in events_out, this function returns an amount of + /// ready events. The timeout argument specifies the number of milliseconds to wait for ready + /// events. Timeout of -1 causes epoll_wait() to block indefinitely, while specifying a timeout + /// equal to zero will return immediately, even if no events are available. + size_t getManyReady(int max_events, epoll_event * events_out, int timeout) const; int getFileDescriptor() const { return epoll_fd; } diff --git a/src/Common/ShellCommandSettings.cpp b/src/Common/ShellCommandSettings.cpp new file mode 100644 index 00000000000..4b6d062c739 --- /dev/null +++ b/src/Common/ShellCommandSettings.cpp @@ -0,0 +1,23 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +ExternalCommandStderrReaction parseExternalCommandStderrReaction(const std::string & config) +{ + auto reaction = magic_enum::enum_cast(Poco::toUpper(config)); + if (!reaction) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown stderr reaction: {}. Possible values are 'none', 'log' and 'throw'", config); + + return *reaction; +} + +} diff --git a/src/Common/ShellCommandSettings.h b/src/Common/ShellCommandSettings.h new file mode 100644 index 00000000000..ea72f38d332 --- /dev/null +++ b/src/Common/ShellCommandSettings.h @@ -0,0 +1,17 @@ +#pragma once + +#include + +namespace DB +{ + +enum class ExternalCommandStderrReaction +{ + NONE, /// Do nothing. + LOG, /// Try to log all outputs of stderr from the external command. + THROW /// Throw exception when the external command outputs something to its stderr. +}; + +ExternalCommandStderrReaction parseExternalCommandStderrReaction(const std::string & config); + +} diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index ec66c5faad3..0e0f09c5ee4 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -181,4 +181,9 @@ IMPLEMENT_SETTING_ENUM(S3QueueMode, ErrorCodes::BAD_ARGUMENTS, IMPLEMENT_SETTING_ENUM(S3QueueAction, ErrorCodes::BAD_ARGUMENTS, {{"keep", S3QueueAction::KEEP}, {"delete", S3QueueAction::DELETE}}) + +IMPLEMENT_SETTING_ENUM(ExternalCommandStderrReaction, ErrorCodes::BAD_ARGUMENTS, + {{"none", ExternalCommandStderrReaction::NONE}, + {"log", ExternalCommandStderrReaction::LOG}, + {"throw", ExternalCommandStderrReaction::THROW}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index da0163f8b6e..034e4c8c887 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -238,4 +239,6 @@ enum class S3QueueAction DECLARE_SETTING_ENUM(S3QueueAction) +DECLARE_SETTING_ENUM(ExternalCommandStderrReaction) + } diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index e40ef07de9e..3f513de6b36 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -275,6 +275,7 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) .command_termination_timeout_seconds = config.getUInt64(settings_config_prefix + ".command_termination_timeout", 10), .command_read_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_read_timeout", 10000), .command_write_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_write_timeout", 10000), + .stderr_reaction = parseExternalCommandStderrReaction(config.getString(settings_config_prefix + ".stderr_reaction", "none")), .is_executable_pool = false, .send_chunk_header = config.getBool(settings_config_prefix + ".send_chunk_header", false), .execute_direct = config.getBool(settings_config_prefix + ".execute_direct", false) diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 94685060a46..bd745112d21 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -233,6 +233,7 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) .command_termination_timeout_seconds = config.getUInt64(settings_config_prefix + ".command_termination_timeout", 10), .command_read_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_read_timeout", 10000), .command_write_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_write_timeout", 10000), + .stderr_reaction = parseExternalCommandStderrReaction(config.getString(settings_config_prefix + ".stderr_reaction", "none")), .pool_size = config.getUInt64(settings_config_prefix + ".pool_size", 16), .max_command_execution_time_seconds = max_command_execution_time, .is_executable_pool = true, diff --git a/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp b/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp index d4ecbf66987..249e301ec67 100644 --- a/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp +++ b/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp @@ -171,6 +171,8 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create size_t command_termination_timeout_seconds = config.getUInt64(key_in_config + ".command_termination_timeout", 10); size_t command_read_timeout_milliseconds = config.getUInt64(key_in_config + ".command_read_timeout", 10000); size_t command_write_timeout_milliseconds = config.getUInt64(key_in_config + ".command_write_timeout", 10000); + ExternalCommandStderrReaction stderr_reaction + = parseExternalCommandStderrReaction(config.getString(key_in_config + ".stderr_reaction", "none")); size_t pool_size = 0; size_t max_command_execution_time = 0; @@ -238,6 +240,7 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create .command_termination_timeout_seconds = command_termination_timeout_seconds, .command_read_timeout_milliseconds = command_read_timeout_milliseconds, .command_write_timeout_milliseconds = command_write_timeout_milliseconds, + .stderr_reaction = stderr_reaction, .pool_size = pool_size, .max_command_execution_time_seconds = max_command_execution_time, .is_executable_pool = is_executable_pool, diff --git a/src/Processors/Executors/PollingQueue.cpp b/src/Processors/Executors/PollingQueue.cpp index 781a7736336..40f968621b1 100644 --- a/src/Processors/Executors/PollingQueue.cpp +++ b/src/Processors/Executors/PollingQueue.cpp @@ -74,7 +74,7 @@ PollingQueue::TaskData PollingQueue::wait(std::unique_lock & lock) epoll_event event; event.data.ptr = nullptr; - epoll.getManyReady(1, &event, true); + epoll.getManyReady(1, &event, -1); lock.lock(); diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index ace175f251c..67c2d848908 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -1,8 +1,8 @@ #include -#include - +#include #include +#include #include #include @@ -21,10 +21,9 @@ namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; extern const int TIMEOUT_EXCEEDED; - extern const int CANNOT_FCNTL; extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR; - extern const int CANNOT_POLL; extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR; + extern const int CANNOT_FCNTL; } static bool tryMakeFdNonBlocking(int fd) @@ -64,68 +63,76 @@ static void makeFdBlocking(int fd) throwFromErrno("Cannot set blocking mode of pipe", ErrorCodes::CANNOT_FCNTL); } -static bool pollFd(int fd, size_t timeout_milliseconds, int events) -{ - pollfd pfd; - pfd.fd = fd; - pfd.events = events; - pfd.revents = 0; - - int res; - - while (true) - { - Stopwatch watch; - res = poll(&pfd, 1, static_cast(timeout_milliseconds)); - - if (res < 0) - { - if (errno != EINTR) - throwFromErrno("Cannot poll", ErrorCodes::CANNOT_POLL); - - const auto elapsed = watch.elapsedMilliseconds(); - if (timeout_milliseconds <= elapsed) - break; - timeout_milliseconds -= elapsed; - } - else - { - break; - } - } - - return res > 0; -} - class TimeoutReadBufferFromFileDescriptor : public BufferWithOwnMemory { public: - explicit TimeoutReadBufferFromFileDescriptor(int fd_, size_t timeout_milliseconds_) - : fd(fd_) + explicit TimeoutReadBufferFromFileDescriptor( + int stdout_fd_, int stderr_fd_, size_t timeout_milliseconds_, ExternalCommandStderrReaction stderr_reaction_) + : stdout_fd(stdout_fd_) + , stderr_fd(stderr_fd_) , timeout_milliseconds(timeout_milliseconds_) + , stderr_reaction(stderr_reaction_) { - makeFdNonBlocking(fd); + makeFdNonBlocking(stdout_fd); + makeFdNonBlocking(stderr_fd); + + epoll.add(stdout_fd); + if (stderr_reaction != ExternalCommandStderrReaction::NONE) + epoll.add(stderr_fd); } bool nextImpl() override { + static constexpr size_t STDERR_BUFFER_SIZE = 16_KiB; size_t bytes_read = 0; while (!bytes_read) { - if (!pollFd(fd, timeout_milliseconds, POLLIN)) + epoll_event events[2]; + events[0].data.fd = events[1].data.fd = -1; + size_t num_events = epoll.getManyReady(2, events, static_cast(timeout_milliseconds)); + if (0 == num_events) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Pipe read timeout exceeded {} milliseconds", timeout_milliseconds); - ssize_t res = ::read(fd, internal_buffer.begin(), internal_buffer.size()); + bool has_stdout = false; + bool has_stderr = false; + for (size_t i = 0; i < num_events; ++i) + { + if (events[i].data.fd == stdout_fd) + has_stdout = true; + else if (events[i].data.fd == stderr_fd) + has_stderr = true; + } - if (-1 == res && errno != EINTR) - throwFromErrno("Cannot read from pipe", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR); + if (has_stderr) + { + stderr_buf.resize(STDERR_BUFFER_SIZE); + ssize_t res = ::read(stderr_fd, stderr_buf.data(), stderr_buf.size()); - if (res == 0) - break; + if (res > 0) + { + stderr_buf.resize(res); + if (stderr_reaction == ExternalCommandStderrReaction::THROW) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Executable generates stderr: {}", stderr_buf); + else if (stderr_reaction == ExternalCommandStderrReaction::LOG) + LOG_WARNING( + &::Poco::Logger::get("TimeoutReadBufferFromFileDescriptor"), "Executable generates stderr: {}", stderr_buf); + } + } - if (res > 0) - bytes_read += res; + if (has_stdout) + { + ssize_t res = ::read(stdout_fd, internal_buffer.begin(), internal_buffer.size()); + + if (-1 == res && errno != EINTR) + throwFromErrno("Cannot read from pipe", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR); + + if (res == 0) + break; + + if (res > 0) + bytes_read += res; + } } if (bytes_read > 0) @@ -143,27 +150,33 @@ public: void reset() const { - makeFdBlocking(fd); + makeFdBlocking(stdout_fd); + makeFdBlocking(stderr_fd); } ~TimeoutReadBufferFromFileDescriptor() override { - tryMakeFdBlocking(fd); + tryMakeFdBlocking(stdout_fd); + tryMakeFdBlocking(stderr_fd); } private: - int fd; + int stdout_fd; + int stderr_fd; size_t timeout_milliseconds; + ExternalCommandStderrReaction stderr_reaction; + Epoll epoll; + String stderr_buf; }; class TimeoutWriteBufferFromFileDescriptor : public BufferWithOwnMemory { public: explicit TimeoutWriteBufferFromFileDescriptor(int fd_, size_t timeout_milliseconds_) - : fd(fd_) - , timeout_milliseconds(timeout_milliseconds_) + : fd(fd_), timeout_milliseconds(timeout_milliseconds_) { makeFdNonBlocking(fd); + epoll.add(fd, nullptr, EPOLLOUT); } void nextImpl() override @@ -175,10 +188,13 @@ public: while (bytes_written != offset()) { - if (!pollFd(fd, timeout_milliseconds, POLLOUT)) + epoll_event events[1]; + events[0].data.fd = -1; + size_t num_events = epoll.getManyReady(1, events, static_cast(timeout_milliseconds)); + if (0 == num_events) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Pipe write timeout exceeded {} milliseconds", timeout_milliseconds); - ssize_t res = ::write(fd, working_buffer.begin() + bytes_written, offset() - bytes_written); + ssize_t res = ::write(events[0].data.fd, working_buffer.begin() + bytes_written, offset() - bytes_written); if ((-1 == res || 0 == res) && errno != EINTR) throwFromErrno("Cannot write into pipe", ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR); @@ -201,6 +217,7 @@ public: private: int fd; size_t timeout_milliseconds; + Epoll epoll; }; class ShellCommandHolder @@ -248,6 +265,7 @@ namespace ContextPtr context_, const std::string & format_, size_t command_read_timeout_milliseconds, + ExternalCommandStderrReaction stderr_reaction, const Block & sample_block_, std::unique_ptr && command_, std::vector && send_data_tasks = {}, @@ -260,7 +278,11 @@ namespace , sample_block(sample_block_) , command(std::move(command_)) , configuration(configuration_) - , timeout_command_out(command->out.getFD(), command_read_timeout_milliseconds) + , timeout_command_out( + command->out.getFD(), + command->err.getFD(), + command_read_timeout_milliseconds, + stderr_reaction) , command_holder(std::move(command_holder_)) , process_pool(process_pool_) { @@ -534,7 +556,8 @@ Pipe ShellCommandSourceCoordinator::createPipe( } int write_buffer_fd = write_buffer->getFD(); - auto timeout_write_buffer = std::make_shared(write_buffer_fd, configuration.command_write_timeout_milliseconds); + auto timeout_write_buffer + = std::make_shared(write_buffer_fd, configuration.command_write_timeout_milliseconds); input_pipes[i].resize(1); @@ -570,6 +593,7 @@ Pipe ShellCommandSourceCoordinator::createPipe( context, configuration.format, configuration.command_read_timeout_milliseconds, + configuration.stderr_reaction, std::move(sample_block), std::move(process), std::move(tasks), diff --git a/src/Processors/Sources/ShellCommandSource.h b/src/Processors/Sources/ShellCommandSource.h index 6dc6781cc4c..6b504aa47bd 100644 --- a/src/Processors/Sources/ShellCommandSource.h +++ b/src/Processors/Sources/ShellCommandSource.h @@ -4,6 +4,7 @@ #include +#include #include #include @@ -57,6 +58,9 @@ public: /// Timeout for writing data to command stdin size_t command_write_timeout_milliseconds = 10000; + /// Reaction when external command outputs data to its stderr. + ExternalCommandStderrReaction stderr_reaction = ExternalCommandStderrReaction::NONE; + /// Pool size valid only if executable_pool = true size_t pool_size = 16; diff --git a/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp b/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp index 06d4a8cef87..5e211bf036d 100644 --- a/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp +++ b/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp @@ -76,7 +76,7 @@ bool RemoteQueryExecutorReadContext::checkTimeout(bool blocking) epoll_event events[3]; events[0].data.fd = events[1].data.fd = events[2].data.fd = -1; - size_t num_events = epoll.getManyReady(3, events, blocking); + size_t num_events = epoll.getManyReady(3, events, blocking ? -1 : 0); bool is_socket_ready = false; diff --git a/src/Storages/ExecutableSettings.h b/src/Storages/ExecutableSettings.h index 9374dac461e..cc30c77b682 100644 --- a/src/Storages/ExecutableSettings.h +++ b/src/Storages/ExecutableSettings.h @@ -14,7 +14,8 @@ class ASTStorage; M(UInt64, max_command_execution_time, 10, "Max command execution time in seconds.", 0) \ M(UInt64, command_termination_timeout, 10, "Command termination timeout in seconds.", 0) \ M(UInt64, command_read_timeout, 10000, "Timeout for reading data from command stdout in milliseconds.", 0) \ - M(UInt64, command_write_timeout, 10000, "Timeout for writing data to command stdin in milliseconds.", 0) + M(UInt64, command_write_timeout, 10000, "Timeout for writing data to command stdin in milliseconds.", 0) \ + M(ExternalCommandStderrReaction, stderr_reaction, ExternalCommandStderrReaction::NONE, "Reaction when external command outputs data to its stderr.", 0) DECLARE_SETTINGS_TRAITS(ExecutableSettingsTraits, LIST_OF_EXECUTABLE_SETTINGS) diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 9fabf1a9fb6..0ec9ea70e3c 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -92,6 +92,7 @@ StorageExecutable::StorageExecutable( .command_termination_timeout_seconds = settings.command_termination_timeout, .command_read_timeout_milliseconds = settings.command_read_timeout, .command_write_timeout_milliseconds = settings.command_write_timeout, + .stderr_reaction = settings.stderr_reaction, .pool_size = settings.pool_size, .max_command_execution_time_seconds = settings.max_command_execution_time, diff --git a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml index d35cb173dd0..c79094c2a05 100644 --- a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml +++ b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml @@ -322,4 +322,27 @@ input_parameter.py {test_parameter:UInt64} + + executable + test_function_always_error_throw_python + String + + UInt64 + + TabSeparated + input_always_error.py + throw + + + + executable + test_function_always_error_log_python + String + + UInt64 + + TabSeparated + input_always_error.py + log + diff --git a/tests/integration/test_executable_user_defined_function/test.py b/tests/integration/test_executable_user_defined_function/test.py index 1f4e14470c3..68d97cabc0a 100644 --- a/tests/integration/test_executable_user_defined_function/test.py +++ b/tests/integration/test_executable_user_defined_function/test.py @@ -1,6 +1,7 @@ import os import sys import time +import uuid import pytest @@ -285,3 +286,23 @@ def test_executable_function_parameter_python(started_cluster): node.query("SELECT test_function_parameter_python(2)(toUInt64(1))") == "Parameter 2 key 1\n" ) + + +def test_executable_function_always_error_python(started_cluster): + skip_test_msan(node) + try: + node.query("SELECT test_function_always_error_throw_python(1)") + assert False, "Exception have to be thrown" + except Exception as ex: + assert "DB::Exception: Executable generates stderr: Fake error" in str(ex) + + query_id = uuid.uuid4().hex + assert ( + node.query("SELECT test_function_always_error_log_python(1)", query_id=query_id) + == "Key 1\n" + ) + assert node.contains_in_log( + "{" + + query_id + + "} TimeoutReadBufferFromFileDescriptor: Executable generates stderr: Fake error" + ) diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/input_always_error.py b/tests/integration/test_executable_user_defined_function/user_scripts/input_always_error.py new file mode 100755 index 00000000000..f50c1235e3c --- /dev/null +++ b/tests/integration/test_executable_user_defined_function/user_scripts/input_always_error.py @@ -0,0 +1,10 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == "__main__": + print("Fake error", file=sys.stderr) + sys.stderr.flush() + for line in sys.stdin: + print("Key " + line, end="") + sys.stdout.flush() From 940ba60ae047a81181b27a9859feff80ca57ca30 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 25 Jul 2023 01:00:06 +0800 Subject: [PATCH 680/777] Fix non-linux build --- src/Processors/Sources/ShellCommandSource.cpp | 74 ++++++++++++++++--- 1 file changed, 65 insertions(+), 9 deletions(-) diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 67c2d848908..8a8ad9b0572 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -1,5 +1,7 @@ #include +#include + #include #include #include @@ -24,6 +26,7 @@ namespace ErrorCodes extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR; extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR; extern const int CANNOT_FCNTL; + extern const int CANNOT_POLL; } static bool tryMakeFdNonBlocking(int fd) @@ -63,6 +66,39 @@ static void makeFdBlocking(int fd) throwFromErrno("Cannot set blocking mode of pipe", ErrorCodes::CANNOT_FCNTL); } +static bool pollFd(int fd, size_t timeout_milliseconds, int events) +{ + pollfd pfd; + pfd.fd = fd; + pfd.events = events; + pfd.revents = 0; + + int res; + + while (true) + { + Stopwatch watch; + res = poll(&pfd, 1, static_cast(timeout_milliseconds)); + + if (res < 0) + { + if (errno != EINTR) + throwFromErrno("Cannot poll", ErrorCodes::CANNOT_POLL); + + const auto elapsed = watch.elapsedMilliseconds(); + if (timeout_milliseconds <= elapsed) + break; + timeout_milliseconds -= elapsed; + } + else + { + break; + } + } + + return res > 0; +} + class TimeoutReadBufferFromFileDescriptor : public BufferWithOwnMemory { public: @@ -76,16 +112,20 @@ public: makeFdNonBlocking(stdout_fd); makeFdNonBlocking(stderr_fd); +#if defined(OS_LINUX) epoll.add(stdout_fd); if (stderr_reaction != ExternalCommandStderrReaction::NONE) epoll.add(stderr_fd); +#endif } bool nextImpl() override { - static constexpr size_t STDERR_BUFFER_SIZE = 16_KiB; size_t bytes_read = 0; +#if defined(OS_LINUX) + static constexpr size_t STDERR_BUFFER_SIZE = 16_KiB; + while (!bytes_read) { epoll_event events[2]; @@ -134,6 +174,24 @@ public: bytes_read += res; } } +#else + while (!bytes_read) + { + if (!pollFd(stdout_fd, timeout_milliseconds, POLLIN)) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Pipe read timeout exceeded {} milliseconds", timeout_milliseconds); + + ssize_t res = ::read(stdout_fd, internal_buffer.begin(), internal_buffer.size()); + + if (-1 == res && errno != EINTR) + throwFromErrno("Cannot read from pipe", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR); + + if (res == 0) + break; + + if (res > 0) + bytes_read += res; + } +#endif if (bytes_read > 0) { @@ -164,9 +222,12 @@ private: int stdout_fd; int stderr_fd; size_t timeout_milliseconds; - ExternalCommandStderrReaction stderr_reaction; + [[maybe_unused]] ExternalCommandStderrReaction stderr_reaction; + +#if defined(OS_LINUX) Epoll epoll; String stderr_buf; +#endif }; class TimeoutWriteBufferFromFileDescriptor : public BufferWithOwnMemory @@ -176,7 +237,6 @@ public: : fd(fd_), timeout_milliseconds(timeout_milliseconds_) { makeFdNonBlocking(fd); - epoll.add(fd, nullptr, EPOLLOUT); } void nextImpl() override @@ -188,13 +248,10 @@ public: while (bytes_written != offset()) { - epoll_event events[1]; - events[0].data.fd = -1; - size_t num_events = epoll.getManyReady(1, events, static_cast(timeout_milliseconds)); - if (0 == num_events) + if (!pollFd(fd, timeout_milliseconds, POLLOUT)) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Pipe write timeout exceeded {} milliseconds", timeout_milliseconds); - ssize_t res = ::write(events[0].data.fd, working_buffer.begin() + bytes_written, offset() - bytes_written); + ssize_t res = ::write(fd, working_buffer.begin() + bytes_written, offset() - bytes_written); if ((-1 == res || 0 == res) && errno != EINTR) throwFromErrno("Cannot write into pipe", ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR); @@ -217,7 +274,6 @@ public: private: int fd; size_t timeout_milliseconds; - Epoll epoll; }; class ShellCommandHolder From dd0c71b32a18f63a7b1481bd77e805552e8d49aa Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 5 Aug 2023 23:13:16 +0800 Subject: [PATCH 681/777] Add error_exit_reaction --- src/Common/ShellCommandSettings.cpp | 17 ++++-- src/Common/ShellCommandSettings.h | 11 +++- src/Core/SettingsEnums.cpp | 5 ++ src/Core/SettingsEnums.h | 2 + .../ExecutableDictionarySource.cpp | 19 +++---- .../ExecutablePoolDictionarySource.cpp | 4 +- ...alUserDefinedExecutableFunctionsLoader.cpp | 3 ++ src/Processors/Sources/ShellCommandSource.cpp | 54 +++++++++++++++---- src/Processors/Sources/ShellCommandSource.h | 10 ++-- src/Storages/ExecutableSettings.h | 3 +- src/Storages/StorageExecutable.cpp | 1 + .../functions/test_function_config.xml | 24 +++++++++ .../test.py | 20 +++++++ .../user_scripts/input_log_error.py | 12 +++++ 14 files changed, 154 insertions(+), 31 deletions(-) create mode 100755 tests/integration/test_executable_user_defined_function/user_scripts/input_log_error.py diff --git a/src/Common/ShellCommandSettings.cpp b/src/Common/ShellCommandSettings.cpp index 4b6d062c739..b1bd8f280de 100644 --- a/src/Common/ShellCommandSettings.cpp +++ b/src/Common/ShellCommandSettings.cpp @@ -1,7 +1,8 @@ +#include + +#include #include #include -#include -#include namespace DB { @@ -15,7 +16,17 @@ ExternalCommandStderrReaction parseExternalCommandStderrReaction(const std::stri { auto reaction = magic_enum::enum_cast(Poco::toUpper(config)); if (!reaction) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown stderr reaction: {}. Possible values are 'none', 'log' and 'throw'", config); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown stderr_reaction: {}. Possible values are 'none', 'log' and 'throw'", config); + + return *reaction; +} + +ExternalCommandErrorExitReaction parseExternalCommandErrorExitReaction(const std::string & config) +{ + auto reaction = magic_enum::enum_cast(Poco::toUpper(config)); + if (!reaction) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Unknown error_exit_reaction: {}. Possible values are 'none', 'log_first' and 'log_last'", config); return *reaction; } diff --git a/src/Common/ShellCommandSettings.h b/src/Common/ShellCommandSettings.h index ea72f38d332..40625a6bbf2 100644 --- a/src/Common/ShellCommandSettings.h +++ b/src/Common/ShellCommandSettings.h @@ -8,10 +8,19 @@ namespace DB enum class ExternalCommandStderrReaction { NONE, /// Do nothing. - LOG, /// Try to log all outputs of stderr from the external command. + LOG, /// Try to log all outputs of stderr from the external command. THROW /// Throw exception when the external command outputs something to its stderr. }; ExternalCommandStderrReaction parseExternalCommandStderrReaction(const std::string & config); +enum class ExternalCommandErrorExitReaction +{ + NONE, /// Do nothing. + LOG_FIRST, /// Try to log first 1_KiB outputs of stderr from the external command. + LOG_LAST /// Same as above, but log last 1_KiB outputs. +}; + +ExternalCommandErrorExitReaction parseExternalCommandErrorExitReaction(const std::string & config); + } diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 0e0f09c5ee4..fb6afa6f8a6 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -186,4 +186,9 @@ IMPLEMENT_SETTING_ENUM(ExternalCommandStderrReaction, ErrorCodes::BAD_ARGUMENTS, {{"none", ExternalCommandStderrReaction::NONE}, {"log", ExternalCommandStderrReaction::LOG}, {"throw", ExternalCommandStderrReaction::THROW}}) + +IMPLEMENT_SETTING_ENUM(ExternalCommandErrorExitReaction, ErrorCodes::BAD_ARGUMENTS, + {{"none", ExternalCommandErrorExitReaction::NONE}, + {"log_first", ExternalCommandErrorExitReaction::LOG_FIRST}, + {"log_last", ExternalCommandErrorExitReaction::LOG_LAST}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 034e4c8c887..7ae0666378e 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -241,4 +241,6 @@ DECLARE_SETTING_ENUM(S3QueueAction) DECLARE_SETTING_ENUM(ExternalCommandStderrReaction) +DECLARE_SETTING_ENUM(ExternalCommandErrorExitReaction) + } diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 3f513de6b36..88b87b4ff04 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -114,10 +114,7 @@ QueryPipeline ExecutableDictionarySource::loadAll() auto command = configuration.command; updateCommandIfNeeded(command, coordinator_configuration.execute_direct, context); - ShellCommandSourceConfiguration command_configuration { - .check_exit_code = true, - }; - return QueryPipeline(coordinator->createPipe(command, configuration.command_arguments, {}, sample_block, context, command_configuration)); + return QueryPipeline(coordinator->createPipe(command, configuration.command_arguments, {}, sample_block, context)); } QueryPipeline ExecutableDictionarySource::loadUpdatedAll() @@ -152,10 +149,7 @@ QueryPipeline ExecutableDictionarySource::loadUpdatedAll() LOG_TRACE(log, "loadUpdatedAll {}", command); - ShellCommandSourceConfiguration command_configuration { - .check_exit_code = true, - }; - return QueryPipeline(coordinator->createPipe(command, command_arguments, {}, sample_block, context, command_configuration)); + return QueryPipeline(coordinator->createPipe(command, command_arguments, {}, sample_block, context)); } QueryPipeline ExecutableDictionarySource::loadIds(const std::vector & ids) @@ -186,11 +180,7 @@ QueryPipeline ExecutableDictionarySource::getStreamForBlock(const Block & block) Pipes shell_input_pipes; shell_input_pipes.emplace_back(std::move(shell_input_pipe)); - ShellCommandSourceConfiguration command_configuration { - .check_exit_code = true, - }; - - auto pipe = coordinator->createPipe(command, configuration.command_arguments, std::move(shell_input_pipes), sample_block, context, command_configuration); + auto pipe = coordinator->createPipe(command, configuration.command_arguments, std::move(shell_input_pipes), sample_block, context); if (configuration.implicit_key) pipe.addTransform(std::make_shared(block, pipe.getHeader())); @@ -276,6 +266,9 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) .command_read_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_read_timeout", 10000), .command_write_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_write_timeout", 10000), .stderr_reaction = parseExternalCommandStderrReaction(config.getString(settings_config_prefix + ".stderr_reaction", "none")), + .error_exit_reaction + = parseExternalCommandErrorExitReaction(config.getString(settings_config_prefix + ".error_exit_reaction", "none")), + .check_exit_code = config.getBool(settings_config_prefix + ".check_exit_code", true), .is_executable_pool = false, .send_chunk_header = config.getBool(settings_config_prefix + ".send_chunk_header", false), .execute_direct = config.getBool(settings_config_prefix + ".execute_direct", false) diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index bd745112d21..6cd6d200aea 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -132,7 +132,6 @@ QueryPipeline ExecutablePoolDictionarySource::getStreamForBlock(const Block & bl ShellCommandSourceConfiguration command_configuration; command_configuration.read_fixed_number_of_rows = true; command_configuration.number_of_rows_to_read = block.rows(); - command_configuration.check_exit_code = true; Pipes shell_input_pipes; shell_input_pipes.emplace_back(std::move(shell_input_pipe)); @@ -234,6 +233,9 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) .command_read_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_read_timeout", 10000), .command_write_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_write_timeout", 10000), .stderr_reaction = parseExternalCommandStderrReaction(config.getString(settings_config_prefix + ".stderr_reaction", "none")), + .error_exit_reaction + = parseExternalCommandErrorExitReaction(config.getString(settings_config_prefix + ".error_exit_reaction", "none")), + .check_exit_code = config.getBool(settings_config_prefix + ".check_exit_code", true), .pool_size = config.getUInt64(settings_config_prefix + ".pool_size", 16), .max_command_execution_time_seconds = max_command_execution_time, .is_executable_pool = true, diff --git a/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp b/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp index 249e301ec67..0a20e8a71e4 100644 --- a/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp +++ b/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp @@ -173,6 +173,8 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create size_t command_write_timeout_milliseconds = config.getUInt64(key_in_config + ".command_write_timeout", 10000); ExternalCommandStderrReaction stderr_reaction = parseExternalCommandStderrReaction(config.getString(key_in_config + ".stderr_reaction", "none")); + ExternalCommandErrorExitReaction error_exit_reaction + = parseExternalCommandErrorExitReaction(config.getString(key_in_config + ".error_exit_reaction", "none")); size_t pool_size = 0; size_t max_command_execution_time = 0; @@ -241,6 +243,7 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create .command_read_timeout_milliseconds = command_read_timeout_milliseconds, .command_write_timeout_milliseconds = command_write_timeout_milliseconds, .stderr_reaction = stderr_reaction, + .error_exit_reaction = error_exit_reaction, .pool_size = pool_size, .max_command_execution_time_seconds = max_command_execution_time, .is_executable_pool = is_executable_pool, diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 8a8ad9b0572..24ffeebe391 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -103,18 +103,23 @@ class TimeoutReadBufferFromFileDescriptor : public BufferWithOwnMemory 0) @@ -157,6 +162,25 @@ public: else if (stderr_reaction == ExternalCommandStderrReaction::LOG) LOG_WARNING( &::Poco::Logger::get("TimeoutReadBufferFromFileDescriptor"), "Executable generates stderr: {}", stderr_buf); + + if (error_exit_reaction == ExternalCommandErrorExitReaction::LOG_FIRST) + { + if (BUFFER_SIZE - error_exit_buf.size() < size_t(res)) + res = BUFFER_SIZE - error_exit_buf.size(); + + if (res > 0) + error_exit_buf.append(stderr_buf.begin(), stderr_buf.begin() + res); + } + else if (error_exit_reaction == ExternalCommandErrorExitReaction::LOG_LAST) + { + if (res + error_exit_buf.size() > BUFFER_SIZE) + { + std::shift_left(error_exit_buf.begin(), error_exit_buf.end(), res + error_exit_buf.size() - BUFFER_SIZE); + error_exit_buf.resize(BUFFER_SIZE - res); + } + + error_exit_buf += stderr_buf; + } } } @@ -218,11 +242,14 @@ public: tryMakeFdBlocking(stderr_fd); } + String error_exit_buf; + private: int stdout_fd; int stderr_fd; size_t timeout_milliseconds; [[maybe_unused]] ExternalCommandStderrReaction stderr_reaction; + [[maybe_unused]] ExternalCommandErrorExitReaction error_exit_reaction; #if defined(OS_LINUX) Epoll epoll; @@ -322,6 +349,8 @@ namespace const std::string & format_, size_t command_read_timeout_milliseconds, ExternalCommandStderrReaction stderr_reaction, + ExternalCommandErrorExitReaction error_exit_reaction, + bool check_exit_code_, const Block & sample_block_, std::unique_ptr && command_, std::vector && send_data_tasks = {}, @@ -335,12 +364,10 @@ namespace , command(std::move(command_)) , configuration(configuration_) , timeout_command_out( - command->out.getFD(), - command->err.getFD(), - command_read_timeout_milliseconds, - stderr_reaction) + command->out.getFD(), command->err.getFD(), command_read_timeout_milliseconds, stderr_reaction, error_exit_reaction) , command_holder(std::move(command_holder_)) , process_pool(process_pool_) + , check_exit_code(check_exit_code_ || error_exit_reaction != ExternalCommandErrorExitReaction::NONE) { for (auto && send_data_task : send_data_tasks) { @@ -389,7 +416,12 @@ namespace thread.join(); if (command_is_invalid) + { command = nullptr; + if (!timeout_command_out.error_exit_buf.empty()) + LOG_ERROR( + &::Poco::Logger::get("ShellCommandSource"), "Executable fails with stderr: {}", timeout_command_out.error_exit_buf); + } if (command_holder && process_pool) { @@ -431,7 +463,7 @@ namespace if (!executor->pull(chunk)) { - if (configuration.check_exit_code) + if (check_exit_code) command->wait(); return {}; } @@ -491,6 +523,8 @@ namespace ShellCommandHolderPtr command_holder; std::shared_ptr process_pool; + bool check_exit_code = false; + QueryPipeline pipeline; std::unique_ptr executor; @@ -650,6 +684,8 @@ Pipe ShellCommandSourceCoordinator::createPipe( configuration.format, configuration.command_read_timeout_milliseconds, configuration.stderr_reaction, + configuration.error_exit_reaction, + configuration.check_exit_code, std::move(sample_block), std::move(process), std::move(tasks), diff --git a/src/Processors/Sources/ShellCommandSource.h b/src/Processors/Sources/ShellCommandSource.h index 6b504aa47bd..ec34a7cfeb3 100644 --- a/src/Processors/Sources/ShellCommandSource.h +++ b/src/Processors/Sources/ShellCommandSource.h @@ -34,9 +34,6 @@ struct ShellCommandSourceConfiguration size_t number_of_rows_to_read = 0; /// Max block size size_t max_block_size = DEFAULT_BLOCK_SIZE; - /// Will throw if the command exited with - /// non-zero status code - size_t check_exit_code = false; }; class ShellCommandSourceCoordinator @@ -61,6 +58,13 @@ public: /// Reaction when external command outputs data to its stderr. ExternalCommandStderrReaction stderr_reaction = ExternalCommandStderrReaction::NONE; + /// Reaction when external command exits with non-zero code. + ExternalCommandErrorExitReaction error_exit_reaction = ExternalCommandErrorExitReaction::NONE; + + /// Will throw if the command exited with + /// non-zero status code + size_t check_exit_code = false; + /// Pool size valid only if executable_pool = true size_t pool_size = 16; diff --git a/src/Storages/ExecutableSettings.h b/src/Storages/ExecutableSettings.h index cc30c77b682..ee9f018ec5a 100644 --- a/src/Storages/ExecutableSettings.h +++ b/src/Storages/ExecutableSettings.h @@ -15,7 +15,8 @@ class ASTStorage; M(UInt64, command_termination_timeout, 10, "Command termination timeout in seconds.", 0) \ M(UInt64, command_read_timeout, 10000, "Timeout for reading data from command stdout in milliseconds.", 0) \ M(UInt64, command_write_timeout, 10000, "Timeout for writing data to command stdin in milliseconds.", 0) \ - M(ExternalCommandStderrReaction, stderr_reaction, ExternalCommandStderrReaction::NONE, "Reaction when external command outputs data to its stderr.", 0) + M(ExternalCommandStderrReaction, stderr_reaction, ExternalCommandStderrReaction::NONE, "Reaction when external command outputs data to its stderr.", 0) \ + M(ExternalCommandErrorExitReaction, error_exit_reaction, ExternalCommandErrorExitReaction::NONE, "Reaction when external command exits with non-zero code.", 0) \ DECLARE_SETTINGS_TRAITS(ExecutableSettingsTraits, LIST_OF_EXECUTABLE_SETTINGS) diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 0ec9ea70e3c..4f4a67c254f 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -93,6 +93,7 @@ StorageExecutable::StorageExecutable( .command_read_timeout_milliseconds = settings.command_read_timeout, .command_write_timeout_milliseconds = settings.command_write_timeout, .stderr_reaction = settings.stderr_reaction, + .error_exit_reaction = settings.error_exit_reaction, .pool_size = settings.pool_size, .max_command_execution_time_seconds = settings.max_command_execution_time, diff --git a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml index c79094c2a05..20a6e1d49bd 100644 --- a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml +++ b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml @@ -345,4 +345,28 @@ input_always_error.py log + + + executable + test_function_always_error_exit_log_first_python + String + + UInt64 + + TabSeparated + input_log_error.py + log_first + + + + executable + test_function_always_error_exit_log_last_python + String + + UInt64 + + TabSeparated + input_log_error.py + log_last + diff --git a/tests/integration/test_executable_user_defined_function/test.py b/tests/integration/test_executable_user_defined_function/test.py index 68d97cabc0a..f459026c7af 100644 --- a/tests/integration/test_executable_user_defined_function/test.py +++ b/tests/integration/test_executable_user_defined_function/test.py @@ -306,3 +306,23 @@ def test_executable_function_always_error_python(started_cluster): + query_id + "} TimeoutReadBufferFromFileDescriptor: Executable generates stderr: Fake error" ) + + query_id = uuid.uuid4().hex + try: + node.query("SELECT test_function_always_error_exit_log_first_python(1)", query_id=query_id) + assert False, "Exception have to be thrown" + except Exception as ex: + assert "DB::Exception: Child process was exited with return code 1" in str(ex) + assert node.contains_in_log( + f"{{{query_id}}} ShellCommandSource: Executable fails with stderr: {'a' * (3 * 1024)}{'b' * 1024}\n" + ) + + query_id = uuid.uuid4().hex + try: + node.query("SELECT test_function_always_error_exit_log_last_python(1)", query_id=query_id) + assert False, "Exception have to be thrown" + except Exception as ex: + assert "DB::Exception: Child process was exited with return code 1" in str(ex) + assert node.contains_in_log( + f"{{{query_id}}} ShellCommandSource: Executable fails with stderr: {'b' * 1024}{'c' * (3 * 1024)}\n" + ) diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/input_log_error.py b/tests/integration/test_executable_user_defined_function/user_scripts/input_log_error.py new file mode 100755 index 00000000000..e61b51ee253 --- /dev/null +++ b/tests/integration/test_executable_user_defined_function/user_scripts/input_log_error.py @@ -0,0 +1,12 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == "__main__": + print(f"{'a' * (3 * 1024)}{'b' * (3 * 1024)}{'c' * (3 * 1024)}", file=sys.stderr) + sys.stderr.flush() + for line in sys.stdin: + print("Key " + line, end="") + sys.stdout.flush() + + sys.exit(1) From e49fb4f348bf8bea932c63d4b4fa81fac94f2168 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 6 Aug 2023 13:04:44 +0800 Subject: [PATCH 682/777] Fix style --- .../test_executable_user_defined_function/test.py | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_executable_user_defined_function/test.py b/tests/integration/test_executable_user_defined_function/test.py index f459026c7af..8588ebc530e 100644 --- a/tests/integration/test_executable_user_defined_function/test.py +++ b/tests/integration/test_executable_user_defined_function/test.py @@ -309,7 +309,10 @@ def test_executable_function_always_error_python(started_cluster): query_id = uuid.uuid4().hex try: - node.query("SELECT test_function_always_error_exit_log_first_python(1)", query_id=query_id) + node.query( + "SELECT test_function_always_error_exit_log_first_python(1)", + query_id=query_id, + ) assert False, "Exception have to be thrown" except Exception as ex: assert "DB::Exception: Child process was exited with return code 1" in str(ex) @@ -319,7 +322,10 @@ def test_executable_function_always_error_python(started_cluster): query_id = uuid.uuid4().hex try: - node.query("SELECT test_function_always_error_exit_log_last_python(1)", query_id=query_id) + node.query( + "SELECT test_function_always_error_exit_log_last_python(1)", + query_id=query_id, + ) assert False, "Exception have to be thrown" except Exception as ex: assert "DB::Exception: Child process was exited with return code 1" in str(ex) From c43bf153f5062e35ad7fc07ffdae3c97864a5e06 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 11 Aug 2023 22:53:55 +0800 Subject: [PATCH 683/777] Refactor --- src/Common/ShellCommandSettings.cpp | 13 +--- src/Common/ShellCommandSettings.h | 15 +--- src/Core/SettingsEnums.cpp | 6 +- src/Core/SettingsEnums.h | 2 - .../ExecutableDictionarySource.cpp | 2 - .../ExecutablePoolDictionarySource.cpp | 2 - ...alUserDefinedExecutableFunctionsLoader.cpp | 5 +- src/Processors/Sources/ShellCommandSource.cpp | 73 +++++++++---------- src/Processors/Sources/ShellCommandSource.h | 7 +- src/Storages/ExecutableSettings.h | 2 +- src/Storages/StorageExecutable.cpp | 2 +- .../functions/test_function_config.xml | 33 ++++++++- .../test.py | 41 ++++++----- .../user_scripts/input_exit_error.py | 10 +++ .../user_scripts/input_log_error.py | 2 - 15 files changed, 111 insertions(+), 104 deletions(-) create mode 100755 tests/integration/test_executable_user_defined_function/user_scripts/input_exit_error.py diff --git a/src/Common/ShellCommandSettings.cpp b/src/Common/ShellCommandSettings.cpp index b1bd8f280de..951a20e949c 100644 --- a/src/Common/ShellCommandSettings.cpp +++ b/src/Common/ShellCommandSettings.cpp @@ -15,18 +15,11 @@ namespace ErrorCodes ExternalCommandStderrReaction parseExternalCommandStderrReaction(const std::string & config) { auto reaction = magic_enum::enum_cast(Poco::toUpper(config)); - if (!reaction) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown stderr_reaction: {}. Possible values are 'none', 'log' and 'throw'", config); - - return *reaction; -} - -ExternalCommandErrorExitReaction parseExternalCommandErrorExitReaction(const std::string & config) -{ - auto reaction = magic_enum::enum_cast(Poco::toUpper(config)); if (!reaction) throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Unknown error_exit_reaction: {}. Possible values are 'none', 'log_first' and 'log_last'", config); + ErrorCodes::BAD_ARGUMENTS, + "Unknown stderr_reaction: {}. Possible values are 'none', 'log', 'log_first', 'log_last' and 'throw'", + config); return *reaction; } diff --git a/src/Common/ShellCommandSettings.h b/src/Common/ShellCommandSettings.h index 40625a6bbf2..4cfd862b873 100644 --- a/src/Common/ShellCommandSettings.h +++ b/src/Common/ShellCommandSettings.h @@ -8,19 +8,12 @@ namespace DB enum class ExternalCommandStderrReaction { NONE, /// Do nothing. - LOG, /// Try to log all outputs of stderr from the external command. - THROW /// Throw exception when the external command outputs something to its stderr. + LOG, /// Try to log all outputs of stderr from the external command immediately. + LOG_FIRST, /// Try to log first 1_KiB outputs of stderr from the external command after exit. + LOG_LAST, /// Same as above, but log last 1_KiB outputs. + THROW /// Immediately throw exception when the external command outputs something to its stderr. }; ExternalCommandStderrReaction parseExternalCommandStderrReaction(const std::string & config); -enum class ExternalCommandErrorExitReaction -{ - NONE, /// Do nothing. - LOG_FIRST, /// Try to log first 1_KiB outputs of stderr from the external command. - LOG_LAST /// Same as above, but log last 1_KiB outputs. -}; - -ExternalCommandErrorExitReaction parseExternalCommandErrorExitReaction(const std::string & config); - } diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index fb6afa6f8a6..a30d8040f47 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -185,10 +185,8 @@ IMPLEMENT_SETTING_ENUM(S3QueueAction, ErrorCodes::BAD_ARGUMENTS, IMPLEMENT_SETTING_ENUM(ExternalCommandStderrReaction, ErrorCodes::BAD_ARGUMENTS, {{"none", ExternalCommandStderrReaction::NONE}, {"log", ExternalCommandStderrReaction::LOG}, + {"log_first", ExternalCommandStderrReaction::LOG_FIRST}, + {"log_last", ExternalCommandStderrReaction::LOG_LAST}, {"throw", ExternalCommandStderrReaction::THROW}}) -IMPLEMENT_SETTING_ENUM(ExternalCommandErrorExitReaction, ErrorCodes::BAD_ARGUMENTS, - {{"none", ExternalCommandErrorExitReaction::NONE}, - {"log_first", ExternalCommandErrorExitReaction::LOG_FIRST}, - {"log_last", ExternalCommandErrorExitReaction::LOG_LAST}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 7ae0666378e..034e4c8c887 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -241,6 +241,4 @@ DECLARE_SETTING_ENUM(S3QueueAction) DECLARE_SETTING_ENUM(ExternalCommandStderrReaction) -DECLARE_SETTING_ENUM(ExternalCommandErrorExitReaction) - } diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 88b87b4ff04..f1acd610274 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -266,8 +266,6 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) .command_read_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_read_timeout", 10000), .command_write_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_write_timeout", 10000), .stderr_reaction = parseExternalCommandStderrReaction(config.getString(settings_config_prefix + ".stderr_reaction", "none")), - .error_exit_reaction - = parseExternalCommandErrorExitReaction(config.getString(settings_config_prefix + ".error_exit_reaction", "none")), .check_exit_code = config.getBool(settings_config_prefix + ".check_exit_code", true), .is_executable_pool = false, .send_chunk_header = config.getBool(settings_config_prefix + ".send_chunk_header", false), diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 6cd6d200aea..d28c73c9c52 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -233,8 +233,6 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) .command_read_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_read_timeout", 10000), .command_write_timeout_milliseconds = config.getUInt64(settings_config_prefix + ".command_write_timeout", 10000), .stderr_reaction = parseExternalCommandStderrReaction(config.getString(settings_config_prefix + ".stderr_reaction", "none")), - .error_exit_reaction - = parseExternalCommandErrorExitReaction(config.getString(settings_config_prefix + ".error_exit_reaction", "none")), .check_exit_code = config.getBool(settings_config_prefix + ".check_exit_code", true), .pool_size = config.getUInt64(settings_config_prefix + ".pool_size", 16), .max_command_execution_time_seconds = max_command_execution_time, diff --git a/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp b/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp index 0a20e8a71e4..ca142479ff1 100644 --- a/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp +++ b/src/Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.cpp @@ -173,8 +173,7 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create size_t command_write_timeout_milliseconds = config.getUInt64(key_in_config + ".command_write_timeout", 10000); ExternalCommandStderrReaction stderr_reaction = parseExternalCommandStderrReaction(config.getString(key_in_config + ".stderr_reaction", "none")); - ExternalCommandErrorExitReaction error_exit_reaction - = parseExternalCommandErrorExitReaction(config.getString(key_in_config + ".error_exit_reaction", "none")); + bool check_exit_code = config.getBool(key_in_config + ".check_exit_code", true); size_t pool_size = 0; size_t max_command_execution_time = 0; @@ -243,7 +242,7 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create .command_read_timeout_milliseconds = command_read_timeout_milliseconds, .command_write_timeout_milliseconds = command_write_timeout_milliseconds, .stderr_reaction = stderr_reaction, - .error_exit_reaction = error_exit_reaction, + .check_exit_code = check_exit_code, .pool_size = pool_size, .max_command_execution_time_seconds = max_command_execution_time, .is_executable_pool = is_executable_pool, diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 24ffeebe391..ef6cd5ef84c 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -14,6 +14,7 @@ #include #include #include +#include namespace DB @@ -106,20 +107,18 @@ public: int stdout_fd_, int stderr_fd_, size_t timeout_milliseconds_, - ExternalCommandStderrReaction stderr_reaction_, - ExternalCommandErrorExitReaction error_exit_reaction_) + ExternalCommandStderrReaction stderr_reaction_) : stdout_fd(stdout_fd_) , stderr_fd(stderr_fd_) , timeout_milliseconds(timeout_milliseconds_) , stderr_reaction(stderr_reaction_) - , error_exit_reaction(error_exit_reaction_) { makeFdNonBlocking(stdout_fd); makeFdNonBlocking(stderr_fd); #if defined(OS_LINUX) epoll.add(stdout_fd); - if (stderr_reaction != ExternalCommandStderrReaction::NONE || error_exit_reaction != ExternalCommandErrorExitReaction::NONE) + if (stderr_reaction != ExternalCommandStderrReaction::NONE) epoll.add(stderr_fd); #endif } @@ -129,8 +128,6 @@ public: size_t bytes_read = 0; #if defined(OS_LINUX) - static constexpr size_t BUFFER_SIZE = 4_KiB; - while (!bytes_read) { epoll_event events[2]; @@ -151,35 +148,26 @@ public: if (has_stderr) { - stderr_buf.resize(BUFFER_SIZE); - ssize_t res = ::read(stderr_fd, stderr_buf.data(), stderr_buf.size()); - + if (stderr_read_buf == nullptr) + stderr_read_buf.reset(new char[BUFFER_SIZE]); + ssize_t res = ::read(stderr_fd, stderr_read_buf.get(), BUFFER_SIZE); if (res > 0) { - stderr_buf.resize(res); + std::string_view str(stderr_read_buf.get(), res); if (stderr_reaction == ExternalCommandStderrReaction::THROW) - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Executable generates stderr: {}", stderr_buf); + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Executable generates stderr: {}", str); else if (stderr_reaction == ExternalCommandStderrReaction::LOG) LOG_WARNING( - &::Poco::Logger::get("TimeoutReadBufferFromFileDescriptor"), "Executable generates stderr: {}", stderr_buf); - - if (error_exit_reaction == ExternalCommandErrorExitReaction::LOG_FIRST) + &::Poco::Logger::get("TimeoutReadBufferFromFileDescriptor"), "Executable generates stderr: {}", str); + else if (stderr_reaction == ExternalCommandStderrReaction::LOG_FIRST) { - if (BUFFER_SIZE - error_exit_buf.size() < size_t(res)) - res = BUFFER_SIZE - error_exit_buf.size(); - + res = std::min(ssize_t(stderr_result_buf.reserve()), res); if (res > 0) - error_exit_buf.append(stderr_buf.begin(), stderr_buf.begin() + res); + stderr_result_buf.insert(stderr_result_buf.end(), str.begin(), str.begin() + res); } - else if (error_exit_reaction == ExternalCommandErrorExitReaction::LOG_LAST) + else if (stderr_reaction == ExternalCommandStderrReaction::LOG_LAST) { - if (res + error_exit_buf.size() > BUFFER_SIZE) - { - std::shift_left(error_exit_buf.begin(), error_exit_buf.end(), res + error_exit_buf.size() - BUFFER_SIZE); - error_exit_buf.resize(BUFFER_SIZE - res); - } - - error_exit_buf += stderr_buf; + stderr_result_buf.insert(stderr_result_buf.end(), str.begin(), str.begin() + res); } } } @@ -240,20 +228,33 @@ public: { tryMakeFdBlocking(stdout_fd); tryMakeFdBlocking(stderr_fd); - } - String error_exit_buf; +#if defined(OS_LINUX) + if (!stderr_result_buf.empty()) + { + String stderr_result; + stderr_result.reserve(stderr_result_buf.size()); + stderr_result.append(stderr_result_buf.begin(), stderr_result_buf.end()); + LOG_WARNING( + &::Poco::Logger::get("ShellCommandSource"), + "Executable generates stderr at the {}: {}", + stderr_reaction == ExternalCommandStderrReaction::LOG_FIRST ? "beginning" : "end", + stderr_result); + } +#endif + } private: int stdout_fd; int stderr_fd; size_t timeout_milliseconds; [[maybe_unused]] ExternalCommandStderrReaction stderr_reaction; - [[maybe_unused]] ExternalCommandErrorExitReaction error_exit_reaction; #if defined(OS_LINUX) + static constexpr size_t BUFFER_SIZE = 4_KiB; Epoll epoll; - String stderr_buf; + std::unique_ptr stderr_read_buf; + boost::circular_buffer_space_optimized stderr_result_buf{BUFFER_SIZE}; #endif }; @@ -349,7 +350,6 @@ namespace const std::string & format_, size_t command_read_timeout_milliseconds, ExternalCommandStderrReaction stderr_reaction, - ExternalCommandErrorExitReaction error_exit_reaction, bool check_exit_code_, const Block & sample_block_, std::unique_ptr && command_, @@ -363,11 +363,10 @@ namespace , sample_block(sample_block_) , command(std::move(command_)) , configuration(configuration_) - , timeout_command_out( - command->out.getFD(), command->err.getFD(), command_read_timeout_milliseconds, stderr_reaction, error_exit_reaction) + , timeout_command_out(command->out.getFD(), command->err.getFD(), command_read_timeout_milliseconds, stderr_reaction) , command_holder(std::move(command_holder_)) , process_pool(process_pool_) - , check_exit_code(check_exit_code_ || error_exit_reaction != ExternalCommandErrorExitReaction::NONE) + , check_exit_code(check_exit_code_) { for (auto && send_data_task : send_data_tasks) { @@ -416,12 +415,7 @@ namespace thread.join(); if (command_is_invalid) - { command = nullptr; - if (!timeout_command_out.error_exit_buf.empty()) - LOG_ERROR( - &::Poco::Logger::get("ShellCommandSource"), "Executable fails with stderr: {}", timeout_command_out.error_exit_buf); - } if (command_holder && process_pool) { @@ -684,7 +678,6 @@ Pipe ShellCommandSourceCoordinator::createPipe( configuration.format, configuration.command_read_timeout_milliseconds, configuration.stderr_reaction, - configuration.error_exit_reaction, configuration.check_exit_code, std::move(sample_block), std::move(process), diff --git a/src/Processors/Sources/ShellCommandSource.h b/src/Processors/Sources/ShellCommandSource.h index ec34a7cfeb3..44bd725bbe2 100644 --- a/src/Processors/Sources/ShellCommandSource.h +++ b/src/Processors/Sources/ShellCommandSource.h @@ -58,11 +58,10 @@ public: /// Reaction when external command outputs data to its stderr. ExternalCommandStderrReaction stderr_reaction = ExternalCommandStderrReaction::NONE; - /// Reaction when external command exits with non-zero code. - ExternalCommandErrorExitReaction error_exit_reaction = ExternalCommandErrorExitReaction::NONE; - /// Will throw if the command exited with - /// non-zero status code + /// non-zero status code. + /// NOTE: If executable pool is used, we cannot check exit code, + /// which makes this configuration no effect. size_t check_exit_code = false; /// Pool size valid only if executable_pool = true diff --git a/src/Storages/ExecutableSettings.h b/src/Storages/ExecutableSettings.h index ee9f018ec5a..f1ceb7ce59f 100644 --- a/src/Storages/ExecutableSettings.h +++ b/src/Storages/ExecutableSettings.h @@ -16,7 +16,7 @@ class ASTStorage; M(UInt64, command_read_timeout, 10000, "Timeout for reading data from command stdout in milliseconds.", 0) \ M(UInt64, command_write_timeout, 10000, "Timeout for writing data to command stdin in milliseconds.", 0) \ M(ExternalCommandStderrReaction, stderr_reaction, ExternalCommandStderrReaction::NONE, "Reaction when external command outputs data to its stderr.", 0) \ - M(ExternalCommandErrorExitReaction, error_exit_reaction, ExternalCommandErrorExitReaction::NONE, "Reaction when external command exits with non-zero code.", 0) \ + M(Bool, check_exit_code, true, "Throw exception if the command exited with non-zero status code.", 0) \ DECLARE_SETTINGS_TRAITS(ExecutableSettingsTraits, LIST_OF_EXECUTABLE_SETTINGS) diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 4f4a67c254f..df03301b5e8 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -93,7 +93,7 @@ StorageExecutable::StorageExecutable( .command_read_timeout_milliseconds = settings.command_read_timeout, .command_write_timeout_milliseconds = settings.command_write_timeout, .stderr_reaction = settings.stderr_reaction, - .error_exit_reaction = settings.error_exit_reaction, + .check_exit_code = settings.check_exit_code, .pool_size = settings.pool_size, .max_command_execution_time_seconds = settings.max_command_execution_time, diff --git a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml index 20a6e1d49bd..99efd99ab31 100644 --- a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml +++ b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml @@ -348,25 +348,50 @@ executable - test_function_always_error_exit_log_first_python + test_function_always_error_log_first_python String UInt64 TabSeparated input_log_error.py - log_first + log_first executable - test_function_always_error_exit_log_last_python + test_function_always_error_log_last_python String UInt64 TabSeparated input_log_error.py - log_last + log_last + + + + executable + test_function_exit_error_ignore_python + String + + UInt64 + + TabSeparated + input_exit_error.py + 0 + + + + executable + test_function_exit_error_fail_python + String + + UInt64 + + TabSeparated + input_exit_error.py + + diff --git a/tests/integration/test_executable_user_defined_function/test.py b/tests/integration/test_executable_user_defined_function/test.py index 8588ebc530e..f2d84249c6c 100644 --- a/tests/integration/test_executable_user_defined_function/test.py +++ b/tests/integration/test_executable_user_defined_function/test.py @@ -302,33 +302,38 @@ def test_executable_function_always_error_python(started_cluster): == "Key 1\n" ) assert node.contains_in_log( - "{" - + query_id - + "} TimeoutReadBufferFromFileDescriptor: Executable generates stderr: Fake error" + f"{{{query_id}}} TimeoutReadBufferFromFileDescriptor: Executable generates stderr: Fake error" ) query_id = uuid.uuid4().hex - try: + assert ( node.query( - "SELECT test_function_always_error_exit_log_first_python(1)", - query_id=query_id, - ) - assert False, "Exception have to be thrown" - except Exception as ex: - assert "DB::Exception: Child process was exited with return code 1" in str(ex) - assert node.contains_in_log( - f"{{{query_id}}} ShellCommandSource: Executable fails with stderr: {'a' * (3 * 1024)}{'b' * 1024}\n" + "SELECT test_function_always_error_log_first_python(1)", query_id=query_id ) + == "Key 1\n" + ) + assert node.contains_in_log( + f"{{{query_id}}} TimeoutReadBufferFromFileDescriptor: Executable generates stderr at the beginning: {'a' * (3 * 1024)}{'b' * 1024}\n" + ) query_id = uuid.uuid4().hex - try: + assert ( node.query( - "SELECT test_function_always_error_exit_log_last_python(1)", - query_id=query_id, + "SELECT test_function_always_error_log_last_python(1)", query_id=query_id ) + == "Key 1\n" + ) + assert node.contains_in_log( + f"{{{query_id}}} TimeoutReadBufferFromFileDescriptor: Executable generates stderr at the end: {'b' * 1024}{'c' * (3 * 1024)}\n" + ) + + assert ( + node.query("SELECT test_function_exit_error_ignore_python(1)") + == "Key 1\n" + ) + + try: + node.query("SELECT test_function_exit_error_fail_python(1)") assert False, "Exception have to be thrown" except Exception as ex: assert "DB::Exception: Child process was exited with return code 1" in str(ex) - assert node.contains_in_log( - f"{{{query_id}}} ShellCommandSource: Executable fails with stderr: {'b' * 1024}{'c' * (3 * 1024)}\n" - ) diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/input_exit_error.py b/tests/integration/test_executable_user_defined_function/user_scripts/input_exit_error.py new file mode 100755 index 00000000000..036d69a8297 --- /dev/null +++ b/tests/integration/test_executable_user_defined_function/user_scripts/input_exit_error.py @@ -0,0 +1,10 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == "__main__": + for line in sys.stdin: + print("Key " + line, end="") + sys.stdout.flush() + + sys.exit(1) diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/input_log_error.py b/tests/integration/test_executable_user_defined_function/user_scripts/input_log_error.py index e61b51ee253..d622f44f75d 100755 --- a/tests/integration/test_executable_user_defined_function/user_scripts/input_log_error.py +++ b/tests/integration/test_executable_user_defined_function/user_scripts/input_log_error.py @@ -8,5 +8,3 @@ if __name__ == "__main__": for line in sys.stdin: print("Key " + line, end="") sys.stdout.flush() - - sys.exit(1) From 69ca36bfdfa7b30f80295ed4be3d63549a14d82b Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 12 Aug 2023 02:14:44 +0800 Subject: [PATCH 684/777] Fix black style again --- .../test_executable_user_defined_function/test.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/integration/test_executable_user_defined_function/test.py b/tests/integration/test_executable_user_defined_function/test.py index f2d84249c6c..ccc06a7b43a 100644 --- a/tests/integration/test_executable_user_defined_function/test.py +++ b/tests/integration/test_executable_user_defined_function/test.py @@ -327,10 +327,7 @@ def test_executable_function_always_error_python(started_cluster): f"{{{query_id}}} TimeoutReadBufferFromFileDescriptor: Executable generates stderr at the end: {'b' * 1024}{'c' * (3 * 1024)}\n" ) - assert ( - node.query("SELECT test_function_exit_error_ignore_python(1)") - == "Key 1\n" - ) + assert node.query("SELECT test_function_exit_error_ignore_python(1)") == "Key 1\n" try: node.query("SELECT test_function_exit_error_fail_python(1)") From b6de70b0424955244b0d442d076629f7e8538805 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 12 Aug 2023 09:55:32 +0800 Subject: [PATCH 685/777] StorageExecutable check_exit_code default to false --- src/Storages/ExecutableSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ExecutableSettings.h b/src/Storages/ExecutableSettings.h index f1ceb7ce59f..10dbae8ac9f 100644 --- a/src/Storages/ExecutableSettings.h +++ b/src/Storages/ExecutableSettings.h @@ -16,7 +16,7 @@ class ASTStorage; M(UInt64, command_read_timeout, 10000, "Timeout for reading data from command stdout in milliseconds.", 0) \ M(UInt64, command_write_timeout, 10000, "Timeout for writing data to command stdin in milliseconds.", 0) \ M(ExternalCommandStderrReaction, stderr_reaction, ExternalCommandStderrReaction::NONE, "Reaction when external command outputs data to its stderr.", 0) \ - M(Bool, check_exit_code, true, "Throw exception if the command exited with non-zero status code.", 0) \ + M(Bool, check_exit_code, false, "Throw exception if the command exited with non-zero status code.", 0) \ DECLARE_SETTINGS_TRAITS(ExecutableSettingsTraits, LIST_OF_EXECUTABLE_SETTINGS) From b2dd055cdfbbeb5e9240df71a54810560ccefe36 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 15 Aug 2023 16:20:54 +0800 Subject: [PATCH 686/777] Use poll instead --- src/Processors/Sources/ShellCommandSource.cpp | 81 +++++++------------ 1 file changed, 30 insertions(+), 51 deletions(-) diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index ef6cd5ef84c..55a7afe0538 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -2,7 +2,6 @@ #include -#include #include #include @@ -67,19 +66,14 @@ static void makeFdBlocking(int fd) throwFromErrno("Cannot set blocking mode of pipe", ErrorCodes::CANNOT_FCNTL); } -static bool pollFd(int fd, size_t timeout_milliseconds, int events) +static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_milliseconds) { - pollfd pfd; - pfd.fd = fd; - pfd.events = events; - pfd.revents = 0; - int res; while (true) { Stopwatch watch; - res = poll(&pfd, 1, static_cast(timeout_milliseconds)); + res = poll(pfds, num, static_cast(timeout_milliseconds)); if (res < 0) { @@ -97,7 +91,17 @@ static bool pollFd(int fd, size_t timeout_milliseconds, int events) } } - return res > 0; + return res; +} + +static bool pollFd(int fd, size_t timeout_milliseconds, int events) +{ + pollfd pfd; + pfd.fd = fd; + pfd.events = events; + pfd.revents = 0; + + return pollWithTimeout(&pfd, 1, timeout_milliseconds) > 0; } class TimeoutReadBufferFromFileDescriptor : public BufferWithOwnMemory @@ -116,35 +120,31 @@ public: makeFdNonBlocking(stdout_fd); makeFdNonBlocking(stderr_fd); -#if defined(OS_LINUX) - epoll.add(stdout_fd); - if (stderr_reaction != ExternalCommandStderrReaction::NONE) - epoll.add(stderr_fd); -#endif + pfds[0].fd = stdout_fd; + pfds[0].events = POLLIN; + pfds[1].fd = stderr_fd; + pfds[1].events = POLLIN; + + if (stderr_reaction == ExternalCommandStderrReaction::NONE) + num_pfds = 1; + else + num_pfds = 2; } bool nextImpl() override { size_t bytes_read = 0; -#if defined(OS_LINUX) while (!bytes_read) { - epoll_event events[2]; - events[0].data.fd = events[1].data.fd = -1; - size_t num_events = epoll.getManyReady(2, events, static_cast(timeout_milliseconds)); + pfds[0].revents = 0; + pfds[1].revents = 0; + size_t num_events = pollWithTimeout(pfds, num_pfds, static_cast(timeout_milliseconds)); if (0 == num_events) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Pipe read timeout exceeded {} milliseconds", timeout_milliseconds); - bool has_stdout = false; - bool has_stderr = false; - for (size_t i = 0; i < num_events; ++i) - { - if (events[i].data.fd == stdout_fd) - has_stdout = true; - else if (events[i].data.fd == stderr_fd) - has_stderr = true; - } + bool has_stdout = pfds[0].revents > 0; + bool has_stderr = pfds[1].revents > 0; if (has_stderr) { @@ -186,24 +186,6 @@ public: bytes_read += res; } } -#else - while (!bytes_read) - { - if (!pollFd(stdout_fd, timeout_milliseconds, POLLIN)) - throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Pipe read timeout exceeded {} milliseconds", timeout_milliseconds); - - ssize_t res = ::read(stdout_fd, internal_buffer.begin(), internal_buffer.size()); - - if (-1 == res && errno != EINTR) - throwFromErrno("Cannot read from pipe", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR); - - if (res == 0) - break; - - if (res > 0) - bytes_read += res; - } -#endif if (bytes_read > 0) { @@ -229,7 +211,6 @@ public: tryMakeFdBlocking(stdout_fd); tryMakeFdBlocking(stderr_fd); -#if defined(OS_LINUX) if (!stderr_result_buf.empty()) { String stderr_result; @@ -241,21 +222,19 @@ public: stderr_reaction == ExternalCommandStderrReaction::LOG_FIRST ? "beginning" : "end", stderr_result); } -#endif } private: int stdout_fd; int stderr_fd; size_t timeout_milliseconds; - [[maybe_unused]] ExternalCommandStderrReaction stderr_reaction; + ExternalCommandStderrReaction stderr_reaction; -#if defined(OS_LINUX) static constexpr size_t BUFFER_SIZE = 4_KiB; - Epoll epoll; + pollfd pfds[2]; + size_t num_pfds; std::unique_ptr stderr_read_buf; boost::circular_buffer_space_optimized stderr_result_buf{BUFFER_SIZE}; -#endif }; class TimeoutWriteBufferFromFileDescriptor : public BufferWithOwnMemory From 1c300a31b1983fdd63f02b5b54058d879bf55d70 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 16 Aug 2023 10:33:35 +0800 Subject: [PATCH 687/777] Fix build --- src/Processors/Sources/ShellCommandSource.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Sources/ShellCommandSource.cpp b/src/Processors/Sources/ShellCommandSource.cpp index 55a7afe0538..effe2950104 100644 --- a/src/Processors/Sources/ShellCommandSource.cpp +++ b/src/Processors/Sources/ShellCommandSource.cpp @@ -73,7 +73,7 @@ static int pollWithTimeout(pollfd * pfds, size_t num, size_t timeout_millisecond while (true) { Stopwatch watch; - res = poll(pfds, num, static_cast(timeout_milliseconds)); + res = poll(pfds, static_cast(num), static_cast(timeout_milliseconds)); if (res < 0) { @@ -139,7 +139,7 @@ public: { pfds[0].revents = 0; pfds[1].revents = 0; - size_t num_events = pollWithTimeout(pfds, num_pfds, static_cast(timeout_milliseconds)); + size_t num_events = pollWithTimeout(pfds, num_pfds, timeout_milliseconds); if (0 == num_events) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Pipe read timeout exceeded {} milliseconds", timeout_milliseconds); From d3edcaae04a660553a15cf0cdb404cba1c7be0d4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 18 Aug 2023 10:25:52 +0200 Subject: [PATCH 688/777] Fixed for using failed_op_index before it is initialized --- src/Common/ZooKeeper/KeeperException.h | 3 +++ src/Common/ZooKeeper/ZooKeeper.cpp | 11 ++++++++--- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/src/Common/ZooKeeper/KeeperException.h b/src/Common/ZooKeeper/KeeperException.h index 4442c41a19d..f957bd2288f 100644 --- a/src/Common/ZooKeeper/KeeperException.h +++ b/src/Common/ZooKeeper/KeeperException.h @@ -24,6 +24,9 @@ public: static void check(Coordination::Error code, const Coordination::Requests & requests, const Coordination::Responses & responses); KeeperMultiException(Coordination::Error code, const Coordination::Requests & requests, const Coordination::Responses & responses); + +private: + KeeperMultiException(Coordination::Error code, size_t failed_op_index_, const Coordination::Requests & requests_, const Coordination::Responses & responses_); }; size_t getFailedOpIndex(Coordination::Error code, const Coordination::Responses & responses); diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 85db0dcd2fe..d4127b76a9d 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -1264,13 +1264,18 @@ size_t getFailedOpIndex(Coordination::Error exception_code, const Coordination:: } -KeeperMultiException::KeeperMultiException(Coordination::Error exception_code, const Coordination::Requests & requests_, const Coordination::Responses & responses_) - : KeeperException(exception_code, "Transaction failed: Op #{}, path", failed_op_index), - requests(requests_), responses(responses_), failed_op_index(getFailedOpIndex(exception_code, responses)) +KeeperMultiException::KeeperMultiException(Coordination::Error exception_code, size_t failed_op_index_, const Coordination::Requests & requests_, const Coordination::Responses & responses_) + : KeeperException(exception_code, "Transaction failed: Op #{}, path", failed_op_index_), + requests(requests_), responses(responses_), failed_op_index(failed_op_index_) { addMessage(getPathForFirstFailedOp()); } +KeeperMultiException::KeeperMultiException(Coordination::Error exception_code, const Coordination::Requests & requests_, const Coordination::Responses & responses_) + : KeeperMultiException(exception_code, getFailedOpIndex(exception_code, responses_), requests_, responses_) +{ +} + std::string KeeperMultiException::getPathForFirstFailedOp() const { From 4f9429d2e4f5f5fe4fdd0d14a999739f6b3a9532 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Fri, 18 Aug 2023 17:58:24 +0800 Subject: [PATCH 689/777] fix ut tests/queries/0_stateless/00900_null_array_orc_load.sh --- src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index b05418bbf35..d46df4c511c 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -662,7 +662,8 @@ static ColumnWithTypeAndName readColumnFromORCColumn( bool & skipped, DataTypePtr type_hint = nullptr) { - if (!inside_nullable && (orc_column->hasNulls || (type_hint && type_hint->isNullable()))) + if (!inside_nullable && (orc_column->hasNulls || (type_hint && type_hint->isNullable())) + && (orc_type->getKind() != orc::LIST && orc_type->getKind() != orc::MAP && orc_type->getKind() != orc::STRUCT)) { DataTypePtr nested_type_hint; if (type_hint) From 428810a5b242ca7f4aab970baf7c9282c6af4810 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 18 Aug 2023 13:37:29 +0300 Subject: [PATCH 690/777] Update setup_export_logs.sh --- docker/test/base/setup_export_logs.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index d68fd392d62..9a48c0fcafc 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -18,7 +18,7 @@ CONNECTION_PARAMETERS=${CONNECTION_PARAMETERS:=""} clickhouse-client --query "SYSTEM FLUSH LOGS" # It's doesn't make sense to try creating tables if SYNC fails -echo "SYSTEM SYNC DATABASE REPLICA default" clickhouse-client --receive_timeout 180 $CONNECTION_PARAMETERS || exit 0 +echo "SYSTEM SYNC DATABASE REPLICA default" | clickhouse-client --receive_timeout 180 $CONNECTION_PARAMETERS || exit 0 # For each system log table: clickhouse-client --query "SHOW TABLES FROM system LIKE '%\\_log'" | while read -r table From 9174bee76aa803473cd92aab42dcb74c96f53a2f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 18 Aug 2023 13:45:35 +0300 Subject: [PATCH 691/777] Update 00002_log_and_exception_messages_formatting.sql --- .../0_stateless/00002_log_and_exception_messages_formatting.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql index e2de14a4a8b..c96018d8091 100644 --- a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -22,7 +22,7 @@ select 'unknown runtime exceptions', max2(coalesce(sum(length(message_format_str -- FIXME some of the following messages are not informative and it has to be fixed create temporary table known_short_messages (s String) as select * from (select -['', '({}) Keys: {}', '({}) {}', 'Aggregating', 'Became leader', 'Cleaning queue', +['', '{} ({})', '({}) Keys: {}', '({}) {}', 'Aggregating', 'Became leader', 'Cleaning queue', 'Creating set.', 'Cyclic aliases', 'Detaching {}', 'Executing {}', 'Fire events: {}', 'Found part {}', 'Loaded queue', 'No sharding key', 'No tables', 'Query: {}', 'Removed', 'Removed part {}', 'Removing parts.', 'Request URI: {}', 'Sending part {}', From 9a4254616a1a3d0a325f4f4d671a9f5a0adb9631 Mon Sep 17 00:00:00 2001 From: Ash Vardanian <1983160+ashvardanian@users.noreply.github.com> Date: Fri, 18 Aug 2023 12:45:34 +0100 Subject: [PATCH 692/777] Docs: Extend USearch implementation details --- .../table-engines/mergetree-family/annindexes.md | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index dbcc76254ca..a2e3184e041 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -221,14 +221,16 @@ SETTINGS annoy_index_search_k_nodes=100; ## USearch {#usearch} -USearch indexes are currently experimental, to use them you first need to `SET allow_experimental_usearch_index = 1`. - -This type of ANN index is based on the [the USearch library](https://github.com/unum-cloud/usearch) which implements the [HNSW -algorithm](https://arxiv.org/abs/1603.09320), i.e. builds a hierarchical graph where each point represents a vector and the edges represent -similarity. +This type of ANN index is based on the [the USearch library](https://github.com/unum-cloud/usearch), which implements the [HNSW +algorithm](https://arxiv.org/abs/1603.09320), i.e., builds a hierarchical graph where each point represents a vector and the edges represent +similarity. Such hierarchical structures can be very efficient on large collections. They may often fetch 0.05% or less data from the +overall dataset, while still providing 99% recall. This is especially useful when working with high-dimensionality vectors, +that are expensive to load and compare. The library also has several hardware-specific SIMD optimizations to accelerate further +distance computations on modern Arm (NEON and SVE) and x86 (AVX2 and AVX-512) CPUs and OS-specific optimizations to allow efficient +navigation around immutable persistent files, without loading them into RAM.
-