mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Merge branch 'corrMatrix' of github.com:FFFFFFFHHHHHHH/ClickHouse into corrMatrix
This commit is contained in:
commit
0a6cc61fd9
@ -8,16 +8,13 @@ set (SRCS
|
||||
getPageSize.cpp
|
||||
getThreadId.cpp
|
||||
JSON.cpp
|
||||
LineReader.cpp
|
||||
mremap.cpp
|
||||
phdr_cache.cpp
|
||||
preciseExp10.cpp
|
||||
setTerminalEcho.cpp
|
||||
shift10.cpp
|
||||
sleep.cpp
|
||||
terminalColors.cpp
|
||||
errnoToString.cpp
|
||||
ReplxxLineReader.cpp
|
||||
StringRef.cpp
|
||||
safeExit.cpp
|
||||
throwError.cpp
|
||||
@ -40,11 +37,6 @@ else ()
|
||||
target_compile_definitions(common PUBLIC WITH_COVERAGE=0)
|
||||
endif ()
|
||||
|
||||
# FIXME: move libraries for line reading out from base
|
||||
if (TARGET ch_rust::skim)
|
||||
target_link_libraries(common PUBLIC ch_rust::skim)
|
||||
endif()
|
||||
|
||||
target_include_directories(common PUBLIC .. "${CMAKE_CURRENT_BINARY_DIR}/..")
|
||||
|
||||
if (OS_DARWIN AND NOT USE_STATIC_LIBRARIES)
|
||||
|
@ -1,28 +0,0 @@
|
||||
#include <base/setTerminalEcho.h>
|
||||
#include <base/errnoToString.h>
|
||||
#include <stdexcept>
|
||||
#include <cstring>
|
||||
#include <string>
|
||||
#include <termios.h>
|
||||
#include <unistd.h>
|
||||
|
||||
|
||||
void setTerminalEcho(bool enable)
|
||||
{
|
||||
/// Obtain terminal attributes,
|
||||
/// toggle the ECHO flag
|
||||
/// and set them back.
|
||||
|
||||
struct termios tty{};
|
||||
|
||||
if (0 != tcgetattr(STDIN_FILENO, &tty))
|
||||
throw std::runtime_error(std::string("setTerminalEcho failed get: ") + errnoToString());
|
||||
|
||||
if (enable)
|
||||
tty.c_lflag |= ECHO;
|
||||
else
|
||||
tty.c_lflag &= ~ECHO;
|
||||
|
||||
if (0 != tcsetattr(STDIN_FILENO, TCSANOW, &tty))
|
||||
throw std::runtime_error(std::string("setTerminalEcho failed set: ") + errnoToString());
|
||||
}
|
@ -1,4 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
/// Enable or disable echoing of typed characters. Throws std::runtime_error on error.
|
||||
void setTerminalEcho(bool enable);
|
@ -51,7 +51,6 @@ function clone
|
||||
)
|
||||
|
||||
ls -lath ||:
|
||||
|
||||
}
|
||||
|
||||
function wget_with_retry
|
||||
|
@ -455,7 +455,8 @@ if [ "$DISABLE_BC_CHECK" -ne "1" ]; then
|
||||
# Start new server
|
||||
mv package_folder/clickhouse /usr/bin/
|
||||
mv package_folder/clickhouse.debug /usr/lib/debug/usr/bin/clickhouse.debug
|
||||
export ZOOKEEPER_FAULT_INJECTION=1
|
||||
# Disable fault injections on start (we don't test them here, and it can lead to tons of requests in case of huge number of tables).
|
||||
export ZOOKEEPER_FAULT_INJECTION=0
|
||||
configure
|
||||
start 500
|
||||
clickhouse-client --query "SELECT 'Backward compatibility check: Server successfully started', 'OK'" >> /test_output/test_results.tsv \
|
||||
|
@ -13,6 +13,10 @@ set (CLICKHOUSE_CLIENT_LINK
|
||||
string_utils
|
||||
)
|
||||
|
||||
if (TARGET ch_rust::skim)
|
||||
list(APPEND CLICKHOUSE_CLIENT_LINK PRIVATE ch_rust::skim)
|
||||
endif()
|
||||
|
||||
# Always use internal readpassphrase
|
||||
list(APPEND CLICKHOUSE_CLIENT_LINK PRIVATE readpassphrase)
|
||||
|
||||
|
@ -18,6 +18,10 @@ if(NOT CLICKHOUSE_ONE_SHARED)
|
||||
target_link_libraries(clickhouse-local-lib PRIVATE clickhouse-server-lib)
|
||||
endif()
|
||||
|
||||
if (TARGET ch_rust::skim)
|
||||
target_link_libraries(clickhouse-local-lib PRIVATE ch_rust::skim)
|
||||
endif()
|
||||
|
||||
# Always use internal readpassphrase
|
||||
target_link_libraries(clickhouse-local-lib PRIVATE readpassphrase)
|
||||
|
||||
|
@ -1,647 +0,0 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/SymbolIndex.h>
|
||||
#include <Common/ArenaAllocator.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <filesystem>
|
||||
|
||||
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<ListNode *>(arena->alloc(sizeof(ListNode)));
|
||||
TreeNode * tree_node = reinterpret_cast<TreeNode *>(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<UInt64> & values, DB::PaddedPODArray<UInt64> & offsets, std::vector<UInt64> & 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<UInt64>;
|
||||
|
||||
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<Trace>;
|
||||
|
||||
Traces dump(size_t max_depth, size_t min_bytes) const
|
||||
{
|
||||
Traces traces;
|
||||
Trace::Frames frames;
|
||||
std::vector<size_t> allocated_total;
|
||||
std::vector<size_t> allocated_self;
|
||||
std::vector<ListNode *> 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<UInt8> & chars, DB::PaddedPODArray<UInt64> & 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<UInt8> & chars, DB::PaddedPODArray<UInt64> & 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<UInt8> & chars,
|
||||
DB::PaddedPODArray<UInt64> & offsets)
|
||||
{
|
||||
DB::WriteBufferFromOwnString out;
|
||||
|
||||
std::unordered_map<uintptr_t, size_t> 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<const void *>(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<UInt64, Pair>;
|
||||
|
||||
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<Entry *>(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<AggregateFunctionFlameGraphTree::ListNode *> 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<UInt8> & chars,
|
||||
DB::PaddedPODArray<UInt64> & 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<AggregateFunctionFlameGraphData, AggregateFunctionFlameGraph>
|
||||
{
|
||||
public:
|
||||
explicit AggregateFunctionFlameGraph(const DataTypes & argument_types_)
|
||||
: IAggregateFunctionDataHelper<AggregateFunctionFlameGraphData, AggregateFunctionFlameGraph>(argument_types_, {}, createResultType())
|
||||
{}
|
||||
|
||||
String getName() const override { return "flameGraph"; }
|
||||
|
||||
static DataTypePtr createResultType()
|
||||
{
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>());
|
||||
}
|
||||
|
||||
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<const ColumnArray *>(columns[0]);
|
||||
|
||||
const auto & trace_offsets = trace->getOffsets();
|
||||
const auto & trace_values = typeid_cast<const ColumnUInt64 *>(&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<const ColumnInt64 *>(columns[1])->getData();
|
||||
allocated = sizes[row_num];
|
||||
}
|
||||
|
||||
UInt64 ptr = 0;
|
||||
if (argument_types.size() >= 3)
|
||||
{
|
||||
const auto & ptrs = typeid_cast<const ColumnUInt64 *>(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<size_t> /* version */) const override
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Serialization for function flameGraph is not implemented.");
|
||||
}
|
||||
|
||||
void deserialize(AggregateDataPtr __restrict, ReadBuffer &, std::optional<size_t> /* 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<ColumnArray &>(to);
|
||||
auto & str = assert_cast<ColumnString &>(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<DataTypeUInt64>();
|
||||
auto trace_type = std::make_shared<DataTypeArray>(ptr_type);
|
||||
auto size_type = std::make_shared<DataTypeInt64>();
|
||||
|
||||
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<AggregateFunctionFlameGraph>(argument_types);
|
||||
}
|
||||
|
||||
void registerAggregateFunctionFlameGraph(AggregateFunctionFactory & factory)
|
||||
{
|
||||
AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = true };
|
||||
|
||||
factory.registerFunction("flameGraph", { createAggregateFunctionFlameGraph, properties });
|
||||
}
|
||||
|
||||
}
|
@ -74,7 +74,6 @@ void registerAggregateFunctionExponentialMovingAverage(AggregateFunctionFactory
|
||||
void registerAggregateFunctionSparkbar(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionIntervalLengthSum(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionAnalysisOfVariance(AggregateFunctionFactory &);
|
||||
void registerAggregateFunctionFlameGraph(AggregateFunctionFactory &);
|
||||
|
||||
class AggregateFunctionCombinatorFactory;
|
||||
void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &);
|
||||
@ -161,7 +160,6 @@ void registerAggregateFunctions()
|
||||
registerAggregateFunctionExponentialMovingAverage(factory);
|
||||
registerAggregateFunctionSparkbar(factory);
|
||||
registerAggregateFunctionAnalysisOfVariance(factory);
|
||||
registerAggregateFunctionFlameGraph(factory);
|
||||
|
||||
registerWindowFunctions(factory);
|
||||
}
|
||||
|
@ -61,14 +61,8 @@ namespace
|
||||
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, Poco::Logger * log)
|
||||
{
|
||||
auto address = makeSocketAddress(host, port, log);
|
||||
#if POCO_VERSION < 0x01080000
|
||||
socket.bind(address, /* reuseAddress = */ true);
|
||||
#else
|
||||
socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ false);
|
||||
#endif
|
||||
|
||||
socket.listen(/* backlog = */ 64);
|
||||
|
||||
return address;
|
||||
}
|
||||
}
|
||||
|
@ -2,11 +2,10 @@
|
||||
|
||||
#include <IO/ReadWriteBufferFromHTTP.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
#include <Poco/URI.h>
|
||||
#include <filesystem>
|
||||
#include <thread>
|
||||
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
namespace DB
|
||||
@ -97,9 +96,13 @@ std::unique_ptr<ShellCommand> IBridgeHelper::startBridgeCommand()
|
||||
|
||||
LOG_TRACE(getLog(), "Starting {}", serviceAlias());
|
||||
|
||||
/// We will terminate it with the KILL signal instead of the TERM signal,
|
||||
/// because it's more reliable for arbitrary third-party ODBC drivers.
|
||||
/// The drivers can spawn threads, install their own signal handlers... we don't care.
|
||||
|
||||
ShellCommand::Config command_config(path.string());
|
||||
command_config.arguments = cmd_args;
|
||||
command_config.terminate_in_destructor_strategy = ShellCommand::DestructorStrategy(true);
|
||||
command_config.terminate_in_destructor_strategy = ShellCommand::DestructorStrategy(true, SIGKILL);
|
||||
|
||||
return ShellCommand::executeDirect(command_config);
|
||||
}
|
||||
|
@ -332,6 +332,12 @@ macro (dbms_target_link_libraries)
|
||||
endforeach ()
|
||||
endmacro ()
|
||||
|
||||
macro (dbms_target_include_directories)
|
||||
foreach (module ${all_modules})
|
||||
target_include_directories (${module} ${ARGN})
|
||||
endforeach ()
|
||||
endmacro ()
|
||||
|
||||
dbms_target_include_directories (PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src")
|
||||
target_include_directories (clickhouse_common_io PUBLIC "${ClickHouse_SOURCE_DIR}/src" "${ClickHouse_BINARY_DIR}/src")
|
||||
|
||||
@ -589,6 +595,11 @@ if (TARGET ch_contrib::annoy)
|
||||
dbms_target_link_libraries(PUBLIC ch_contrib::annoy)
|
||||
endif()
|
||||
|
||||
if (TARGET ch_rust::skim)
|
||||
# Add only -I, library is needed only for clickhouse-client/clickhouse-local
|
||||
dbms_target_include_directories(PRIVATE $<TARGET_PROPERTY:ch_rust::skim,INTERFACE_INCLUDE_DIRECTORIES>)
|
||||
endif()
|
||||
|
||||
include ("${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake")
|
||||
|
||||
if (ENABLE_TESTS)
|
||||
|
@ -1,3 +1,3 @@
|
||||
if (ENABLE_EXAMPLES)
|
||||
add_subdirectory(examples)
|
||||
endif()
|
||||
endif()
|
||||
|
@ -1,42 +1,30 @@
|
||||
#include <Client/ClientBase.h>
|
||||
#include <Client/LineReader.h>
|
||||
#include <Client/ClientBaseHelpers.h>
|
||||
#include <Client/TestHint.h>
|
||||
#include <Client/InternalTextLogs.h>
|
||||
#include <Client/TestTags.h>
|
||||
|
||||
#include <iostream>
|
||||
#include <filesystem>
|
||||
#include <map>
|
||||
#include <unordered_map>
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#include <base/argsToConfig.h>
|
||||
#include <base/safeExit.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Core/Protocol.h>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <base/argsToConfig.h>
|
||||
#include <base/LineReader.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <base/safeExit.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||
#include <Common/tests/gtest_global_context.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Core/Protocol.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Access/AccessControl.h>
|
||||
|
||||
#include "config_version.h"
|
||||
|
||||
#include <Common/UTF8Helpers.h>
|
||||
#include <Common/TerminalSize.h>
|
||||
#include <Common/clearPasswordFromCommandLine.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Common/NetException.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
|
||||
#include <Client/ClientBaseHelpers.h>
|
||||
#include <Client/TestHint.h>
|
||||
#include "TestTags.h"
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Parsers/ParserQuery.h>
|
||||
@ -53,26 +41,36 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
|
||||
#include <Processors/Formats/Impl/NullFormat.h>
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <Processors/Formats/IOutputFormat.h>
|
||||
#include <QueryPipeline/QueryPipeline.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
|
||||
#include <Processors/Executors/PullingAsyncPipelineExecutor.h>
|
||||
#include <Processors/Transforms/AddingDefaultsTransform.h>
|
||||
#include <QueryPipeline/QueryPipeline.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Interpreters/ReplaceQueryParameterVisitor.h>
|
||||
#include <Interpreters/ProfileEventsExt.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
#include <IO/CompressionMethod.h>
|
||||
#include <Client/InternalTextLogs.h>
|
||||
#include <IO/ForkWriteBuffer.h>
|
||||
#include <Parsers/Kusto/ParserKQLStatement.h>
|
||||
|
||||
#include <Access/AccessControl.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
|
||||
#include <boost/algorithm/string/case_conv.hpp>
|
||||
#include <iostream>
|
||||
#include <filesystem>
|
||||
#include <map>
|
||||
#include <unordered_map>
|
||||
|
||||
#include "config_version.h"
|
||||
#include "config.h"
|
||||
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
@ -1036,7 +1034,13 @@ void ClientBase::onEndOfStream()
|
||||
progress_indication.clearProgressOutput(*tty_buf);
|
||||
|
||||
if (output_format)
|
||||
{
|
||||
/// Do our best to estimate the start of the query so the output format matches the one reported by the server
|
||||
bool is_running = false;
|
||||
output_format->setStartTime(
|
||||
clock_gettime_ns(CLOCK_MONOTONIC) - static_cast<UInt64>(progress_indication.elapsedSeconds() * 1000000000), is_running);
|
||||
output_format->finalize();
|
||||
}
|
||||
|
||||
resetOutput();
|
||||
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include "config.h"
|
||||
|
||||
#if USE_REPLXX
|
||||
# include <base/ReplxxLineReader.h>
|
||||
# include <Client/ReplxxLineReader.h>
|
||||
#endif
|
||||
|
||||
|
||||
|
@ -9,7 +9,6 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/isLocalAddress.h>
|
||||
#include <Common/DNSResolver.h>
|
||||
#include <base/setTerminalEcho.h>
|
||||
#include <base/scope_guard.h>
|
||||
|
||||
#include <readpassphrase/readpassphrase.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <base/LineReader.h>
|
||||
#include <Client/LineReader.h>
|
||||
|
||||
#include <iostream>
|
||||
#include <string_view>
|
||||
@ -65,6 +65,9 @@ void addNewWords(Words & to, const Words & from, Compare comp)
|
||||
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
replxx::Replxx::completions_t LineReader::Suggest::getCompletions(const String & prefix, size_t prefix_length)
|
||||
{
|
||||
std::string_view last_word;
|
||||
@ -202,3 +205,5 @@ LineReader::InputStatus LineReader::readOneLine(const String & prompt)
|
||||
trim(input);
|
||||
return INPUT_LINE;
|
||||
}
|
||||
|
||||
}
|
@ -9,6 +9,9 @@
|
||||
#include <base/types.h>
|
||||
#include <base/defines.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class LineReader
|
||||
{
|
||||
public:
|
||||
@ -68,3 +71,5 @@ protected:
|
||||
virtual InputStatus readOneLine(const String & prompt);
|
||||
virtual void addToHistory(const String &) {}
|
||||
};
|
||||
|
||||
}
|
@ -1,6 +1,10 @@
|
||||
#include <base/ReplxxLineReader.h>
|
||||
#include <Client/ReplxxLineReader.h>
|
||||
#include <base/errnoToString.h>
|
||||
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/copyData.h>
|
||||
|
||||
#include <stdexcept>
|
||||
#include <chrono>
|
||||
#include <cerrno>
|
||||
@ -108,13 +112,11 @@ void writeRetry(int fd, const std::string & data)
|
||||
}
|
||||
std::string readFile(const std::string & path)
|
||||
{
|
||||
std::ifstream t(path);
|
||||
std::string str;
|
||||
t.seekg(0, std::ios::end);
|
||||
str.reserve(t.tellg());
|
||||
t.seekg(0, std::ios::beg);
|
||||
str.assign((std::istreambuf_iterator<char>(t)), std::istreambuf_iterator<char>());
|
||||
return str;
|
||||
std::string out;
|
||||
DB::WriteBufferFromString out_buffer(out);
|
||||
DB::ReadBufferFromFile in_buffer(path);
|
||||
DB::copyData(in_buffer, out_buffer);
|
||||
return out;
|
||||
}
|
||||
|
||||
/// Simple wrapper for temporary files.
|
||||
@ -269,6 +271,9 @@ void convertHistoryFile(const std::string & path, replxx::Replxx & rx)
|
||||
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static bool replxx_last_is_delimiter = false;
|
||||
void ReplxxLineReader::setLastIsDelimiter(bool flag)
|
||||
{
|
||||
@ -508,3 +513,5 @@ void ReplxxLineReader::enableBracketedPaste()
|
||||
bracketed_paste_enabled = true;
|
||||
rx.enable_bracketed_paste();
|
||||
}
|
||||
|
||||
}
|
@ -1,9 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#include "LineReader.h"
|
||||
|
||||
#include <replxx.hxx>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ReplxxLineReader : public LineReader
|
||||
{
|
||||
public:
|
||||
@ -36,3 +38,5 @@ private:
|
||||
|
||||
std::string editor;
|
||||
};
|
||||
|
||||
}
|
@ -5,8 +5,8 @@
|
||||
#include <Client/Connection.h>
|
||||
#include <Client/IServerConnection.h>
|
||||
#include <Client/LocalConnection.h>
|
||||
#include <Client/LineReader.h>
|
||||
#include <IO/ConnectionTimeouts.h>
|
||||
#include <base/LineReader.h>
|
||||
#include <thread>
|
||||
|
||||
|
||||
|
@ -1,16 +0,0 @@
|
||||
#pragma once
|
||||
#include <cstddef>
|
||||
|
||||
/// 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;
|
||||
};
|
@ -92,10 +92,8 @@ public:
|
||||
void * alloc(size_t size, size_t alignment = 0)
|
||||
{
|
||||
checkSize(size);
|
||||
auto trace = CurrentMemoryTracker::alloc(size);
|
||||
void * ptr = allocNoTrack(size, alignment);
|
||||
trace.onAlloc(ptr, size);
|
||||
return ptr;
|
||||
CurrentMemoryTracker::alloc(size);
|
||||
return allocNoTrack(size, alignment);
|
||||
}
|
||||
|
||||
/// Free memory range.
|
||||
@ -105,8 +103,7 @@ public:
|
||||
{
|
||||
checkSize(size);
|
||||
freeNoTrack(buf, size);
|
||||
auto trace = CurrentMemoryTracker::free(size);
|
||||
trace.onFree(buf, size);
|
||||
CurrentMemoryTracker::free(size);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -132,16 +129,13 @@ 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);
|
||||
CurrentMemoryTracker::realloc(old_size, new_size);
|
||||
|
||||
void * new_buf = ::realloc(buf, new_size);
|
||||
if (nullptr == new_buf)
|
||||
DB::throwFromErrno(fmt::format("Allocator: Cannot realloc from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
|
||||
|
||||
buf = new_buf;
|
||||
trace.onAlloc(buf, new_size);
|
||||
|
||||
if constexpr (clear_memory)
|
||||
if (new_size > old_size)
|
||||
memset(reinterpret_cast<char *>(buf) + old_size, 0, new_size - old_size);
|
||||
@ -149,8 +143,7 @@ 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);
|
||||
CurrentMemoryTracker::realloc(old_size, new_size);
|
||||
|
||||
// On apple and freebsd self-implemented mremap used (common/mremap.h)
|
||||
buf = clickhouse_mremap(buf, old_size, new_size, MREMAP_MAYMOVE,
|
||||
@ -159,17 +152,14 @@ 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.
|
||||
auto trace = CurrentMemoryTracker::realloc(old_size, new_size);
|
||||
trace.onFree(buf, old_size);
|
||||
CurrentMemoryTracker::realloc(old_size, new_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;
|
||||
|
@ -30,24 +30,21 @@ struct AllocatorWithMemoryTracking
|
||||
throw std::bad_alloc();
|
||||
|
||||
size_t bytes = n * sizeof(T);
|
||||
auto trace = CurrentMemoryTracker::alloc(bytes);
|
||||
CurrentMemoryTracker::alloc(bytes);
|
||||
|
||||
T * p = static_cast<T *>(malloc(bytes));
|
||||
if (!p)
|
||||
throw std::bad_alloc();
|
||||
|
||||
trace.onAlloc(p, bytes);
|
||||
|
||||
return p;
|
||||
}
|
||||
|
||||
void deallocate(T * p, size_t n) noexcept
|
||||
{
|
||||
size_t bytes = n * sizeof(T);
|
||||
|
||||
free(p);
|
||||
auto trace = CurrentMemoryTracker::free(bytes);
|
||||
trace.onFree(p, bytes);
|
||||
|
||||
size_t bytes = n * sizeof(T);
|
||||
CurrentMemoryTracker::free(bytes);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -37,7 +37,7 @@ MemoryTracker * getMemoryTracker()
|
||||
|
||||
using DB::current_thread;
|
||||
|
||||
AllocationTrace CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded)
|
||||
void 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,9 +55,8 @@ AllocationTrace CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory
|
||||
|
||||
if (will_be > current_thread->untracked_memory_limit)
|
||||
{
|
||||
auto res = memory_tracker->allocImpl(will_be, throw_if_memory_exceeded);
|
||||
memory_tracker->allocImpl(will_be, throw_if_memory_exceeded);
|
||||
current_thread->untracked_memory = 0;
|
||||
return res;
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -69,40 +68,36 @@ AllocationTrace CurrentMemoryTracker::allocImpl(Int64 size, bool throw_if_memory
|
||||
/// total_memory_tracker only, ignore untracked_memory
|
||||
else
|
||||
{
|
||||
return memory_tracker->allocImpl(size, throw_if_memory_exceeded);
|
||||
memory_tracker->allocImpl(size, throw_if_memory_exceeded);
|
||||
}
|
||||
|
||||
return AllocationTrace(memory_tracker->getSampleProbability());
|
||||
}
|
||||
|
||||
return AllocationTrace(0);
|
||||
}
|
||||
|
||||
void CurrentMemoryTracker::check()
|
||||
{
|
||||
if (auto * memory_tracker = getMemoryTracker())
|
||||
std::ignore = memory_tracker->allocImpl(0, true);
|
||||
memory_tracker->allocImpl(0, true);
|
||||
}
|
||||
|
||||
AllocationTrace CurrentMemoryTracker::alloc(Int64 size)
|
||||
void CurrentMemoryTracker::alloc(Int64 size)
|
||||
{
|
||||
bool throw_if_memory_exceeded = true;
|
||||
return allocImpl(size, throw_if_memory_exceeded);
|
||||
allocImpl(size, throw_if_memory_exceeded);
|
||||
}
|
||||
|
||||
AllocationTrace CurrentMemoryTracker::allocNoThrow(Int64 size)
|
||||
void CurrentMemoryTracker::allocNoThrow(Int64 size)
|
||||
{
|
||||
bool throw_if_memory_exceeded = false;
|
||||
return allocImpl(size, throw_if_memory_exceeded);
|
||||
allocImpl(size, throw_if_memory_exceeded);
|
||||
}
|
||||
|
||||
AllocationTrace CurrentMemoryTracker::realloc(Int64 old_size, Int64 new_size)
|
||||
void CurrentMemoryTracker::realloc(Int64 old_size, Int64 new_size)
|
||||
{
|
||||
Int64 addition = new_size - old_size;
|
||||
return addition > 0 ? alloc(addition) : free(-addition);
|
||||
addition > 0 ? alloc(addition) : free(-addition);
|
||||
}
|
||||
|
||||
AllocationTrace CurrentMemoryTracker::free(Int64 size)
|
||||
void CurrentMemoryTracker::free(Int64 size)
|
||||
{
|
||||
if (auto * memory_tracker = getMemoryTracker())
|
||||
{
|
||||
@ -111,20 +106,15 @@ AllocationTrace CurrentMemoryTracker::free(Int64 size)
|
||||
current_thread->untracked_memory -= size;
|
||||
if (current_thread->untracked_memory < -current_thread->untracked_memory_limit)
|
||||
{
|
||||
Int64 untracked_memory = current_thread->untracked_memory;
|
||||
memory_tracker->free(-current_thread->untracked_memory);
|
||||
current_thread->untracked_memory = 0;
|
||||
return memory_tracker->free(-untracked_memory);
|
||||
}
|
||||
}
|
||||
/// total_memory_tracker only, ignore untracked_memory
|
||||
else
|
||||
{
|
||||
return memory_tracker->free(size);
|
||||
memory_tracker->free(size);
|
||||
}
|
||||
|
||||
return AllocationTrace(memory_tracker->getSampleProbability());
|
||||
}
|
||||
|
||||
return AllocationTrace(0);
|
||||
}
|
||||
|
||||
|
@ -1,20 +1,19 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
#include <Common/AllocationTrace.h>
|
||||
|
||||
/// 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.
|
||||
[[nodiscard]] static AllocationTrace alloc(Int64 size);
|
||||
[[nodiscard]] static AllocationTrace allocNoThrow(Int64 size);
|
||||
[[nodiscard]] static AllocationTrace realloc(Int64 old_size, Int64 new_size);
|
||||
static void alloc(Int64 size);
|
||||
static void allocNoThrow(Int64 size);
|
||||
static void realloc(Int64 old_size, Int64 new_size);
|
||||
|
||||
/// This function should be called after memory deallocation.
|
||||
[[nodiscard]] static AllocationTrace free(Int64 size);
|
||||
static void free(Int64 size);
|
||||
static void check();
|
||||
|
||||
private:
|
||||
[[nodiscard]] static AllocationTrace allocImpl(Int64 size, bool throw_if_memory_exceeded);
|
||||
static void allocImpl(Int64 size, bool throw_if_memory_exceeded);
|
||||
};
|
||||
|
@ -57,8 +57,7 @@ public:
|
||||
}
|
||||
|
||||
/// Do not count guard page in memory usage.
|
||||
auto trace = CurrentMemoryTracker::alloc(num_pages * page_size);
|
||||
trace.onAlloc(vp, num_pages * page_size);
|
||||
CurrentMemoryTracker::alloc(num_pages * page_size);
|
||||
|
||||
boost::context::stack_context sctx;
|
||||
sctx.size = num_bytes;
|
||||
@ -78,7 +77,6 @@ public:
|
||||
::munmap(vp, sctx.size);
|
||||
|
||||
/// Do not count guard page in memory usage.
|
||||
auto trace = CurrentMemoryTracker::free(sctx.size - page_size);
|
||||
trace.onFree(vp, sctx.size - page_size);
|
||||
CurrentMemoryTracker::free(sctx.size - page_size);
|
||||
}
|
||||
};
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include "MemoryTracker.h"
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/VariableContext.h>
|
||||
#include <Common/TraceSender.h>
|
||||
#include <Common/Exception.h>
|
||||
@ -83,53 +82,6 @@ inline std::string_view toDescription(OvercommitResult result)
|
||||
}
|
||||
}
|
||||
|
||||
bool shouldTrackAllocation(DB::Float64 probability, void * ptr)
|
||||
{
|
||||
return sipHash64(uintptr_t(ptr)) < std::numeric_limits<uint64_t>::max() * probability;
|
||||
}
|
||||
|
||||
AllocationTrace updateAllocationTrace(AllocationTrace trace, const std::optional<double> & sample_probability)
|
||||
{
|
||||
if (unlikely(sample_probability))
|
||||
return AllocationTrace(*sample_probability);
|
||||
|
||||
return trace;
|
||||
}
|
||||
|
||||
AllocationTrace getAllocationTrace(std::optional<double> & 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
|
||||
@ -183,7 +135,7 @@ void MemoryTracker::logMemoryUsage(Int64 current) const
|
||||
}
|
||||
|
||||
|
||||
AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker)
|
||||
void 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);
|
||||
@ -202,14 +154,9 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed
|
||||
|
||||
/// Since the MemoryTrackerBlockerInThread should respect the level, we should go to the next parent.
|
||||
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 getAllocationTrace(sample_probability);
|
||||
loaded_next->allocImpl(size, throw_if_memory_exceeded,
|
||||
level == VariableContext::Process ? this : query_tracker);
|
||||
return;
|
||||
}
|
||||
|
||||
/** Using memory_order_relaxed means that if allocations are done simultaneously,
|
||||
@ -236,6 +183,14 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed
|
||||
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)))
|
||||
{
|
||||
@ -354,22 +309,16 @@ 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 getAllocationTrace(sample_probability);
|
||||
loaded_next->allocImpl(size, throw_if_memory_exceeded,
|
||||
level == VariableContext::Process ? this : query_tracker);
|
||||
}
|
||||
|
||||
void MemoryTracker::adjustWithUntrackedMemory(Int64 untracked_memory)
|
||||
{
|
||||
if (untracked_memory > 0)
|
||||
std::ignore = allocImpl(untracked_memory, /*throw_if_memory_exceeded*/ false);
|
||||
allocImpl(untracked_memory, /*throw_if_memory_exceeded*/ false);
|
||||
else
|
||||
std::ignore = free(-untracked_memory);
|
||||
free(-untracked_memory);
|
||||
}
|
||||
|
||||
bool MemoryTracker::updatePeak(Int64 will_be, bool log_memory_usage)
|
||||
@ -388,7 +337,8 @@ bool MemoryTracker::updatePeak(Int64 will_be, bool log_memory_usage)
|
||||
return false;
|
||||
}
|
||||
|
||||
AllocationTrace MemoryTracker::free(Int64 size)
|
||||
|
||||
void MemoryTracker::free(Int64 size)
|
||||
{
|
||||
if (MemoryTrackerBlockerInThread::isBlocked(level))
|
||||
{
|
||||
@ -403,9 +353,15 @@ 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);
|
||||
loaded_next->free(size);
|
||||
return;
|
||||
}
|
||||
|
||||
return getAllocationTrace(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});
|
||||
}
|
||||
|
||||
Int64 accounted_size = size;
|
||||
@ -433,15 +389,12 @@ 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);
|
||||
loaded_next->free(size);
|
||||
|
||||
auto metric_loaded = metric.load(std::memory_order_relaxed);
|
||||
if (metric_loaded != CurrentMetrics::end())
|
||||
CurrentMetrics::sub(metric_loaded, accounted_size);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
@ -525,14 +478,3 @@ 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;
|
||||
}
|
||||
|
@ -2,11 +2,9 @@
|
||||
|
||||
#include <atomic>
|
||||
#include <chrono>
|
||||
#include <optional>
|
||||
#include <base/types.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/VariableContext.h>
|
||||
#include <Common/AllocationTrace.h>
|
||||
|
||||
#if !defined(NDEBUG)
|
||||
#define MEMORY_TRACKER_DEBUG_CHECKS
|
||||
@ -67,7 +65,7 @@ private:
|
||||
double fault_probability = 0;
|
||||
|
||||
/// To randomly sample allocations and deallocations in trace_log.
|
||||
std::optional<double> sample_probability;
|
||||
double sample_probability = 0;
|
||||
|
||||
/// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy).
|
||||
/// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker.
|
||||
@ -92,8 +90,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);
|
||||
void allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryTracker * query_tracker = nullptr);
|
||||
void free(Int64 size);
|
||||
public:
|
||||
|
||||
static constexpr auto USAGE_EVENT_NAME = "MemoryTrackerUsage";
|
||||
@ -148,8 +146,6 @@ public:
|
||||
sample_probability = value;
|
||||
}
|
||||
|
||||
double getSampleProbability();
|
||||
|
||||
void setProfilerStep(Int64 value)
|
||||
{
|
||||
profiler_step = value;
|
||||
|
@ -28,5 +28,4 @@ public:
|
||||
}
|
||||
|
||||
friend class MemoryTracker;
|
||||
friend struct AllocationTrace;
|
||||
};
|
||||
|
@ -72,11 +72,11 @@ ShellCommand::~ShellCommand()
|
||||
if (process_terminated_normally)
|
||||
return;
|
||||
|
||||
LOG_TRACE(getLogger(), "Will kill shell command pid {} with SIGTERM", pid);
|
||||
LOG_TRACE(getLogger(), "Will kill shell command pid {} with signal {}", pid, config.terminate_in_destructor_strategy.termination_signal);
|
||||
|
||||
int retcode = kill(pid, SIGTERM);
|
||||
int retcode = kill(pid, config.terminate_in_destructor_strategy.termination_signal);
|
||||
if (retcode != 0)
|
||||
LOG_WARNING(getLogger(), "Cannot kill shell command pid {} errno '{}'", pid, errnoToString());
|
||||
LOG_WARNING(getLogger(), "Cannot kill shell command pid {}, error: '{}'", pid, errnoToString());
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -27,18 +27,18 @@ namespace DB
|
||||
class ShellCommand final
|
||||
{
|
||||
public:
|
||||
|
||||
~ShellCommand();
|
||||
|
||||
struct DestructorStrategy final
|
||||
{
|
||||
explicit DestructorStrategy(bool terminate_in_destructor_, size_t wait_for_normal_exit_before_termination_seconds_ = 0)
|
||||
: terminate_in_destructor(terminate_in_destructor_)
|
||||
explicit DestructorStrategy(bool terminate_in_destructor_, int termination_signal_, size_t wait_for_normal_exit_before_termination_seconds_ = 0)
|
||||
: terminate_in_destructor(terminate_in_destructor_), termination_signal(termination_signal_)
|
||||
, wait_for_normal_exit_before_termination_seconds(wait_for_normal_exit_before_termination_seconds_)
|
||||
{
|
||||
}
|
||||
|
||||
bool terminate_in_destructor;
|
||||
int termination_signal;
|
||||
|
||||
/// If terminate in destructor is true, command will wait until send SIGTERM signal to created process
|
||||
size_t wait_for_normal_exit_before_termination_seconds = 0;
|
||||
@ -64,7 +64,7 @@ public:
|
||||
|
||||
bool pipe_stdin_only = false;
|
||||
|
||||
DestructorStrategy terminate_in_destructor_strategy = DestructorStrategy(false);
|
||||
DestructorStrategy terminate_in_destructor_strategy = DestructorStrategy(false, 0);
|
||||
};
|
||||
|
||||
/// Run the command using /bin/sh -c.
|
||||
|
@ -40,6 +40,10 @@ public:
|
||||
* Pass CLOCK_MONOTONIC_COARSE, if you need better performance with acceptable cost of several milliseconds of inaccuracy.
|
||||
*/
|
||||
explicit Stopwatch(clockid_t clock_type_ = CLOCK_MONOTONIC) : clock_type(clock_type_) { start(); }
|
||||
explicit Stopwatch(clockid_t clock_type_, UInt64 start_nanoseconds, bool is_running_)
|
||||
: start_ns(start_nanoseconds), clock_type(clock_type_), is_running(is_running_)
|
||||
{
|
||||
}
|
||||
|
||||
void start() { start_ns = nanoseconds(); is_running = true; }
|
||||
void stop() { stop_ns = nanoseconds(); is_running = false; }
|
||||
@ -51,6 +55,8 @@ public:
|
||||
UInt64 elapsedMilliseconds() const { return elapsedNanoseconds() / 1000000UL; }
|
||||
double elapsedSeconds() const { return static_cast<double>(elapsedNanoseconds()) / 1000000000ULL; }
|
||||
|
||||
UInt64 getStart() { return start_ns; }
|
||||
|
||||
private:
|
||||
UInt64 start_ns = 0;
|
||||
UInt64 stop_ns = 0;
|
||||
|
@ -33,7 +33,6 @@ 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
|
||||
|
||||
@ -75,7 +74,6 @@ 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);
|
||||
|
||||
|
@ -28,9 +28,8 @@ class TraceSender
|
||||
public:
|
||||
struct Extras
|
||||
{
|
||||
/// size, ptr - for memory tracing is the amount of memory allocated; for other trace types it is 0.
|
||||
/// size - 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{};
|
||||
|
@ -342,7 +342,6 @@ ZooKeeper::ZooKeeper(
|
||||
default_acls.emplace_back(std::move(acl));
|
||||
}
|
||||
|
||||
|
||||
/// It makes sense (especially, for async requests) to inject a fault in two places:
|
||||
/// pushRequest (before request is sent) and receiveEvent (after request was executed).
|
||||
if (0 < args.send_fault_probability && args.send_fault_probability <= 1)
|
||||
|
@ -9,11 +9,7 @@ extern "C" void * clickhouse_malloc(size_t size)
|
||||
{
|
||||
void * res = malloc(size);
|
||||
if (res)
|
||||
{
|
||||
AllocationTrace trace;
|
||||
size_t actual_size = Memory::trackMemory(size, trace);
|
||||
trace.onAlloc(res, actual_size);
|
||||
}
|
||||
Memory::trackMemory(size);
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -21,29 +17,17 @@ extern "C" void * clickhouse_calloc(size_t number_of_members, size_t size)
|
||||
{
|
||||
void * res = calloc(number_of_members, size);
|
||||
if (res)
|
||||
{
|
||||
AllocationTrace trace;
|
||||
size_t actual_size = Memory::trackMemory(number_of_members * size, trace);
|
||||
trace.onAlloc(res, actual_size);
|
||||
}
|
||||
Memory::trackMemory(number_of_members * size);
|
||||
return res;
|
||||
}
|
||||
|
||||
extern "C" void * clickhouse_realloc(void * ptr, size_t size)
|
||||
{
|
||||
if (ptr)
|
||||
{
|
||||
AllocationTrace trace;
|
||||
size_t actual_size = Memory::untrackMemory(ptr, trace);
|
||||
trace.onFree(ptr, actual_size);
|
||||
}
|
||||
Memory::untrackMemory(ptr);
|
||||
void * res = realloc(ptr, size);
|
||||
if (res)
|
||||
{
|
||||
AllocationTrace trace;
|
||||
size_t actual_size = Memory::trackMemory(size, trace);
|
||||
trace.onAlloc(res, actual_size);
|
||||
}
|
||||
Memory::trackMemory(size);
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -58,9 +42,7 @@ extern "C" void * clickhouse_reallocarray(void * ptr, size_t number_of_members,
|
||||
|
||||
extern "C" void clickhouse_free(void * ptr)
|
||||
{
|
||||
AllocationTrace trace;
|
||||
size_t actual_size = Memory::untrackMemory(ptr, trace);
|
||||
trace.onFree(ptr, actual_size);
|
||||
Memory::untrackMemory(ptr);
|
||||
free(ptr);
|
||||
}
|
||||
|
||||
@ -68,10 +50,6 @@ extern "C" int clickhouse_posix_memalign(void ** memptr, size_t alignment, size_
|
||||
{
|
||||
int res = posix_memalign(memptr, alignment, size);
|
||||
if (res == 0)
|
||||
{
|
||||
AllocationTrace trace;
|
||||
size_t actual_size = Memory::trackMemory(size, trace);
|
||||
trace.onAlloc(*memptr, actual_size);
|
||||
}
|
||||
Memory::trackMemory(size);
|
||||
return res;
|
||||
}
|
||||
|
@ -112,19 +112,16 @@ inline ALWAYS_INLINE size_t getActualAllocationSize(size_t size, TAlign... align
|
||||
|
||||
template <std::same_as<std::align_val_t>... TAlign>
|
||||
requires DB::OptionalArgument<TAlign...>
|
||||
inline ALWAYS_INLINE size_t trackMemory(std::size_t size, AllocationTrace & trace, TAlign... align)
|
||||
inline ALWAYS_INLINE void trackMemory(std::size_t size, TAlign... align)
|
||||
{
|
||||
std::size_t actual_size = getActualAllocationSize(size, align...);
|
||||
trace = CurrentMemoryTracker::allocNoThrow(actual_size);
|
||||
return actual_size;
|
||||
CurrentMemoryTracker::allocNoThrow(actual_size);
|
||||
}
|
||||
|
||||
template <std::same_as<std::align_val_t>... TAlign>
|
||||
requires DB::OptionalArgument<TAlign...>
|
||||
inline ALWAYS_INLINE size_t untrackMemory(void * ptr [[maybe_unused]], AllocationTrace & trace, std::size_t size [[maybe_unused]] = 0, TAlign... align [[maybe_unused]]) noexcept
|
||||
inline ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t size [[maybe_unused]] = 0, TAlign... align [[maybe_unused]]) noexcept
|
||||
{
|
||||
std::size_t actual_size = 0;
|
||||
|
||||
try
|
||||
{
|
||||
#if USE_JEMALLOC
|
||||
@ -133,26 +130,23 @@ inline ALWAYS_INLINE size_t untrackMemory(void * ptr [[maybe_unused]], Allocatio
|
||||
if (likely(ptr != nullptr))
|
||||
{
|
||||
if constexpr (sizeof...(TAlign) == 1)
|
||||
actual_size = sallocx(ptr, MALLOCX_ALIGN(alignToSizeT(align...)));
|
||||
CurrentMemoryTracker::free(sallocx(ptr, MALLOCX_ALIGN(alignToSizeT(align...))));
|
||||
else
|
||||
actual_size = sallocx(ptr, 0);
|
||||
CurrentMemoryTracker::free(sallocx(ptr, 0));
|
||||
}
|
||||
#else
|
||||
if (size)
|
||||
actual_size = size;
|
||||
CurrentMemoryTracker::free(size);
|
||||
# if defined(_GNU_SOURCE)
|
||||
/// It's innaccurate resource free for sanitizers. malloc_usable_size() result is greater or equal to allocated size.
|
||||
else
|
||||
actual_size = malloc_usable_size(ptr);
|
||||
CurrentMemoryTracker::free(malloc_usable_size(ptr));
|
||||
# endif
|
||||
#endif
|
||||
trace = CurrentMemoryTracker::free(actual_size);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
}
|
||||
|
||||
return actual_size;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -50,74 +50,50 @@ static struct InitializeJemallocZoneAllocatorForOSX
|
||||
|
||||
void * operator new(std::size_t size)
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::trackMemory(size, trace);
|
||||
void * ptr = Memory::newImpl(size);
|
||||
trace.onAlloc(ptr, actual_size);
|
||||
return ptr;
|
||||
Memory::trackMemory(size);
|
||||
return Memory::newImpl(size);
|
||||
}
|
||||
|
||||
void * operator new(std::size_t size, std::align_val_t 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;
|
||||
Memory::trackMemory(size, align);
|
||||
return Memory::newImpl(size, align);
|
||||
}
|
||||
|
||||
void * operator new[](std::size_t size)
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::trackMemory(size, trace);
|
||||
void * ptr = Memory::newImpl(size);
|
||||
trace.onAlloc(ptr, actual_size);
|
||||
return ptr;
|
||||
Memory::trackMemory(size);
|
||||
return Memory::newImpl(size);
|
||||
}
|
||||
|
||||
void * operator new[](std::size_t size, std::align_val_t 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;
|
||||
Memory::trackMemory(size, align);
|
||||
return Memory::newImpl(size, align);
|
||||
}
|
||||
|
||||
void * operator new(std::size_t size, const std::nothrow_t &) noexcept
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::trackMemory(size, trace);
|
||||
void * ptr = Memory::newNoExept(size);
|
||||
trace.onAlloc(ptr, actual_size);
|
||||
return ptr;
|
||||
Memory::trackMemory(size);
|
||||
return Memory::newNoExept(size);
|
||||
}
|
||||
|
||||
void * operator new[](std::size_t size, const std::nothrow_t &) noexcept
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::trackMemory(size, trace);
|
||||
void * ptr = Memory::newNoExept(size);
|
||||
trace.onAlloc(ptr, actual_size);
|
||||
return ptr;
|
||||
Memory::trackMemory(size);
|
||||
return Memory::newNoExept(size);
|
||||
}
|
||||
|
||||
void * operator new(std::size_t size, std::align_val_t align, const std::nothrow_t &) noexcept
|
||||
{
|
||||
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;
|
||||
Memory::trackMemory(size, align);
|
||||
return Memory::newNoExept(size, align);
|
||||
}
|
||||
|
||||
void * operator new[](std::size_t size, std::align_val_t align, const std::nothrow_t &) noexcept
|
||||
{
|
||||
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;
|
||||
Memory::trackMemory(size, align);
|
||||
return Memory::newNoExept(size, align);
|
||||
}
|
||||
|
||||
/// delete
|
||||
@ -133,64 +109,48 @@ void * operator new[](std::size_t size, std::align_val_t align, const std::nothr
|
||||
|
||||
void operator delete(void * ptr) noexcept
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::untrackMemory(ptr, trace);
|
||||
trace.onFree(ptr, actual_size);
|
||||
Memory::untrackMemory(ptr);
|
||||
Memory::deleteImpl(ptr);
|
||||
}
|
||||
|
||||
void operator delete(void * ptr, std::align_val_t align) noexcept
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::untrackMemory(ptr, trace, 0, align);
|
||||
trace.onFree(ptr, actual_size);
|
||||
Memory::untrackMemory(ptr, 0, align);
|
||||
Memory::deleteImpl(ptr);
|
||||
}
|
||||
|
||||
void operator delete[](void * ptr) noexcept
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::untrackMemory(ptr, trace);
|
||||
trace.onFree(ptr, actual_size);
|
||||
Memory::untrackMemory(ptr);
|
||||
Memory::deleteImpl(ptr);
|
||||
}
|
||||
|
||||
void operator delete[](void * ptr, std::align_val_t align) noexcept
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::untrackMemory(ptr, trace, 0, align);
|
||||
trace.onFree(ptr, actual_size);
|
||||
Memory::untrackMemory(ptr, 0, align);
|
||||
Memory::deleteImpl(ptr);
|
||||
}
|
||||
|
||||
void operator delete(void * ptr, std::size_t size) noexcept
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::untrackMemory(ptr, trace, size);
|
||||
trace.onFree(ptr, actual_size);
|
||||
Memory::untrackMemory(ptr, size);
|
||||
Memory::deleteSized(ptr, size);
|
||||
}
|
||||
|
||||
void operator delete(void * ptr, std::size_t size, std::align_val_t align) noexcept
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::untrackMemory(ptr, trace, size, align);
|
||||
trace.onFree(ptr, actual_size);
|
||||
Memory::untrackMemory(ptr, size, align);
|
||||
Memory::deleteSized(ptr, size, align);
|
||||
}
|
||||
|
||||
void operator delete[](void * ptr, std::size_t size) noexcept
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::untrackMemory(ptr, trace, size);
|
||||
trace.onFree(ptr, actual_size);
|
||||
Memory::untrackMemory(ptr, size);
|
||||
Memory::deleteSized(ptr, size);
|
||||
}
|
||||
|
||||
void operator delete[](void * ptr, std::size_t size, std::align_val_t align) noexcept
|
||||
{
|
||||
AllocationTrace trace;
|
||||
std::size_t actual_size = Memory::untrackMemory(ptr, trace, size, align);
|
||||
trace.onFree(ptr, actual_size);
|
||||
Memory::untrackMemory(ptr, size, align);
|
||||
Memory::deleteSized(ptr, size, align);
|
||||
}
|
||||
|
@ -145,13 +145,9 @@ MongoDBDictionarySource::MongoDBDictionarySource(
|
||||
connection->connect(host, port);
|
||||
if (!user.empty())
|
||||
{
|
||||
#if POCO_VERSION >= 0x01070800
|
||||
Poco::MongoDB::Database poco_db(db);
|
||||
if (!poco_db.authenticate(*connection, user, password, method.empty() ? Poco::MongoDB::Database::AUTH_SCRAM_SHA1 : method))
|
||||
throw Exception(ErrorCodes::MONGODB_CANNOT_AUTHENTICATE, "Cannot authenticate in MongoDB, incorrect user or password");
|
||||
#else
|
||||
authenticate(*connection, db, user, password);
|
||||
#endif
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -317,6 +317,9 @@ static void addExistingProgressToOutputFormat(OutputFormatPtr format, ContextPtr
|
||||
auto current_progress = element_id->getProgressIn();
|
||||
Progress read_progress{current_progress.read_rows, current_progress.read_bytes, current_progress.total_rows_to_read};
|
||||
format->onProgress(read_progress);
|
||||
|
||||
/// Update the start of the statistics to use the start of the query, and not the creation of the format class
|
||||
format->setStartTime(element_id->getQueryCPUStartTime(), true);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -69,7 +69,8 @@ static bool isUnlimitedQuery(const IAST * ast)
|
||||
}
|
||||
|
||||
|
||||
ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr query_context)
|
||||
ProcessList::EntryPtr
|
||||
ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr query_context, UInt64 watch_start_nanoseconds)
|
||||
{
|
||||
EntryPtr res;
|
||||
|
||||
@ -243,13 +244,16 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as
|
||||
/// since allocation and deallocation could happen in different threads
|
||||
}
|
||||
|
||||
auto process_it = processes.emplace(processes.end(), std::make_shared<QueryStatus>(
|
||||
query_context,
|
||||
query_,
|
||||
client_info,
|
||||
priorities.insert(static_cast<int>(settings.priority)),
|
||||
std::move(thread_group),
|
||||
query_kind));
|
||||
auto process_it = processes.emplace(
|
||||
processes.end(),
|
||||
std::make_shared<QueryStatus>(
|
||||
query_context,
|
||||
query_,
|
||||
client_info,
|
||||
priorities.insert(static_cast<int>(settings.priority)),
|
||||
std::move(thread_group),
|
||||
query_kind,
|
||||
watch_start_nanoseconds));
|
||||
|
||||
increaseQueryKindAmount(query_kind);
|
||||
|
||||
@ -344,11 +348,13 @@ QueryStatus::QueryStatus(
|
||||
const ClientInfo & client_info_,
|
||||
QueryPriorities::Handle && priority_handle_,
|
||||
ThreadGroupStatusPtr && thread_group_,
|
||||
IAST::QueryKind query_kind_)
|
||||
IAST::QueryKind query_kind_,
|
||||
UInt64 watch_start_nanoseconds)
|
||||
: WithContext(context_)
|
||||
, query(query_)
|
||||
, client_info(client_info_)
|
||||
, thread_group(std::move(thread_group_))
|
||||
, watch(CLOCK_MONOTONIC, watch_start_nanoseconds, true)
|
||||
, priority_handle(std::move(priority_handle_))
|
||||
, global_overcommit_tracker(context_->getGlobalOvercommitTracker())
|
||||
, query_kind(query_kind_)
|
||||
@ -522,7 +528,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even
|
||||
|
||||
res.query = query;
|
||||
res.client_info = client_info;
|
||||
res.elapsed_seconds = watch.elapsedSeconds();
|
||||
res.elapsed_microseconds = watch.elapsedMicroseconds();
|
||||
res.is_cancelled = is_killed.load(std::memory_order_relaxed);
|
||||
res.is_all_data_sent = is_all_data_sent.load(std::memory_order_relaxed);
|
||||
res.read_rows = progress_in.read_rows;
|
||||
|
@ -53,7 +53,7 @@ class ProcessListEntry;
|
||||
struct QueryStatusInfo
|
||||
{
|
||||
String query;
|
||||
double elapsed_seconds;
|
||||
UInt64 elapsed_microseconds;
|
||||
size_t read_rows;
|
||||
size_t read_bytes;
|
||||
size_t total_rows;
|
||||
@ -142,15 +142,14 @@ protected:
|
||||
CurrentMetrics::Increment num_queries_increment;
|
||||
|
||||
public:
|
||||
|
||||
QueryStatus(
|
||||
ContextPtr context_,
|
||||
const String & query_,
|
||||
const ClientInfo & client_info_,
|
||||
QueryPriorities::Handle && priority_handle_,
|
||||
ThreadGroupStatusPtr && thread_group_,
|
||||
IAST::QueryKind query_kind_
|
||||
);
|
||||
IAST::QueryKind query_kind_,
|
||||
UInt64 watch_start_nanoseconds);
|
||||
|
||||
~QueryStatus();
|
||||
|
||||
@ -221,6 +220,9 @@ public:
|
||||
bool checkTimeLimit();
|
||||
/// Same as checkTimeLimit but it never throws
|
||||
[[nodiscard]] bool checkTimeLimitSoft();
|
||||
|
||||
/// Get the reference for the start of the query. Used to synchronize with other Stopwatches
|
||||
UInt64 getQueryCPUStartTime() { return watch.getStart(); }
|
||||
};
|
||||
|
||||
using QueryStatusPtr = std::shared_ptr<QueryStatus>;
|
||||
@ -382,7 +384,7 @@ public:
|
||||
* If timeout is passed - throw an exception.
|
||||
* Don't count KILL QUERY queries.
|
||||
*/
|
||||
EntryPtr insert(const String & query_, const IAST * ast, ContextMutablePtr query_context);
|
||||
EntryPtr insert(const String & query_, const IAST * ast, ContextMutablePtr query_context, UInt64 watch_start_nanoseconds);
|
||||
|
||||
/// Number of currently executing queries.
|
||||
size_t size() const { return processes.size(); }
|
||||
|
@ -97,9 +97,6 @@ void TraceCollector::run()
|
||||
Int64 size;
|
||||
readPODBinary(size, in);
|
||||
|
||||
UInt64 ptr;
|
||||
readPODBinary(ptr, in);
|
||||
|
||||
ProfileEvents::Event event;
|
||||
readPODBinary(event, in);
|
||||
|
||||
@ -115,7 +112,7 @@ void TraceCollector::run()
|
||||
|
||||
UInt64 time = static_cast<UInt64>(ts.tv_sec * 1000000000LL + ts.tv_nsec);
|
||||
UInt64 time_in_microseconds = static_cast<UInt64>((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, ptr, event, increment};
|
||||
TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size, event, increment};
|
||||
trace_log->add(element);
|
||||
}
|
||||
}
|
||||
|
@ -38,7 +38,6 @@ NamesAndTypesList TraceLogElement::getNamesAndTypes()
|
||||
{"query_id", std::make_shared<DataTypeString>()},
|
||||
{"trace", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())},
|
||||
{"size", std::make_shared<DataTypeInt64>()},
|
||||
{"ptr", std::make_shared<DataTypeUInt64>()},
|
||||
{"event", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
|
||||
{"increment", std::make_shared<DataTypeInt64>()},
|
||||
};
|
||||
@ -58,7 +57,6 @@ 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())
|
||||
|
@ -27,10 +27,8 @@ struct TraceLogElement
|
||||
UInt64 thread_id{};
|
||||
String query_id{};
|
||||
Array trace{};
|
||||
/// Allocation size in bytes for TraceType::Memory and TraceType::MemorySample.
|
||||
/// Allocation size in bytes for TraceType::Memory.
|
||||
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.
|
||||
|
@ -201,26 +201,32 @@ static void logException(ContextPtr context, QueryLogElement & elem)
|
||||
elem.stack_trace);
|
||||
}
|
||||
|
||||
static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr context, UInt64 current_time_us, ASTPtr ast, const std::shared_ptr<OpenTelemetry::SpanHolder> & query_span)
|
||||
static void onExceptionBeforeStart(
|
||||
const String & query_for_logging,
|
||||
ContextPtr context,
|
||||
ASTPtr ast,
|
||||
const std::shared_ptr<OpenTelemetry::SpanHolder> & query_span,
|
||||
UInt64 elapsed_millliseconds)
|
||||
{
|
||||
auto query_end_time = std::chrono::system_clock::now();
|
||||
|
||||
/// Exception before the query execution.
|
||||
if (auto quota = context->getQuota())
|
||||
quota->used(QuotaType::ERRORS, 1, /* check_exceeded = */ false);
|
||||
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
const auto & client_info = context->getClientInfo();
|
||||
|
||||
/// Log the start of query execution into the table if necessary.
|
||||
QueryLogElement elem;
|
||||
|
||||
elem.type = QueryLogElementType::EXCEPTION_BEFORE_START;
|
||||
|
||||
// all callers to onExceptionBeforeStart method construct the timespec for event_time and
|
||||
// event_time_microseconds from the same time point. So, it can be assumed that both of these
|
||||
// times are equal up to the precision of a second.
|
||||
elem.event_time = current_time_us / 1000000;
|
||||
elem.event_time_microseconds = current_time_us;
|
||||
elem.query_start_time = current_time_us / 1000000;
|
||||
elem.query_start_time_microseconds = current_time_us;
|
||||
elem.event_time = timeInSeconds(query_end_time);
|
||||
elem.event_time_microseconds = timeInMicroseconds(query_end_time);
|
||||
elem.query_start_time = client_info.initial_query_start_time;
|
||||
elem.query_start_time_microseconds = client_info.initial_query_start_time_microseconds;
|
||||
elem.query_duration_ms = elapsed_millliseconds;
|
||||
|
||||
elem.current_database = context->getCurrentDatabase();
|
||||
elem.query = query_for_logging;
|
||||
@ -324,19 +330,32 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
/// we still have enough span logs for the execution of external queries.
|
||||
std::shared_ptr<OpenTelemetry::SpanHolder> query_span = internal ? nullptr : std::make_shared<OpenTelemetry::SpanHolder>("query");
|
||||
|
||||
const auto current_time = std::chrono::system_clock::now();
|
||||
auto query_start_time = std::chrono::system_clock::now();
|
||||
|
||||
/// Used to set the watch in QueryStatus and the output formats. It is not based on query_start_time as that might be based on
|
||||
/// the value passed by the client
|
||||
Stopwatch start_watch{CLOCK_MONOTONIC};
|
||||
|
||||
auto & client_info = context->getClientInfo();
|
||||
|
||||
// If it's not an internal query and we don't see an initial_query_start_time yet, initialize it
|
||||
// to current time. Internal queries are those executed without an independent client context,
|
||||
// thus should not set initial_query_start_time, because it might introduce data race. It's also
|
||||
// possible to have unset initial_query_start_time for non-internal and non-initial queries. For
|
||||
// example, the query is from an initiator that is running an old version of clickhouse.
|
||||
if (!internal && client_info.initial_query_start_time == 0)
|
||||
if (!internal)
|
||||
{
|
||||
client_info.initial_query_start_time = timeInSeconds(current_time);
|
||||
client_info.initial_query_start_time_microseconds = timeInMicroseconds(current_time);
|
||||
// If it's not an internal query and we don't see an initial_query_start_time yet, initialize it
|
||||
// to current time. Internal queries are those executed without an independent client context,
|
||||
// thus should not set initial_query_start_time, because it might introduce data race. It's also
|
||||
// possible to have unset initial_query_start_time for non-internal and non-initial queries. For
|
||||
// example, the query is from an initiator that is running an old version of clickhouse.
|
||||
// On the other hand, if it's initialized then take it as the start of the query
|
||||
if (client_info.initial_query_start_time == 0)
|
||||
{
|
||||
client_info.initial_query_start_time = timeInSeconds(query_start_time);
|
||||
client_info.initial_query_start_time_microseconds = timeInMicroseconds(query_start_time);
|
||||
}
|
||||
else
|
||||
{
|
||||
query_start_time = std::chrono::time_point<std::chrono::system_clock>(
|
||||
std::chrono::microseconds{client_info.initial_query_start_time_microseconds});
|
||||
}
|
||||
}
|
||||
|
||||
assert(internal || CurrentThread::get().getQueryContext());
|
||||
@ -413,7 +432,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
logQuery(query_for_logging, context, internal, stage);
|
||||
|
||||
if (!internal)
|
||||
onExceptionBeforeStart(query_for_logging, context, timeInMicroseconds(current_time), ast, query_span);
|
||||
onExceptionBeforeStart(query_for_logging, context, ast, query_span, start_watch.elapsedMilliseconds());
|
||||
throw;
|
||||
}
|
||||
|
||||
@ -514,7 +533,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
if (!internal && !ast->as<ASTShowProcesslistQuery>())
|
||||
{
|
||||
/// processlist also has query masked now, to avoid secrets leaks though SHOW PROCESSLIST by other users.
|
||||
process_list_entry = context->getProcessList().insert(query_for_logging, ast.get(), context);
|
||||
process_list_entry = context->getProcessList().insert(query_for_logging, ast.get(), context, start_watch.getStart());
|
||||
context->setProcessListElement(process_list_entry->getQueryStatus());
|
||||
}
|
||||
|
||||
@ -720,10 +739,10 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
|
||||
elem.type = QueryLogElementType::QUERY_START; //-V1048
|
||||
|
||||
elem.event_time = timeInSeconds(current_time);
|
||||
elem.event_time_microseconds = timeInMicroseconds(current_time);
|
||||
elem.query_start_time = timeInSeconds(current_time);
|
||||
elem.query_start_time_microseconds = timeInMicroseconds(current_time);
|
||||
elem.event_time = timeInSeconds(query_start_time);
|
||||
elem.event_time_microseconds = timeInMicroseconds(query_start_time);
|
||||
elem.query_start_time = timeInSeconds(query_start_time);
|
||||
elem.query_start_time_microseconds = timeInMicroseconds(query_start_time);
|
||||
|
||||
elem.current_database = context->getCurrentDatabase();
|
||||
elem.query = query_for_logging;
|
||||
@ -772,25 +791,29 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
}
|
||||
|
||||
/// Common code for finish and exception callbacks
|
||||
auto status_info_to_query_log = [](QueryLogElement & element, const QueryStatusInfo & info, const ASTPtr query_ast, const ContextPtr context_ptr) mutable
|
||||
auto status_info_to_query_log
|
||||
= [](QueryLogElement & element, const QueryStatusInfo & info, const ASTPtr query_ast, const ContextPtr context_ptr) mutable
|
||||
{
|
||||
UInt64 query_time = static_cast<UInt64>(info.elapsed_seconds * 1000000);
|
||||
ProfileEvents::increment(ProfileEvents::QueryTimeMicroseconds, query_time);
|
||||
const auto time_now = std::chrono::system_clock::now();
|
||||
UInt64 elapsed_microseconds = info.elapsed_microseconds;
|
||||
element.event_time = timeInSeconds(time_now);
|
||||
element.event_time_microseconds = timeInMicroseconds(time_now);
|
||||
element.query_duration_ms = elapsed_microseconds / 1000;
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::QueryTimeMicroseconds, elapsed_microseconds);
|
||||
if (query_ast->as<ASTSelectQuery>() || query_ast->as<ASTSelectWithUnionQuery>())
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::SelectQueryTimeMicroseconds, query_time);
|
||||
ProfileEvents::increment(ProfileEvents::SelectQueryTimeMicroseconds, elapsed_microseconds);
|
||||
}
|
||||
else if (query_ast->as<ASTInsertQuery>())
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::InsertQueryTimeMicroseconds, query_time);
|
||||
ProfileEvents::increment(ProfileEvents::InsertQueryTimeMicroseconds, elapsed_microseconds);
|
||||
}
|
||||
else
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::OtherQueryTimeMicroseconds, query_time);
|
||||
ProfileEvents::increment(ProfileEvents::OtherQueryTimeMicroseconds, elapsed_microseconds);
|
||||
}
|
||||
|
||||
element.query_duration_ms = static_cast<UInt64>(info.elapsed_seconds * 1000);
|
||||
|
||||
element.read_rows = info.read_rows;
|
||||
element.read_bytes = info.read_bytes;
|
||||
|
||||
@ -844,16 +867,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
CurrentThread::finalizePerformanceCounters();
|
||||
|
||||
QueryStatusInfo info = process_list_elem->getInfo(true, context->getSettingsRef().log_profile_events);
|
||||
|
||||
double elapsed_seconds = info.elapsed_seconds;
|
||||
|
||||
elem.type = QueryLogElementType::QUERY_FINISH;
|
||||
|
||||
// construct event_time and event_time_microseconds using the same time point
|
||||
// so that the two times will always be equal up to a precision of a second.
|
||||
const auto finish_time = std::chrono::system_clock::now();
|
||||
elem.event_time = timeInSeconds(finish_time);
|
||||
elem.event_time_microseconds = timeInMicroseconds(finish_time);
|
||||
status_info_to_query_log(elem, info, ast, context);
|
||||
|
||||
if (pulling_pipeline)
|
||||
@ -877,9 +892,15 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
|
||||
if (elem.read_rows != 0)
|
||||
{
|
||||
LOG_INFO(&Poco::Logger::get("executeQuery"), "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.",
|
||||
elem.read_rows, ReadableSize(elem.read_bytes), elapsed_seconds,
|
||||
static_cast<size_t>(elem.read_rows / elapsed_seconds),
|
||||
double elapsed_seconds = static_cast<double>(info.elapsed_microseconds) / 1000000.0;
|
||||
double rows_per_second = static_cast<double>(elem.read_rows) / elapsed_seconds;
|
||||
LOG_INFO(
|
||||
&Poco::Logger::get("executeQuery"),
|
||||
"Read {} rows, {} in {} sec., {} rows/sec., {}/sec.",
|
||||
elem.read_rows,
|
||||
ReadableSize(elem.read_bytes),
|
||||
elapsed_seconds,
|
||||
rows_per_second,
|
||||
ReadableSize(elem.read_bytes / elapsed_seconds));
|
||||
}
|
||||
|
||||
@ -893,8 +914,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
if (auto processors_profile_log = context->getProcessorsProfileLog())
|
||||
{
|
||||
ProcessorProfileLogElement processor_elem;
|
||||
processor_elem.event_time = timeInSeconds(finish_time);
|
||||
processor_elem.event_time_microseconds = timeInMicroseconds(finish_time);
|
||||
processor_elem.event_time = elem.event_time;
|
||||
processor_elem.event_time_microseconds = elem.event_time_microseconds;
|
||||
processor_elem.query_id = elem.client_info.current_query_id;
|
||||
|
||||
auto get_proc_id = [](const IProcessor & proc) -> UInt64
|
||||
@ -969,7 +990,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
}
|
||||
};
|
||||
|
||||
auto exception_callback = [elem,
|
||||
auto exception_callback = [start_watch,
|
||||
elem,
|
||||
context,
|
||||
ast,
|
||||
log_queries,
|
||||
@ -992,14 +1014,6 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
quota->used(QuotaType::ERRORS, 1, /* check_exceeded = */ false);
|
||||
|
||||
elem.type = QueryLogElementType::EXCEPTION_WHILE_PROCESSING;
|
||||
|
||||
// event_time and event_time_microseconds are being constructed from the same time point
|
||||
// to ensure that both the times will be equal up to the precision of a second.
|
||||
const auto time_now = std::chrono::system_clock::now();
|
||||
|
||||
elem.event_time = timeInSeconds(time_now);
|
||||
elem.event_time_microseconds = timeInMicroseconds(time_now);
|
||||
elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time);
|
||||
elem.exception_code = getCurrentExceptionCode();
|
||||
elem.exception = getCurrentExceptionMessage(false);
|
||||
|
||||
@ -1008,12 +1022,19 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
|
||||
/// Update performance counters before logging to query_log
|
||||
CurrentThread::finalizePerformanceCounters();
|
||||
const auto time_now = std::chrono::system_clock::now();
|
||||
elem.event_time = timeInSeconds(time_now);
|
||||
elem.event_time_microseconds = timeInMicroseconds(time_now);
|
||||
|
||||
if (process_list_elem)
|
||||
{
|
||||
QueryStatusInfo info = process_list_elem->getInfo(true, current_settings.log_profile_events, false);
|
||||
status_info_to_query_log(elem, info, ast, context);
|
||||
}
|
||||
else
|
||||
{
|
||||
elem.query_duration_ms = start_watch.elapsedMilliseconds();
|
||||
}
|
||||
|
||||
if (current_settings.calculate_text_stack_trace)
|
||||
setExceptionStackTrace(elem);
|
||||
@ -1063,7 +1084,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
||||
}
|
||||
|
||||
if (!internal)
|
||||
onExceptionBeforeStart(query_for_logging, context, timeInMicroseconds(current_time), ast, query_span);
|
||||
onExceptionBeforeStart(query_for_logging, context, ast, query_span, start_watch.elapsedMilliseconds());
|
||||
|
||||
throw;
|
||||
}
|
||||
|
@ -2398,6 +2398,7 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos
|
||||
|
||||
if (layers.back()->previousType() == OperatorType::Comparison)
|
||||
{
|
||||
auto old_pos = pos;
|
||||
SubqueryFunctionType subquery_function_type = SubqueryFunctionType::NONE;
|
||||
|
||||
if (any_parser.ignore(pos, expected) && subquery_parser.parse(pos, tmp, expected))
|
||||
@ -2423,6 +2424,10 @@ Action ParserExpressionImpl::tryParseOperand(Layers & layers, IParser::Pos & pos
|
||||
layers.back()->pushOperand(std::move(function));
|
||||
return Action::OPERATOR;
|
||||
}
|
||||
else
|
||||
{
|
||||
pos = old_pos;
|
||||
}
|
||||
}
|
||||
|
||||
/// Try to find any unary operators
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Interpreters/OpenTelemetrySpanLog.h>
|
||||
#include <Parsers/ParserQuery.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
#include <Parsers/ASTExplainQuery.h>
|
||||
#include <Parsers/Lexer.h>
|
||||
#include <Parsers/TokenIterator.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
@ -263,7 +264,19 @@ ASTPtr tryParseQuery(
|
||||
|
||||
ASTInsertQuery * insert = nullptr;
|
||||
if (parse_res)
|
||||
insert = res->as<ASTInsertQuery>();
|
||||
{
|
||||
if (auto * explain = res->as<ASTExplainQuery>())
|
||||
{
|
||||
if (auto explained_query = explain->getExplainedQuery())
|
||||
{
|
||||
insert = explained_query->as<ASTInsertQuery>();
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
insert = res->as<ASTInsertQuery>();
|
||||
}
|
||||
}
|
||||
|
||||
// If parsed query ends at data for insertion. Data for insertion could be
|
||||
// in any format and not necessary be lexical correct, so we can't perform
|
||||
|
@ -76,6 +76,15 @@ public:
|
||||
|
||||
void doNotWritePrefix() { need_write_prefix = false; }
|
||||
|
||||
/// Reset the statistics watch to a specific point in time
|
||||
/// If set to not running it will stop on the call (elapsed = now() - given start)
|
||||
void setStartTime(UInt64 start, bool is_running)
|
||||
{
|
||||
statistics.watch = Stopwatch(CLOCK_MONOTONIC, start, true);
|
||||
if (!is_running)
|
||||
statistics.watch.stop();
|
||||
}
|
||||
|
||||
protected:
|
||||
friend class ParallelFormattingOutputFormat;
|
||||
|
||||
@ -132,9 +141,6 @@ protected:
|
||||
Chunk extremes;
|
||||
};
|
||||
|
||||
void setOutsideStatistics(Statistics statistics_) { statistics = std::make_shared<Statistics>(std::move(statistics_)); }
|
||||
std::shared_ptr<Statistics> getOutsideStatistics() const { return statistics; }
|
||||
|
||||
/// In some formats the way we print extremes depends on
|
||||
/// were totals printed or not. In this case in parallel formatting
|
||||
/// we should notify underling format if totals were printed.
|
||||
@ -160,10 +166,10 @@ protected:
|
||||
bool need_write_suffix = true;
|
||||
|
||||
RowsBeforeLimitCounterPtr rows_before_limit_counter;
|
||||
Statistics statistics;
|
||||
|
||||
private:
|
||||
size_t rows_read_before = 0;
|
||||
std::shared_ptr<Statistics> statistics = nullptr;
|
||||
bool are_totals_written = false;
|
||||
|
||||
/// Counters for consumed chunks. Are used for QueryLog.
|
||||
|
@ -77,10 +77,6 @@ void JSONColumnsWithMetadataBlockOutputFormat::consumeTotals(Chunk chunk)
|
||||
|
||||
void JSONColumnsWithMetadataBlockOutputFormat::finalizeImpl()
|
||||
{
|
||||
auto outside_statistics = getOutsideStatistics();
|
||||
if (outside_statistics)
|
||||
statistics = std::move(*outside_statistics);
|
||||
|
||||
JSONUtils::writeAdditionalInfo(
|
||||
rows,
|
||||
statistics.rows_before_limit,
|
||||
|
@ -60,7 +60,6 @@ protected:
|
||||
void writeExtremesElement(const char * title, const Columns & columns, size_t row_num);
|
||||
|
||||
DataTypes types;
|
||||
Statistics statistics;
|
||||
size_t rows;
|
||||
};
|
||||
|
||||
|
@ -112,10 +112,6 @@ void JSONRowOutputFormat::writeAfterExtremes()
|
||||
|
||||
void JSONRowOutputFormat::finalizeImpl()
|
||||
{
|
||||
auto outside_statistics = getOutsideStatistics();
|
||||
if (outside_statistics)
|
||||
statistics = std::move(*outside_statistics);
|
||||
|
||||
JSONUtils::writeAdditionalInfo(
|
||||
row_count,
|
||||
statistics.rows_before_limit,
|
||||
|
@ -66,7 +66,6 @@ protected:
|
||||
size_t row_count = 0;
|
||||
Names names; /// The column names are pre-escaped to be put into JSON string literal.
|
||||
|
||||
Statistics statistics;
|
||||
FormatSettings settings;
|
||||
|
||||
bool yield_strings;
|
||||
|
@ -79,11 +79,14 @@ void MySQLOutputFormat::finalizeImpl()
|
||||
CurrentThread::finalizePerformanceCounters();
|
||||
QueryStatusInfo info = process_list_elem->getInfo();
|
||||
affected_rows = info.written_rows;
|
||||
double elapsed_seconds = static_cast<double>(info.elapsed_microseconds) / 1000000.0;
|
||||
human_readable_info = fmt::format(
|
||||
"Read {} rows, {} in {} sec., {} rows/sec., {}/sec.",
|
||||
info.read_rows, ReadableSize(info.read_bytes), info.elapsed_seconds,
|
||||
static_cast<size_t>(info.read_rows / info.elapsed_seconds),
|
||||
ReadableSize(info.read_bytes / info.elapsed_seconds));
|
||||
info.read_rows,
|
||||
ReadableSize(info.read_bytes),
|
||||
elapsed_seconds,
|
||||
static_cast<size_t>(info.read_rows / elapsed_seconds),
|
||||
ReadableSize(info.read_bytes / elapsed_seconds));
|
||||
}
|
||||
|
||||
const auto & header = getPort(PortKind::Main).getHeader();
|
||||
|
@ -216,7 +216,7 @@ namespace DB
|
||||
}
|
||||
case ProcessingUnitType::FINALIZE:
|
||||
{
|
||||
formatter->setOutsideStatistics(std::move(unit.statistics));
|
||||
formatter->statistics = std::move(unit.statistics);
|
||||
formatter->finalizeImpl();
|
||||
break;
|
||||
}
|
||||
|
@ -227,7 +227,6 @@ private:
|
||||
size_t rows_consumed = 0;
|
||||
std::atomic_bool are_totals_written = false;
|
||||
|
||||
Statistics statistics;
|
||||
/// We change statistics in onProgress() which can be called from different threads.
|
||||
std::mutex statistics_mutex;
|
||||
bool save_totals_and_extremes_in_statistics;
|
||||
|
@ -137,10 +137,6 @@ void TemplateBlockOutputFormat::finalizeImpl()
|
||||
return;
|
||||
|
||||
size_t parts = format.format_idx_to_column_idx.size();
|
||||
auto outside_statistics = getOutsideStatistics();
|
||||
if (outside_statistics)
|
||||
statistics = std::move(*outside_statistics);
|
||||
|
||||
for (size_t i = 0; i < parts; ++i)
|
||||
{
|
||||
auto type = std::make_shared<DataTypeUInt64>();
|
||||
|
@ -57,8 +57,6 @@ private:
|
||||
ParsedTemplateFormatString format;
|
||||
ParsedTemplateFormatString row_format;
|
||||
|
||||
Statistics statistics;
|
||||
|
||||
size_t row_count = 0;
|
||||
|
||||
std::string row_between_delimiter;
|
||||
|
@ -190,9 +190,6 @@ void XMLRowOutputFormat::finalizeImpl()
|
||||
writeIntText(row_count, *ostr);
|
||||
writeCString("</rows>\n", *ostr);
|
||||
|
||||
auto outside_statistics = getOutsideStatistics();
|
||||
if (outside_statistics)
|
||||
statistics = std::move(*outside_statistics);
|
||||
|
||||
writeRowsBeforeLimitAtLeast();
|
||||
|
||||
|
@ -61,7 +61,6 @@ private:
|
||||
NamesAndTypes fields;
|
||||
Names field_tag_names;
|
||||
|
||||
Statistics statistics;
|
||||
const FormatSettings format_settings;
|
||||
};
|
||||
|
||||
|
@ -44,9 +44,6 @@ public:
|
||||
QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &) override;
|
||||
|
||||
void describePipeline(FormatSettings & settings) const override;
|
||||
|
||||
private:
|
||||
Processors processors;
|
||||
};
|
||||
|
||||
void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets subqueries_for_sets, ContextPtr context);
|
||||
|
@ -113,4 +113,9 @@ void IQueryPlanStep::describePipeline(const Processors & processors, FormatSetti
|
||||
doDescribeProcessor(*prev, count, settings);
|
||||
}
|
||||
|
||||
void IQueryPlanStep::appendExtraProcessors(const Processors & extra_processors)
|
||||
{
|
||||
processors.insert(processors.end(), extra_processors.begin(), extra_processors.end());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -110,6 +110,9 @@ public:
|
||||
/// Get description of processors added in current step. Should be called after updatePipeline().
|
||||
virtual void describePipeline(FormatSettings & /*settings*/) const {}
|
||||
|
||||
/// Append extra processors for this step.
|
||||
void appendExtraProcessors(const Processors & extra_processors);
|
||||
|
||||
protected:
|
||||
DataStreams input_streams;
|
||||
std::optional<DataStream> output_stream;
|
||||
@ -117,6 +120,10 @@ protected:
|
||||
/// Text description about what current step does.
|
||||
std::string step_description;
|
||||
|
||||
/// This field is used to store added processors from this step.
|
||||
/// It is used only for introspection (EXPLAIN PIPELINE).
|
||||
Processors processors;
|
||||
|
||||
static void describePipeline(const Processors & processors, FormatSettings & settings);
|
||||
};
|
||||
|
||||
|
@ -15,10 +15,6 @@ public:
|
||||
virtual void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) = 0;
|
||||
|
||||
void describePipeline(FormatSettings & settings) const override;
|
||||
|
||||
protected:
|
||||
/// We collect processors got after pipeline transformation.
|
||||
Processors processors;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -70,9 +70,4 @@ void ITransformingStep::describePipeline(FormatSettings & settings) const
|
||||
IQueryPlanStep::describePipeline(processors, settings);
|
||||
}
|
||||
|
||||
void ITransformingStep::appendExtraProcessors(const Processors & extra_processors)
|
||||
{
|
||||
processors.insert(processors.end(), extra_processors.begin(), extra_processors.end());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -75,9 +75,6 @@ public:
|
||||
|
||||
void describePipeline(FormatSettings & settings) const override;
|
||||
|
||||
/// Append extra processors for this step.
|
||||
void appendExtraProcessors(const Processors & extra_processors);
|
||||
|
||||
/// Enforcement is supposed to be done through the special settings that will be taken into account by remote nodes during query planning (e.g. force_aggregation_in_order).
|
||||
/// Should be called only if data_stream_traits.can_enforce_sorting_properties_in_distributed_query == true.
|
||||
virtual void adjustSettingsToEnforceSortingPropertiesInDistributedQuery(ContextMutablePtr) const
|
||||
@ -100,8 +97,7 @@ protected:
|
||||
private:
|
||||
virtual void updateOutputStream() = 0;
|
||||
|
||||
/// We collect processors got after pipeline transformation.
|
||||
Processors processors;
|
||||
/// If we should collect processors got after pipeline transformation.
|
||||
bool collect_processors;
|
||||
|
||||
const DataStreamTraits data_stream_traits;
|
||||
|
@ -42,10 +42,10 @@ IntersectOrExceptStep::IntersectOrExceptStep(
|
||||
QueryPipelineBuilderPtr IntersectOrExceptStep::updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
auto pipeline = std::make_unique<QueryPipelineBuilder>();
|
||||
QueryPipelineProcessorsCollector collector(*pipeline, this);
|
||||
|
||||
if (pipelines.empty())
|
||||
{
|
||||
QueryPipelineProcessorsCollector collector(*pipeline, this);
|
||||
pipeline->init(Pipe(std::make_shared<NullSource>(output_stream->header)));
|
||||
processors = collector.detachProcessors();
|
||||
return pipeline;
|
||||
@ -56,6 +56,7 @@ QueryPipelineBuilderPtr IntersectOrExceptStep::updatePipeline(QueryPipelineBuild
|
||||
/// Just in case.
|
||||
if (!isCompatibleHeader(cur_pipeline->getHeader(), getOutputStream().header))
|
||||
{
|
||||
QueryPipelineProcessorsCollector collector(*cur_pipeline, this);
|
||||
auto converting_dag = ActionsDAG::makeConvertingActions(
|
||||
cur_pipeline->getHeader().getColumnsWithTypeAndName(),
|
||||
getOutputStream().header.getColumnsWithTypeAndName(),
|
||||
@ -66,16 +67,20 @@ QueryPipelineBuilderPtr IntersectOrExceptStep::updatePipeline(QueryPipelineBuild
|
||||
{
|
||||
return std::make_shared<ExpressionTransform>(cur_header, converting_actions);
|
||||
});
|
||||
|
||||
auto added_processors = collector.detachProcessors();
|
||||
processors.insert(processors.end(), added_processors.begin(), added_processors.end());
|
||||
}
|
||||
|
||||
/// For the case of union.
|
||||
cur_pipeline->addTransform(std::make_shared<ResizeProcessor>(header, cur_pipeline->getNumStreams(), 1));
|
||||
}
|
||||
|
||||
*pipeline = QueryPipelineBuilder::unitePipelines(std::move(pipelines), max_threads);
|
||||
pipeline->addTransform(std::make_shared<IntersectOrExceptTransform>(header, current_operator));
|
||||
*pipeline = QueryPipelineBuilder::unitePipelines(std::move(pipelines), max_threads, &processors);
|
||||
auto transform = std::make_shared<IntersectOrExceptTransform>(header, current_operator);
|
||||
processors.push_back(transform);
|
||||
pipeline->addTransform(std::move(transform));
|
||||
|
||||
processors = collector.detachProcessors();
|
||||
return pipeline;
|
||||
}
|
||||
|
||||
|
@ -24,7 +24,6 @@ private:
|
||||
Block header;
|
||||
Operator current_operator;
|
||||
size_t max_threads;
|
||||
Processors processors;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -37,7 +37,6 @@ private:
|
||||
size_t max_block_size;
|
||||
size_t max_streams;
|
||||
bool keep_left_read_in_order;
|
||||
Processors processors;
|
||||
};
|
||||
|
||||
/// Special step for the case when Join is already filled.
|
||||
|
@ -62,10 +62,10 @@ void UnionStep::updateOutputSortDescription()
|
||||
QueryPipelineBuilderPtr UnionStep::updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
auto pipeline = std::make_unique<QueryPipelineBuilder>();
|
||||
QueryPipelineProcessorsCollector collector(*pipeline, this);
|
||||
|
||||
if (pipelines.empty())
|
||||
{
|
||||
QueryPipelineProcessorsCollector collector(*pipeline, this);
|
||||
pipeline->init(Pipe(std::make_shared<NullSource>(output_stream->header)));
|
||||
processors = collector.detachProcessors();
|
||||
return pipeline;
|
||||
@ -80,6 +80,7 @@ QueryPipelineBuilderPtr UnionStep::updatePipeline(QueryPipelineBuilders pipeline
|
||||
/// But, just in case, convert it to the same header if not.
|
||||
if (!isCompatibleHeader(cur_pipeline->getHeader(), getOutputStream().header))
|
||||
{
|
||||
QueryPipelineProcessorsCollector collector(*cur_pipeline, this);
|
||||
auto converting_dag = ActionsDAG::makeConvertingActions(
|
||||
cur_pipeline->getHeader().getColumnsWithTypeAndName(),
|
||||
getOutputStream().header.getColumnsWithTypeAndName(),
|
||||
@ -90,12 +91,13 @@ QueryPipelineBuilderPtr UnionStep::updatePipeline(QueryPipelineBuilders pipeline
|
||||
{
|
||||
return std::make_shared<ExpressionTransform>(cur_header, converting_actions);
|
||||
});
|
||||
|
||||
auto added_processors = collector.detachProcessors();
|
||||
processors.insert(processors.end(), added_processors.begin(), added_processors.end());
|
||||
}
|
||||
}
|
||||
|
||||
*pipeline = QueryPipelineBuilder::unitePipelines(std::move(pipelines), max_threads);
|
||||
|
||||
processors = collector.detachProcessors();
|
||||
*pipeline = QueryPipelineBuilder::unitePipelines(std::move(pipelines), max_threads, &processors);
|
||||
return pipeline;
|
||||
}
|
||||
|
||||
|
@ -24,7 +24,6 @@ public:
|
||||
private:
|
||||
Block header;
|
||||
size_t max_threads;
|
||||
Processors processors;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -474,7 +474,7 @@ Pipe ShellCommandSourceCoordinator::createPipe(
|
||||
std::unique_ptr<ShellCommand> process;
|
||||
std::unique_ptr<ShellCommandHolder> process_holder;
|
||||
|
||||
auto destructor_strategy = ShellCommand::DestructorStrategy{true /*terminate_in_destructor*/, configuration.command_termination_timeout_seconds};
|
||||
auto destructor_strategy = ShellCommand::DestructorStrategy{true /*terminate_in_destructor*/, SIGTERM, configuration.command_termination_timeout_seconds};
|
||||
command_config.terminate_in_destructor_strategy = destructor_strategy;
|
||||
|
||||
bool is_executable_pool = (process_pool != nullptr);
|
||||
|
@ -29,111 +29,11 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TYPE_MISMATCH;
|
||||
extern const int MONGODB_CANNOT_AUTHENTICATE;
|
||||
extern const int UNKNOWN_TYPE;
|
||||
extern const int MONGODB_ERROR;
|
||||
}
|
||||
|
||||
|
||||
#if POCO_VERSION < 0x01070800
|
||||
/// See https://pocoproject.org/forum/viewtopic.php?f=10&t=6326&p=11426&hilit=mongodb+auth#p11485
|
||||
void authenticate(Poco::MongoDB::Connection & connection, const std::string & database, const std::string & user, const std::string & password)
|
||||
{
|
||||
Poco::MongoDB::Database db(database);
|
||||
|
||||
/// Challenge-response authentication.
|
||||
std::string nonce;
|
||||
|
||||
/// First step: request nonce.
|
||||
{
|
||||
auto command = db.createCommand();
|
||||
command->setNumberToReturn(1);
|
||||
command->selector().add<Int32>("getnonce", 1);
|
||||
|
||||
Poco::MongoDB::ResponseMessage response;
|
||||
connection.sendRequest(*command, response);
|
||||
|
||||
if (response.documents().empty())
|
||||
throw Exception(
|
||||
"Cannot authenticate in MongoDB: server returned empty response for 'getnonce' command",
|
||||
ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
|
||||
|
||||
auto doc = response.documents()[0];
|
||||
try
|
||||
{
|
||||
double ok = doc->get<double>("ok", 0);
|
||||
if (ok != 1)
|
||||
throw Exception(
|
||||
"Cannot authenticate in MongoDB: server returned response for 'getnonce' command that"
|
||||
" has field 'ok' missing or having wrong value",
|
||||
ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
|
||||
|
||||
nonce = doc->get<std::string>("nonce", "");
|
||||
if (nonce.empty())
|
||||
throw Exception(
|
||||
"Cannot authenticate in MongoDB: server returned response for 'getnonce' command that"
|
||||
" has field 'nonce' missing or empty",
|
||||
ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
|
||||
}
|
||||
catch (Poco::NotFoundException & e)
|
||||
{
|
||||
throw Exception(
|
||||
"Cannot authenticate in MongoDB: server returned response for 'getnonce' command that has missing required field: "
|
||||
+ e.displayText(),
|
||||
ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
|
||||
}
|
||||
}
|
||||
|
||||
/// Second step: use nonce to calculate digest and send it back to the server.
|
||||
/// Digest is hex_md5(n.nonce + username + hex_md5(username + ":mongo:" + password))
|
||||
{
|
||||
std::string first = user + ":mongo:" + password;
|
||||
|
||||
Poco::MD5Engine md5;
|
||||
md5.update(first);
|
||||
std::string digest_first(Poco::DigestEngine::digestToHex(md5.digest()));
|
||||
std::string second = nonce + user + digest_first;
|
||||
md5.reset();
|
||||
md5.update(second);
|
||||
std::string digest_second(Poco::DigestEngine::digestToHex(md5.digest()));
|
||||
|
||||
auto command = db.createCommand();
|
||||
command->setNumberToReturn(1);
|
||||
command->selector()
|
||||
.add<Int32>("authenticate", 1)
|
||||
.add<std::string>("user", user)
|
||||
.add<std::string>("nonce", nonce)
|
||||
.add<std::string>("key", digest_second);
|
||||
|
||||
Poco::MongoDB::ResponseMessage response;
|
||||
connection.sendRequest(*command, response);
|
||||
|
||||
if (response.empty())
|
||||
throw Exception(
|
||||
"Cannot authenticate in MongoDB: server returned empty response for 'authenticate' command",
|
||||
ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
|
||||
|
||||
auto doc = response.documents()[0];
|
||||
try
|
||||
{
|
||||
double ok = doc->get<double>("ok", 0);
|
||||
if (ok != 1)
|
||||
throw Exception(
|
||||
"Cannot authenticate in MongoDB: server returned response for 'authenticate' command that"
|
||||
" has field 'ok' missing or having wrong value",
|
||||
ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
|
||||
}
|
||||
catch (Poco::NotFoundException & e)
|
||||
{
|
||||
throw Exception(
|
||||
"Cannot authenticate in MongoDB: server returned response for 'authenticate' command that has missing required field: "
|
||||
+ e.displayText(),
|
||||
ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
|
||||
}
|
||||
}
|
||||
}
|
||||
#endif
|
||||
|
||||
std::unique_ptr<Poco::MongoDB::Cursor> createCursor(const std::string & database, const std::string & collection, const Block & sample_block_to_select)
|
||||
{
|
||||
auto cursor = std::make_unique<Poco::MongoDB::Cursor>(database, collection);
|
||||
|
@ -309,6 +309,7 @@ QueryPipelineBuilder QueryPipelineBuilder::unitePipelines(
|
||||
pipeline.limitMaxThreads(max_threads_limit);
|
||||
}
|
||||
|
||||
pipeline.setCollectedProcessors(nullptr);
|
||||
return pipeline;
|
||||
}
|
||||
|
||||
@ -384,11 +385,7 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelinesRightLe
|
||||
/// Collect the NEW processors for the right pipeline.
|
||||
QueryPipelineProcessorsCollector collector(*right);
|
||||
/// Remember the last step of the right pipeline.
|
||||
ExpressionStep * step = typeid_cast<ExpressionStep *>(right->pipe.processors->back()->getQueryPlanStep());
|
||||
if (!step)
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "The top step of the right pipeline should be ExpressionStep");
|
||||
}
|
||||
IQueryPlanStep * step = right->pipe.processors->back()->getQueryPlanStep();
|
||||
|
||||
/// In case joined subquery has totals, and we don't, add default chunk to totals.
|
||||
bool default_totals = false;
|
||||
@ -560,7 +557,8 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelinesRightLe
|
||||
|
||||
/// Move the collected processors to the last step in the right pipeline.
|
||||
Processors processors = collector.detachProcessors();
|
||||
step->appendExtraProcessors(processors);
|
||||
if (step)
|
||||
step->appendExtraProcessors(processors);
|
||||
|
||||
left->pipe.processors->insert(left->pipe.processors->end(), right->pipe.processors->begin(), right->pipe.processors->end());
|
||||
left->resources = std::move(right->resources);
|
||||
|
@ -576,6 +576,9 @@ void IMergeTreeDataPart::assertState(const std::initializer_list<MergeTreeDataPa
|
||||
states_str += ' ';
|
||||
}
|
||||
|
||||
if (!states_str.empty())
|
||||
states_str.pop_back();
|
||||
|
||||
throw Exception("Unexpected state of part " + getNameWithState() + ". Expected: " + states_str, ErrorCodes::NOT_FOUND_EXPECTED_DATA_PART);
|
||||
}
|
||||
}
|
||||
|
@ -88,10 +88,6 @@ 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);
|
||||
|
@ -1836,6 +1836,9 @@ void MergeTreeData::rollbackDeletingParts(const MergeTreeData::DataPartsVector &
|
||||
|
||||
void MergeTreeData::removePartsFinally(const MergeTreeData::DataPartsVector & parts)
|
||||
{
|
||||
if (parts.empty())
|
||||
return;
|
||||
|
||||
{
|
||||
auto lock = lockParts();
|
||||
|
||||
@ -1852,12 +1855,12 @@ void MergeTreeData::removePartsFinally(const MergeTreeData::DataPartsVector & pa
|
||||
|
||||
(*it)->assertState({DataPartState::Deleting});
|
||||
|
||||
LOG_DEBUG(log, "Finally removing part from memory {}", part->name);
|
||||
|
||||
data_parts_indexes.erase(it);
|
||||
}
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Removing {} parts from memory: Parts: [{}]", parts.size(), fmt::join(parts, ", "));
|
||||
|
||||
/// Data parts is still alive (since DataPartsVector holds shared_ptrs) and contain useful metainformation for logging
|
||||
/// NOTE: There is no need to log parts deletion somewhere else, all deleting parts pass through this function and pass away
|
||||
|
||||
@ -1910,13 +1913,14 @@ void MergeTreeData::flushAllInMemoryPartsIfNeeded()
|
||||
size_t MergeTreeData::clearOldPartsFromFilesystem(bool force)
|
||||
{
|
||||
DataPartsVector parts_to_remove = grabOldParts(force);
|
||||
if (parts_to_remove.empty())
|
||||
return 0;
|
||||
|
||||
clearPartsFromFilesystem(parts_to_remove);
|
||||
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.
|
||||
if (!parts_to_remove.empty())
|
||||
getContext()->dropMMappedFileCache();
|
||||
getContext()->dropMMappedFileCache();
|
||||
|
||||
return parts_to_remove.size();
|
||||
}
|
||||
@ -1980,7 +1984,8 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t
|
||||
ThreadPool pool(num_threads);
|
||||
|
||||
/// NOTE: Under heavy system load you may get "Cannot schedule a task" from ThreadPool.
|
||||
LOG_DEBUG(log, "Removing {} parts from filesystem: {} (concurrently)", parts_to_remove.size(), fmt::join(parts_to_remove, ", "));
|
||||
LOG_DEBUG(
|
||||
log, "Removing {} parts from filesystem (concurrently): Parts: [{}]", parts_to_remove.size(), fmt::join(parts_to_remove, ", "));
|
||||
for (const DataPartPtr & part : parts_to_remove)
|
||||
{
|
||||
pool.scheduleOrThrowOnError([&, thread_group = CurrentThread::getGroup()]
|
||||
@ -2005,7 +2010,8 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t
|
||||
}
|
||||
else if (!parts_to_remove.empty())
|
||||
{
|
||||
LOG_DEBUG(log, "Removing {} parts from filesystem: {}", parts_to_remove.size(), fmt::join(parts_to_remove, ", "));
|
||||
LOG_DEBUG(
|
||||
log, "Removing {} parts from filesystem (serially): Parts: [{}]", parts_to_remove.size(), fmt::join(parts_to_remove, ", "));
|
||||
for (const DataPartPtr & part : parts_to_remove)
|
||||
{
|
||||
preparePartForRemoval(part)->remove();
|
||||
@ -5424,6 +5430,7 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData:
|
||||
part->getDataPartStorage().commitTransaction();
|
||||
|
||||
if (txn)
|
||||
{
|
||||
for (const auto & part : precommitted_parts)
|
||||
{
|
||||
DataPartPtr covering_part;
|
||||
@ -5445,6 +5452,7 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData:
|
||||
|
||||
MergeTreeTransaction::addNewPartAndRemoveCovered(data.shared_from_this(), part, covered_parts, txn);
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeData::WriteAheadLogPtr wal;
|
||||
auto get_inited_wal = [&] ()
|
||||
|
@ -1148,7 +1148,8 @@ private:
|
||||
|
||||
void prepare()
|
||||
{
|
||||
ctx->new_data_part->getDataPartStorage().createDirectories();
|
||||
if (ctx->new_data_part->isStoredOnDisk())
|
||||
ctx->new_data_part->getDataPartStorage().createDirectories();
|
||||
|
||||
/// Note: this is done before creating input streams, because otherwise data.data_parts_mutex
|
||||
/// (which is locked in data.getTotalActiveSizeInBytes())
|
||||
|
@ -539,7 +539,7 @@ std::vector<String> ReplicatedMergeTreeSinkImpl<async_insert>::commitPart(
|
||||
///
|
||||
/// metadata_snapshot->check(part->getColumns());
|
||||
|
||||
String temporary_part_relative_path = part->getDataPartStorage().getPartDirectory();
|
||||
const String temporary_part_relative_path = part->getDataPartStorage().getPartDirectory();
|
||||
|
||||
/// There is one case when we need to retry transaction in a loop.
|
||||
/// But don't do it too many times - just as defensive measure.
|
||||
@ -820,6 +820,14 @@ std::vector<String> ReplicatedMergeTreeSinkImpl<async_insert>::commitPart(
|
||||
part->name);
|
||||
}
|
||||
|
||||
auto rename_part_to_temporary = [&temporary_part_relative_path, &transaction, &part]()
|
||||
{
|
||||
transaction.rollbackPartsToTemporaryState();
|
||||
|
||||
part->is_temp = true;
|
||||
part->renameTo(temporary_part_relative_path, false);
|
||||
};
|
||||
|
||||
try
|
||||
{
|
||||
ThreadFuzzer::maybeInjectSleep();
|
||||
@ -828,11 +836,7 @@ std::vector<String> ReplicatedMergeTreeSinkImpl<async_insert>::commitPart(
|
||||
}
|
||||
catch (const Exception &)
|
||||
{
|
||||
transaction.rollbackPartsToTemporaryState();
|
||||
|
||||
part->is_temp = true;
|
||||
part->renameTo(temporary_part_relative_path, false);
|
||||
|
||||
rename_part_to_temporary();
|
||||
throw;
|
||||
}
|
||||
|
||||
@ -906,10 +910,7 @@ std::vector<String> ReplicatedMergeTreeSinkImpl<async_insert>::commitPart(
|
||||
|
||||
/// We will try to add this part again on the new iteration as it's just a new part.
|
||||
/// So remove it from storage parts set immediately and transfer state to temporary.
|
||||
transaction.rollbackPartsToTemporaryState();
|
||||
|
||||
part->is_temp = true;
|
||||
part->renameTo(temporary_part_relative_path, false);
|
||||
rename_part_to_temporary();
|
||||
|
||||
if constexpr (async_insert)
|
||||
{
|
||||
@ -931,8 +932,20 @@ std::vector<String> ReplicatedMergeTreeSinkImpl<async_insert>::commitPart(
|
||||
}
|
||||
else if (multi_code == Coordination::Error::ZNODEEXISTS && failed_op_path == quorum_info.status_path)
|
||||
{
|
||||
storage.unlockSharedData(*part, zookeeper);
|
||||
transaction.rollback();
|
||||
try
|
||||
{
|
||||
storage.unlockSharedData(*part, zookeeper);
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
{
|
||||
/// suppress this exception since need to rename part to temporary next
|
||||
LOG_DEBUG(log, "Unlocking shared data failed during error handling: code={} message={}", e.code, e.message());
|
||||
}
|
||||
|
||||
/// Part was not committed to keeper
|
||||
/// So make it temporary to avoid its resurrection on restart
|
||||
rename_part_to_temporary();
|
||||
|
||||
throw Exception("Another quorum insert has been already started", ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE);
|
||||
}
|
||||
else
|
||||
|
@ -30,6 +30,7 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
@ -192,10 +193,15 @@ void registerStorageExecutable(StorageFactory & factory)
|
||||
std::vector<ASTPtr> input_queries;
|
||||
for (size_t i = 2; i < args.engine_args.size(); ++i)
|
||||
{
|
||||
if (args.engine_args[i]->children.empty())
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS, "StorageExecutable argument \"{}\" is invalid query",
|
||||
args.engine_args[i]->formatForErrorMessage());
|
||||
|
||||
ASTPtr query = args.engine_args[i]->children.at(0);
|
||||
if (!query->as<ASTSelectWithUnionQuery>())
|
||||
throw Exception(
|
||||
ErrorCodes::UNSUPPORTED_METHOD, "StorageExecutable argument is invalid input query {}",
|
||||
ErrorCodes::UNSUPPORTED_METHOD, "StorageExecutable argument \"{}\" is invalid input query",
|
||||
query->formatForErrorMessage());
|
||||
|
||||
input_queries.emplace_back(std::move(query));
|
||||
|
@ -72,16 +72,14 @@ void StorageMongoDB::connectIfNotConnected()
|
||||
auto auth_db = database_name;
|
||||
if (auth_source != query_params.end())
|
||||
auth_db = auth_source->second;
|
||||
#if POCO_VERSION >= 0x01070800
|
||||
|
||||
if (!username.empty() && !password.empty())
|
||||
{
|
||||
Poco::MongoDB::Database poco_db(auth_db);
|
||||
if (!poco_db.authenticate(*connection, username, password, Poco::MongoDB::Database::AUTH_SCRAM_SHA1))
|
||||
throw Exception("Cannot authenticate in MongoDB, incorrect user or password", ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
|
||||
}
|
||||
#else
|
||||
authenticate(*connection, database_name, username, password);
|
||||
#endif
|
||||
|
||||
authenticated = true;
|
||||
}
|
||||
}
|
||||
@ -213,7 +211,6 @@ StorageMongoDBConfiguration StorageMongoDB::getConfiguration(ASTs engine_args, C
|
||||
|
||||
if (engine_args.size() >= 6)
|
||||
configuration.options = checkAndGetLiteralArgument<String>(engine_args[5], "database");
|
||||
|
||||
}
|
||||
|
||||
context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port));
|
||||
|
@ -82,6 +82,20 @@ bool hasJoin(const ASTSelectWithUnionQuery & ast)
|
||||
return false;
|
||||
}
|
||||
|
||||
/** There are no limits on the maximum size of the result for the view.
|
||||
* Since the result of the view is not the result of the entire query.
|
||||
*/
|
||||
ContextPtr getViewContext(ContextPtr context)
|
||||
{
|
||||
auto view_context = Context::createCopy(context);
|
||||
Settings view_settings = context->getSettings();
|
||||
view_settings.max_result_rows = 0;
|
||||
view_settings.max_result_bytes = 0;
|
||||
view_settings.extremes = false;
|
||||
view_context->setSettings(view_settings);
|
||||
return view_context;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
StorageView::StorageView(
|
||||
@ -127,13 +141,13 @@ void StorageView::read(
|
||||
|
||||
if (context->getSettingsRef().allow_experimental_analyzer)
|
||||
{
|
||||
InterpreterSelectQueryAnalyzer interpreter(current_inner_query, options, context);
|
||||
InterpreterSelectQueryAnalyzer interpreter(current_inner_query, options, getViewContext(context));
|
||||
interpreter.addStorageLimits(*query_info.storage_limits);
|
||||
query_plan = std::move(interpreter).extractQueryPlan();
|
||||
}
|
||||
else
|
||||
{
|
||||
InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, options, column_names);
|
||||
InterpreterSelectWithUnionQuery interpreter(current_inner_query, getViewContext(context), options, column_names);
|
||||
interpreter.addStorageLimits(*query_info.storage_limits);
|
||||
interpreter.buildQueryPlan(query_plan);
|
||||
}
|
||||
|
@ -119,7 +119,7 @@ void StorageSystemProcesses::fillData(MutableColumns & res_columns, ContextPtr c
|
||||
res_columns[i++]->insert(process.client_info.quota_key);
|
||||
res_columns[i++]->insert(process.client_info.distributed_depth);
|
||||
|
||||
res_columns[i++]->insert(process.elapsed_seconds);
|
||||
res_columns[i++]->insert(static_cast<double>(process.elapsed_microseconds) / 100000.0);
|
||||
res_columns[i++]->insert(process.is_cancelled);
|
||||
res_columns[i++]->insert(process.is_all_data_sent);
|
||||
res_columns[i++]->insert(process.read_rows);
|
||||
|
@ -207,16 +207,16 @@ if __name__ == "__main__":
|
||||
args = parse_args()
|
||||
check_name = args.check_name
|
||||
kill_timeout = args.kill_timeout
|
||||
validate_bugix_check = args.validate_bugfix
|
||||
validate_bugfix_check = args.validate_bugfix
|
||||
|
||||
flaky_check = "flaky" in check_name.lower()
|
||||
|
||||
run_changed_tests = flaky_check or validate_bugix_check
|
||||
run_changed_tests = flaky_check or validate_bugfix_check
|
||||
gh = Github(get_best_robot_token(), per_page=100)
|
||||
|
||||
# For validate_bugix_check we need up to date information about labels, so pr_event_from_api is used
|
||||
# For validate_bugfix_check we need up to date information about labels, so pr_event_from_api is used
|
||||
pr_info = PRInfo(
|
||||
need_changed_files=run_changed_tests, pr_event_from_api=validate_bugix_check
|
||||
need_changed_files=run_changed_tests, pr_event_from_api=validate_bugfix_check
|
||||
)
|
||||
|
||||
atexit.register(update_mergeable_check, gh, pr_info, check_name)
|
||||
@ -224,7 +224,7 @@ if __name__ == "__main__":
|
||||
if not os.path.exists(temp_path):
|
||||
os.makedirs(temp_path)
|
||||
|
||||
if validate_bugix_check and "pr-bugfix" not in pr_info.labels:
|
||||
if validate_bugfix_check and "pr-bugfix" not in pr_info.labels:
|
||||
if args.post_commit_status == "file":
|
||||
post_commit_status_to_file(
|
||||
os.path.join(temp_path, "post_commit_status.tsv"),
|
||||
@ -256,7 +256,7 @@ if __name__ == "__main__":
|
||||
tests_to_run = get_tests_to_run(pr_info)
|
||||
if not tests_to_run:
|
||||
commit = get_commit(gh, pr_info.sha)
|
||||
state = override_status("success", check_name, validate_bugix_check)
|
||||
state = override_status("success", check_name, validate_bugfix_check)
|
||||
if args.post_commit_status == "commit_status":
|
||||
commit.create_status(
|
||||
context=check_name_with_group,
|
||||
@ -279,7 +279,7 @@ if __name__ == "__main__":
|
||||
if not os.path.exists(packages_path):
|
||||
os.makedirs(packages_path)
|
||||
|
||||
if validate_bugix_check:
|
||||
if validate_bugfix_check:
|
||||
download_last_release(packages_path)
|
||||
else:
|
||||
download_all_deb_packages(check_name, reports_path, packages_path)
|
||||
@ -297,7 +297,7 @@ if __name__ == "__main__":
|
||||
additional_envs = get_additional_envs(
|
||||
check_name, run_by_hash_num, run_by_hash_total
|
||||
)
|
||||
if validate_bugix_check:
|
||||
if validate_bugfix_check:
|
||||
additional_envs.append("GLOBAL_TAGS=no-random-settings")
|
||||
|
||||
run_command = get_run_command(
|
||||
@ -327,7 +327,7 @@ if __name__ == "__main__":
|
||||
state, description, test_results, additional_logs = process_results(
|
||||
result_path, server_log_path
|
||||
)
|
||||
state = override_status(state, check_name, invert=validate_bugix_check)
|
||||
state = override_status(state, check_name, invert=validate_bugfix_check)
|
||||
|
||||
ch_helper = ClickHouseHelper()
|
||||
mark_flaky_tests(ch_helper, check_name, test_results)
|
||||
|
@ -153,7 +153,7 @@ if __name__ == "__main__":
|
||||
|
||||
args = parse_args()
|
||||
check_name = args.check_name
|
||||
validate_bugix_check = args.validate_bugfix
|
||||
validate_bugfix_check = args.validate_bugfix
|
||||
|
||||
if "RUN_BY_HASH_NUM" in os.environ:
|
||||
run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM", "0"))
|
||||
@ -171,13 +171,13 @@ if __name__ == "__main__":
|
||||
|
||||
is_flaky_check = "flaky" in check_name
|
||||
|
||||
# For validate_bugix_check we need up to date information about labels, so pr_event_from_api is used
|
||||
# For validate_bugfix_check we need up to date information about labels, so pr_event_from_api is used
|
||||
pr_info = PRInfo(
|
||||
need_changed_files=is_flaky_check or validate_bugix_check,
|
||||
pr_event_from_api=validate_bugix_check,
|
||||
need_changed_files=is_flaky_check or validate_bugfix_check,
|
||||
pr_event_from_api=validate_bugfix_check,
|
||||
)
|
||||
|
||||
if validate_bugix_check and "pr-bugfix" not in pr_info.labels:
|
||||
if validate_bugfix_check and "pr-bugfix" not in pr_info.labels:
|
||||
if args.post_commit_status == "file":
|
||||
post_commit_status_to_file(
|
||||
os.path.join(temp_path, "post_commit_status.tsv"),
|
||||
@ -209,7 +209,7 @@ if __name__ == "__main__":
|
||||
if not os.path.exists(build_path):
|
||||
os.makedirs(build_path)
|
||||
|
||||
if validate_bugix_check:
|
||||
if validate_bugfix_check:
|
||||
download_last_release(build_path)
|
||||
else:
|
||||
download_all_deb_packages(check_name, reports_path, build_path)
|
||||
@ -252,7 +252,7 @@ if __name__ == "__main__":
|
||||
subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True)
|
||||
|
||||
state, description, test_results, additional_logs = process_results(result_path)
|
||||
state = override_status(state, check_name, invert=validate_bugix_check)
|
||||
state = override_status(state, check_name, invert=validate_bugfix_check)
|
||||
|
||||
ch_helper = ClickHouseHelper()
|
||||
mark_flaky_tests(ch_helper, check_name, test_results)
|
||||
|
@ -111,12 +111,21 @@ def clickhouse_execute_http(
|
||||
if default_format is not None:
|
||||
params["default_format"] = default_format
|
||||
|
||||
client.request(
|
||||
"POST",
|
||||
f"/?{base_args.client_options_query_str}{urllib.parse.urlencode(params)}",
|
||||
)
|
||||
res = client.getresponse()
|
||||
data = res.read()
|
||||
for i in range(MAX_RETRIES):
|
||||
try:
|
||||
client.request(
|
||||
"POST",
|
||||
f"/?{base_args.client_options_query_str}{urllib.parse.urlencode(params)}",
|
||||
)
|
||||
res = client.getresponse()
|
||||
data = res.read()
|
||||
break
|
||||
except Exception as ex:
|
||||
if i == MAX_RETRIES - 1:
|
||||
raise ex
|
||||
|
||||
sleep(i + 1)
|
||||
|
||||
if res.status != 200:
|
||||
raise HTTPError(data.decode(), res.status)
|
||||
|
||||
@ -1001,18 +1010,28 @@ class TestCase:
|
||||
seconds_left = max(
|
||||
args.timeout - (datetime.now() - start_time).total_seconds(), 20
|
||||
)
|
||||
drop_database_query = "DROP DATABASE " + database
|
||||
if args.replicated_database:
|
||||
drop_database_query += " ON CLUSTER test_cluster_database_replicated"
|
||||
|
||||
try:
|
||||
drop_database_query = "DROP DATABASE " + database
|
||||
if args.replicated_database:
|
||||
drop_database_query += " ON CLUSTER test_cluster_database_replicated"
|
||||
clickhouse_execute(
|
||||
args,
|
||||
drop_database_query,
|
||||
timeout=seconds_left,
|
||||
settings={
|
||||
"log_comment": args.testcase_basename,
|
||||
},
|
||||
)
|
||||
# It's possible to get an error "New table appeared in database being dropped or detached. Try again."
|
||||
for _ in range(1, 60):
|
||||
try:
|
||||
clickhouse_execute(
|
||||
args,
|
||||
drop_database_query,
|
||||
timeout=seconds_left,
|
||||
settings={
|
||||
"log_comment": args.testcase_basename,
|
||||
},
|
||||
)
|
||||
except HTTPError as e:
|
||||
if need_retry(args, e.message, e.message, 0):
|
||||
continue
|
||||
raise
|
||||
break
|
||||
|
||||
except socket.timeout:
|
||||
total_time = (datetime.now() - start_time).total_seconds()
|
||||
return (
|
||||
|
@ -4,7 +4,6 @@
|
||||
<host>localhost</host>
|
||||
<port>9181</port>
|
||||
</node>
|
||||
|
||||
<!-- Settings for fault injection.
|
||||
Approximate probability of request success:
|
||||
(1 - send_fault_probability) * (1 - recv_fault_probability) = 0.99998 * 0.99998 = 0.99996
|
||||
|
@ -103,8 +103,10 @@ ln -sf --backup=simple --suffix=_original.xml \
|
||||
$SRC_PATH/config.d/query_masking_rules.xml $DEST_SERVER_PATH/config.d/
|
||||
|
||||
if [[ -n "$ZOOKEEPER_FAULT_INJECTION" ]] && [[ "$ZOOKEEPER_FAULT_INJECTION" -eq 1 ]]; then
|
||||
rm -f $DEST_SERVER_PATH/config.d/zookeeper.xml ||:
|
||||
ln -sf $SRC_PATH/config.d/zookeeper_fault_injection.xml $DEST_SERVER_PATH/config.d/
|
||||
else
|
||||
rm -f $DEST_SERVER_PATH/config.d/zookeeper_fault_injection.xml ||:
|
||||
ln -sf $SRC_PATH/config.d/zookeeper.xml $DEST_SERVER_PATH/config.d/
|
||||
fi
|
||||
|
||||
|
@ -983,9 +983,22 @@ if __name__ == "__main__":
|
||||
runner = ClickhouseIntegrationTestsRunner(result_path, params)
|
||||
|
||||
logging.info("Running tests")
|
||||
|
||||
# Avoid overlaps with previous runs
|
||||
logging.info("Clearing dmesg before run")
|
||||
subprocess.check_call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL
|
||||
"dmesg --clear", shell=True
|
||||
)
|
||||
|
||||
state, description, test_results, _ = runner.run_impl(repo_path, build_path)
|
||||
logging.info("Tests finished")
|
||||
|
||||
# Dump dmesg (to capture possible OOMs)
|
||||
logging.info("Dumping dmesg")
|
||||
subprocess.check_call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL
|
||||
"dmesg -T", shell=True
|
||||
)
|
||||
|
||||
status = (state, description)
|
||||
out_results_file = os.path.join(str(runner.path()), "test_results.tsv")
|
||||
out_status_file = os.path.join(str(runner.path()), "check_status.tsv")
|
||||
|
@ -413,11 +413,5 @@ if __name__ == "__main__":
|
||||
subprocess.check_call(f"docker kill {' '.join(containers)}", shell=True)
|
||||
print(f"Containers {containers} killed")
|
||||
|
||||
# Avoid overlaps with previous runs
|
||||
subprocess.check_call("dmesg --clear", shell=True)
|
||||
|
||||
print(("Running pytest container as: '" + cmd + "'."))
|
||||
subprocess.check_call(cmd, shell=True)
|
||||
|
||||
# Dump dmesg (to capture possible OOMs)
|
||||
subprocess.check_call("dmesg -T", shell=True)
|
||||
|
@ -1,4 +1,5 @@
|
||||
import pytest
|
||||
import time
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from kazoo.client import KazooClient, KazooState
|
||||
from kazoo.security import ACL, make_digest_acl, make_acl
|
||||
@ -45,301 +46,405 @@ def get_genuine_zk():
|
||||
return cluster.get_kazoo_client("zoo1")
|
||||
|
||||
|
||||
# FIXME: this sleep is a workaround for the bug that is fixed by this patch [1].
|
||||
#
|
||||
# The problem is that after AUTH_FAILED (that is caused by the line above)
|
||||
# there can be a race, because of which, stop() will hang indefinitely.
|
||||
#
|
||||
# [1]: https://github.com/python-zk/kazoo/pull/688
|
||||
def zk_auth_failure_workaround():
|
||||
time.sleep(2)
|
||||
|
||||
|
||||
def zk_stop_and_close(zk):
|
||||
if zk:
|
||||
zk.stop()
|
||||
zk.close()
|
||||
|
||||
|
||||
@pytest.mark.parametrize(("get_zk"), [get_genuine_zk, get_fake_zk])
|
||||
def test_remove_acl(started_cluster, get_zk):
|
||||
auth_connection = get_zk()
|
||||
auth_connection = None
|
||||
|
||||
auth_connection.add_auth("digest", "user1:password1")
|
||||
try:
|
||||
auth_connection = get_zk()
|
||||
|
||||
# Consistent with zookeeper, accept generated digest
|
||||
auth_connection.create(
|
||||
"/test_remove_acl1",
|
||||
b"dataX",
|
||||
acl=[
|
||||
make_acl(
|
||||
"digest",
|
||||
"user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=",
|
||||
read=True,
|
||||
write=False,
|
||||
create=False,
|
||||
delete=False,
|
||||
admin=False,
|
||||
)
|
||||
],
|
||||
)
|
||||
auth_connection.create(
|
||||
"/test_remove_acl2",
|
||||
b"dataX",
|
||||
acl=[
|
||||
make_acl(
|
||||
"digest",
|
||||
"user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=",
|
||||
read=True,
|
||||
write=True,
|
||||
create=False,
|
||||
delete=False,
|
||||
admin=False,
|
||||
)
|
||||
],
|
||||
)
|
||||
auth_connection.create(
|
||||
"/test_remove_acl3",
|
||||
b"dataX",
|
||||
acl=[make_acl("digest", "user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=", all=True)],
|
||||
)
|
||||
auth_connection.add_auth("digest", "user1:password1")
|
||||
|
||||
auth_connection.delete("/test_remove_acl2")
|
||||
# Consistent with zookeeper, accept generated digest
|
||||
auth_connection.create(
|
||||
"/test_remove_acl1",
|
||||
b"dataX",
|
||||
acl=[
|
||||
make_acl(
|
||||
"digest",
|
||||
"user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=",
|
||||
read=True,
|
||||
write=False,
|
||||
create=False,
|
||||
delete=False,
|
||||
admin=False,
|
||||
)
|
||||
],
|
||||
)
|
||||
auth_connection.create(
|
||||
"/test_remove_acl2",
|
||||
b"dataX",
|
||||
acl=[
|
||||
make_acl(
|
||||
"digest",
|
||||
"user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=",
|
||||
read=True,
|
||||
write=True,
|
||||
create=False,
|
||||
delete=False,
|
||||
admin=False,
|
||||
)
|
||||
],
|
||||
)
|
||||
auth_connection.create(
|
||||
"/test_remove_acl3",
|
||||
b"dataX",
|
||||
acl=[make_acl("digest", "user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=", all=True)],
|
||||
)
|
||||
|
||||
auth_connection.create(
|
||||
"/test_remove_acl4",
|
||||
b"dataX",
|
||||
acl=[
|
||||
make_acl(
|
||||
"digest",
|
||||
"user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=",
|
||||
read=True,
|
||||
write=True,
|
||||
create=True,
|
||||
delete=False,
|
||||
admin=False,
|
||||
)
|
||||
],
|
||||
)
|
||||
auth_connection.delete("/test_remove_acl2")
|
||||
|
||||
acls, stat = auth_connection.get_acls("/test_remove_acl3")
|
||||
auth_connection.create(
|
||||
"/test_remove_acl4",
|
||||
b"dataX",
|
||||
acl=[
|
||||
make_acl(
|
||||
"digest",
|
||||
"user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=",
|
||||
read=True,
|
||||
write=True,
|
||||
create=True,
|
||||
delete=False,
|
||||
admin=False,
|
||||
)
|
||||
],
|
||||
)
|
||||
|
||||
assert stat.aversion == 0
|
||||
assert len(acls) == 1
|
||||
for acl in acls:
|
||||
assert acl.acl_list == ["ALL"]
|
||||
assert acl.perms == 31
|
||||
acls, stat = auth_connection.get_acls("/test_remove_acl3")
|
||||
|
||||
assert stat.aversion == 0
|
||||
assert len(acls) == 1
|
||||
for acl in acls:
|
||||
assert acl.acl_list == ["ALL"]
|
||||
assert acl.perms == 31
|
||||
finally:
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
|
||||
@pytest.mark.parametrize(("get_zk"), [get_genuine_zk, get_fake_zk])
|
||||
def test_digest_auth_basic(started_cluster, get_zk):
|
||||
auth_connection = get_zk()
|
||||
try:
|
||||
auth_connection = None
|
||||
no_auth_connection = None
|
||||
|
||||
auth_connection.add_auth("digest", "user1:password1")
|
||||
auth_connection = get_zk()
|
||||
auth_connection.add_auth("digest", "user1:password1")
|
||||
|
||||
auth_connection.create("/test_no_acl", b"")
|
||||
auth_connection.create(
|
||||
"/test_all_acl", b"data", acl=[make_acl("auth", "", all=True)]
|
||||
)
|
||||
# Consistent with zookeeper, accept generated digest
|
||||
auth_connection.create(
|
||||
"/test_all_digest_acl",
|
||||
b"dataX",
|
||||
acl=[make_acl("digest", "user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=", all=True)],
|
||||
)
|
||||
auth_connection.create("/test_no_acl", b"")
|
||||
auth_connection.create(
|
||||
"/test_all_acl", b"data", acl=[make_acl("auth", "", all=True)]
|
||||
)
|
||||
# Consistent with zookeeper, accept generated digest
|
||||
auth_connection.create(
|
||||
"/test_all_digest_acl",
|
||||
b"dataX",
|
||||
acl=[make_acl("digest", "user1:XDkd2dsEuhc9ImU3q8pa8UOdtpI=", all=True)],
|
||||
)
|
||||
|
||||
assert auth_connection.get("/test_all_acl")[0] == b"data"
|
||||
assert auth_connection.get("/test_all_digest_acl")[0] == b"dataX"
|
||||
assert auth_connection.get("/test_all_acl")[0] == b"data"
|
||||
assert auth_connection.get("/test_all_digest_acl")[0] == b"dataX"
|
||||
|
||||
no_auth_connection = get_zk()
|
||||
no_auth_connection.set("/test_no_acl", b"hello")
|
||||
no_auth_connection = get_zk()
|
||||
no_auth_connection.set("/test_no_acl", b"hello")
|
||||
assert no_auth_connection.get("/test_no_acl")[0] == b"hello"
|
||||
|
||||
# no ACL, so cannot access these nodes
|
||||
assert no_auth_connection.get("/test_no_acl")[0] == b"hello"
|
||||
# no ACL, so cannot access these nodes
|
||||
with pytest.raises(NoAuthError):
|
||||
no_auth_connection.set("/test_all_acl", b"hello")
|
||||
|
||||
with pytest.raises(NoAuthError):
|
||||
no_auth_connection.set("/test_all_acl", b"hello")
|
||||
with pytest.raises(NoAuthError):
|
||||
no_auth_connection.get("/test_all_acl")
|
||||
|
||||
with pytest.raises(NoAuthError):
|
||||
no_auth_connection.get("/test_all_acl")
|
||||
with pytest.raises(NoAuthError):
|
||||
no_auth_connection.get("/test_all_digest_acl")
|
||||
|
||||
with pytest.raises(NoAuthError):
|
||||
no_auth_connection.get("/test_all_digest_acl")
|
||||
# still doesn't help
|
||||
with pytest.raises(AuthFailedError):
|
||||
no_auth_connection.add_auth("world", "anyone")
|
||||
|
||||
# still doesn't help
|
||||
with pytest.raises(AuthFailedError):
|
||||
no_auth_connection.add_auth("world", "anyone")
|
||||
zk_auth_failure_workaround()
|
||||
zk_stop_and_close(no_auth_connection)
|
||||
# session became broken, reconnect
|
||||
no_auth_connection = get_zk()
|
||||
|
||||
# session became broken, reconnect
|
||||
no_auth_connection = get_zk()
|
||||
# wrong auth
|
||||
no_auth_connection.add_auth("digest", "user2:password2")
|
||||
|
||||
# wrong auth
|
||||
no_auth_connection.add_auth("digest", "user2:password2")
|
||||
with pytest.raises(NoAuthError):
|
||||
no_auth_connection.set("/test_all_acl", b"hello")
|
||||
|
||||
with pytest.raises(NoAuthError):
|
||||
no_auth_connection.set("/test_all_acl", b"hello")
|
||||
with pytest.raises(NoAuthError):
|
||||
no_auth_connection.set("/test_all_acl", b"hello")
|
||||
|
||||
with pytest.raises(NoAuthError):
|
||||
no_auth_connection.set("/test_all_acl", b"hello")
|
||||
with pytest.raises(NoAuthError):
|
||||
no_auth_connection.get("/test_all_acl")
|
||||
|
||||
with pytest.raises(NoAuthError):
|
||||
no_auth_connection.get("/test_all_acl")
|
||||
with pytest.raises(NoAuthError):
|
||||
no_auth_connection.get("/test_all_digest_acl")
|
||||
|
||||
with pytest.raises(NoAuthError):
|
||||
no_auth_connection.get("/test_all_digest_acl")
|
||||
# but can access some non restricted nodes
|
||||
no_auth_connection.create("/some_allowed_node", b"data")
|
||||
|
||||
# but can access some non restricted nodes
|
||||
no_auth_connection.create("/some_allowed_node", b"data")
|
||||
|
||||
# auth added, go on
|
||||
no_auth_connection.add_auth("digest", "user1:password1")
|
||||
for path in ["/test_no_acl", "/test_all_acl"]:
|
||||
no_auth_connection.set(path, b"auth_added")
|
||||
assert no_auth_connection.get(path)[0] == b"auth_added"
|
||||
# auth added, go on
|
||||
no_auth_connection.add_auth("digest", "user1:password1")
|
||||
for path in ["/test_no_acl", "/test_all_acl"]:
|
||||
no_auth_connection.set(path, b"auth_added")
|
||||
assert no_auth_connection.get(path)[0] == b"auth_added"
|
||||
finally:
|
||||
zk_stop_and_close(auth_connection)
|
||||
zk_stop_and_close(no_auth_connection)
|
||||
|
||||
|
||||
def test_super_auth(started_cluster):
|
||||
auth_connection = get_fake_zk()
|
||||
|
||||
auth_connection.add_auth("digest", "user1:password1")
|
||||
|
||||
auth_connection.create("/test_super_no_acl", b"")
|
||||
auth_connection.create(
|
||||
"/test_super_all_acl", b"data", acl=[make_acl("auth", "", all=True)]
|
||||
)
|
||||
try:
|
||||
auth_connection.add_auth("digest", "user1:password1")
|
||||
auth_connection.create("/test_super_no_acl", b"")
|
||||
auth_connection.create(
|
||||
"/test_super_all_acl", b"data", acl=[make_acl("auth", "", all=True)]
|
||||
)
|
||||
finally:
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
super_connection = get_fake_zk()
|
||||
super_connection.add_auth("digest", "super:admin")
|
||||
|
||||
for path in ["/test_super_no_acl", "/test_super_all_acl"]:
|
||||
super_connection.set(path, b"value")
|
||||
assert super_connection.get(path)[0] == b"value"
|
||||
try:
|
||||
super_connection.add_auth("digest", "super:admin")
|
||||
for path in ["/test_super_no_acl", "/test_super_all_acl"]:
|
||||
super_connection.set(path, b"value")
|
||||
assert super_connection.get(path)[0] == b"value"
|
||||
finally:
|
||||
zk_stop_and_close(super_connection)
|
||||
|
||||
|
||||
@pytest.mark.parametrize(("get_zk"), [get_genuine_zk, get_fake_zk])
|
||||
def test_digest_auth_multiple(started_cluster, get_zk):
|
||||
auth_connection = get_zk()
|
||||
auth_connection.add_auth("digest", "user1:password1")
|
||||
auth_connection.add_auth("digest", "user2:password2")
|
||||
auth_connection.add_auth("digest", "user3:password3")
|
||||
auth_connection = None
|
||||
one_auth_connection = None
|
||||
other_auth_connection = None
|
||||
|
||||
auth_connection.create(
|
||||
"/test_multi_all_acl", b"data", acl=[make_acl("auth", "", all=True)]
|
||||
)
|
||||
try:
|
||||
auth_connection = get_zk()
|
||||
auth_connection.add_auth("digest", "user1:password1")
|
||||
auth_connection.add_auth("digest", "user2:password2")
|
||||
auth_connection.add_auth("digest", "user3:password3")
|
||||
|
||||
one_auth_connection = get_zk()
|
||||
one_auth_connection.add_auth("digest", "user1:password1")
|
||||
auth_connection.create(
|
||||
"/test_multi_all_acl", b"data", acl=[make_acl("auth", "", all=True)]
|
||||
)
|
||||
|
||||
one_auth_connection.set("/test_multi_all_acl", b"X")
|
||||
assert one_auth_connection.get("/test_multi_all_acl")[0] == b"X"
|
||||
one_auth_connection = get_zk()
|
||||
one_auth_connection.add_auth("digest", "user1:password1")
|
||||
|
||||
other_auth_connection = get_zk()
|
||||
other_auth_connection.add_auth("digest", "user2:password2")
|
||||
one_auth_connection.set("/test_multi_all_acl", b"X")
|
||||
assert one_auth_connection.get("/test_multi_all_acl")[0] == b"X"
|
||||
|
||||
other_auth_connection.set("/test_multi_all_acl", b"Y")
|
||||
other_auth_connection = get_zk()
|
||||
other_auth_connection.add_auth("digest", "user2:password2")
|
||||
|
||||
assert other_auth_connection.get("/test_multi_all_acl")[0] == b"Y"
|
||||
other_auth_connection.set("/test_multi_all_acl", b"Y")
|
||||
|
||||
assert other_auth_connection.get("/test_multi_all_acl")[0] == b"Y"
|
||||
finally:
|
||||
zk_stop_and_close(auth_connection)
|
||||
zk_stop_and_close(one_auth_connection)
|
||||
zk_stop_and_close(other_auth_connection)
|
||||
|
||||
|
||||
@pytest.mark.parametrize(("get_zk"), [get_genuine_zk, get_fake_zk])
|
||||
def test_partial_auth(started_cluster, get_zk):
|
||||
auth_connection = get_zk()
|
||||
auth_connection.add_auth("digest", "user1:password1")
|
||||
try:
|
||||
auth_connection.add_auth("digest", "user1:password1")
|
||||
|
||||
auth_connection.create(
|
||||
"/test_partial_acl",
|
||||
b"data",
|
||||
acl=[
|
||||
make_acl(
|
||||
"auth", "", read=False, write=True, create=True, delete=True, admin=True
|
||||
)
|
||||
],
|
||||
)
|
||||
auth_connection.create(
|
||||
"/test_partial_acl",
|
||||
b"data",
|
||||
acl=[
|
||||
make_acl(
|
||||
"auth",
|
||||
"",
|
||||
read=False,
|
||||
write=True,
|
||||
create=True,
|
||||
delete=True,
|
||||
admin=True,
|
||||
)
|
||||
],
|
||||
)
|
||||
|
||||
auth_connection.set("/test_partial_acl", b"X")
|
||||
auth_connection.create(
|
||||
"/test_partial_acl/subnode",
|
||||
b"X",
|
||||
acl=[
|
||||
make_acl(
|
||||
"auth", "", read=False, write=True, create=True, delete=True, admin=True
|
||||
)
|
||||
],
|
||||
)
|
||||
auth_connection.set("/test_partial_acl", b"X")
|
||||
auth_connection.create(
|
||||
"/test_partial_acl/subnode",
|
||||
b"X",
|
||||
acl=[
|
||||
make_acl(
|
||||
"auth",
|
||||
"",
|
||||
read=False,
|
||||
write=True,
|
||||
create=True,
|
||||
delete=True,
|
||||
admin=True,
|
||||
)
|
||||
],
|
||||
)
|
||||
|
||||
with pytest.raises(NoAuthError):
|
||||
auth_connection.get("/test_partial_acl")
|
||||
with pytest.raises(NoAuthError):
|
||||
auth_connection.get("/test_partial_acl")
|
||||
|
||||
with pytest.raises(NoAuthError):
|
||||
auth_connection.get_children("/test_partial_acl")
|
||||
with pytest.raises(NoAuthError):
|
||||
auth_connection.get_children("/test_partial_acl")
|
||||
|
||||
# exists works without read perm
|
||||
assert auth_connection.exists("/test_partial_acl") is not None
|
||||
# exists works without read perm
|
||||
assert auth_connection.exists("/test_partial_acl") is not None
|
||||
|
||||
auth_connection.create(
|
||||
"/test_partial_acl_create",
|
||||
b"data",
|
||||
acl=[
|
||||
make_acl(
|
||||
"auth", "", read=True, write=True, create=False, delete=True, admin=True
|
||||
)
|
||||
],
|
||||
)
|
||||
with pytest.raises(NoAuthError):
|
||||
auth_connection.create("/test_partial_acl_create/subnode")
|
||||
auth_connection.create(
|
||||
"/test_partial_acl_create",
|
||||
b"data",
|
||||
acl=[
|
||||
make_acl(
|
||||
"auth",
|
||||
"",
|
||||
read=True,
|
||||
write=True,
|
||||
create=False,
|
||||
delete=True,
|
||||
admin=True,
|
||||
)
|
||||
],
|
||||
)
|
||||
with pytest.raises(NoAuthError):
|
||||
auth_connection.create("/test_partial_acl_create/subnode")
|
||||
|
||||
auth_connection.create(
|
||||
"/test_partial_acl_set",
|
||||
b"data",
|
||||
acl=[
|
||||
make_acl(
|
||||
"auth", "", read=True, write=False, create=True, delete=True, admin=True
|
||||
)
|
||||
],
|
||||
)
|
||||
with pytest.raises(NoAuthError):
|
||||
auth_connection.set("/test_partial_acl_set", b"X")
|
||||
auth_connection.create(
|
||||
"/test_partial_acl_set",
|
||||
b"data",
|
||||
acl=[
|
||||
make_acl(
|
||||
"auth",
|
||||
"",
|
||||
read=True,
|
||||
write=False,
|
||||
create=True,
|
||||
delete=True,
|
||||
admin=True,
|
||||
)
|
||||
],
|
||||
)
|
||||
with pytest.raises(NoAuthError):
|
||||
auth_connection.set("/test_partial_acl_set", b"X")
|
||||
|
||||
# not allowed to delete child node
|
||||
auth_connection.create(
|
||||
"/test_partial_acl_delete",
|
||||
b"data",
|
||||
acl=[
|
||||
make_acl(
|
||||
"auth", "", read=True, write=True, create=True, delete=False, admin=True
|
||||
)
|
||||
],
|
||||
)
|
||||
auth_connection.create("/test_partial_acl_delete/subnode")
|
||||
with pytest.raises(NoAuthError):
|
||||
auth_connection.delete("/test_partial_acl_delete/subnode")
|
||||
# not allowed to delete child node
|
||||
auth_connection.create(
|
||||
"/test_partial_acl_delete",
|
||||
b"data",
|
||||
acl=[
|
||||
make_acl(
|
||||
"auth",
|
||||
"",
|
||||
read=True,
|
||||
write=True,
|
||||
create=True,
|
||||
delete=False,
|
||||
admin=True,
|
||||
)
|
||||
],
|
||||
)
|
||||
auth_connection.create("/test_partial_acl_delete/subnode")
|
||||
with pytest.raises(NoAuthError):
|
||||
auth_connection.delete("/test_partial_acl_delete/subnode")
|
||||
finally:
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
|
||||
def test_bad_auth(started_cluster):
|
||||
def test_bad_auth_1(started_cluster):
|
||||
auth_connection = get_fake_zk()
|
||||
|
||||
with pytest.raises(AuthFailedError):
|
||||
auth_connection.add_auth("world", "anyone")
|
||||
zk_auth_failure_workaround()
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
|
||||
def test_bad_auth_2(started_cluster):
|
||||
auth_connection = get_fake_zk()
|
||||
with pytest.raises(AuthFailedError):
|
||||
print("Sending 1")
|
||||
auth_connection.add_auth("adssagf", "user1:password1")
|
||||
zk_auth_failure_workaround()
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
|
||||
def test_bad_auth_3(started_cluster):
|
||||
auth_connection = get_fake_zk()
|
||||
with pytest.raises(AuthFailedError):
|
||||
print("Sending 2")
|
||||
auth_connection.add_auth("digest", "")
|
||||
zk_auth_failure_workaround()
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
|
||||
def test_bad_auth_4(started_cluster):
|
||||
auth_connection = get_fake_zk()
|
||||
with pytest.raises(AuthFailedError):
|
||||
print("Sending 3")
|
||||
auth_connection.add_auth("", "user1:password1")
|
||||
zk_auth_failure_workaround()
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
|
||||
def test_bad_auth_5(started_cluster):
|
||||
auth_connection = get_fake_zk()
|
||||
with pytest.raises(AuthFailedError):
|
||||
print("Sending 4")
|
||||
auth_connection.add_auth("digest", "user1")
|
||||
zk_auth_failure_workaround()
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
|
||||
def test_bad_auth_6(started_cluster):
|
||||
auth_connection = get_fake_zk()
|
||||
with pytest.raises(AuthFailedError):
|
||||
print("Sending 5")
|
||||
auth_connection.add_auth("digest", "user1:password:otherpassword")
|
||||
zk_auth_failure_workaround()
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
|
||||
def test_bad_auth_7(started_cluster):
|
||||
auth_connection = get_fake_zk()
|
||||
with pytest.raises(AuthFailedError):
|
||||
print("Sending 6")
|
||||
auth_connection.add_auth("auth", "user1:password")
|
||||
zk_auth_failure_workaround()
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
|
||||
def test_bad_auth_8(started_cluster):
|
||||
auth_connection = get_fake_zk()
|
||||
with pytest.raises(AuthFailedError):
|
||||
print("Sending 7")
|
||||
auth_connection.add_auth("world", "somebody")
|
||||
zk_auth_failure_workaround()
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
|
||||
def test_bad_auth_9(started_cluster):
|
||||
auth_connection = get_fake_zk()
|
||||
with pytest.raises(InvalidACLError):
|
||||
print("Sending 8")
|
||||
@ -358,7 +463,11 @@ def test_bad_auth(started_cluster):
|
||||
)
|
||||
],
|
||||
)
|
||||
zk_auth_failure_workaround()
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
|
||||
def test_bad_auth_10(started_cluster):
|
||||
auth_connection = get_fake_zk()
|
||||
with pytest.raises(InvalidACLError):
|
||||
print("Sending 9")
|
||||
@ -377,7 +486,11 @@ def test_bad_auth(started_cluster):
|
||||
)
|
||||
],
|
||||
)
|
||||
zk_auth_failure_workaround()
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
|
||||
def test_bad_auth_11(started_cluster):
|
||||
auth_connection = get_fake_zk()
|
||||
with pytest.raises(InvalidACLError):
|
||||
print("Sending 10")
|
||||
@ -390,7 +503,11 @@ def test_bad_auth(started_cluster):
|
||||
)
|
||||
],
|
||||
)
|
||||
zk_auth_failure_workaround()
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
|
||||
def test_bad_auth_12(started_cluster):
|
||||
auth_connection = get_fake_zk()
|
||||
with pytest.raises(InvalidACLError):
|
||||
print("Sending 11")
|
||||
@ -409,7 +526,11 @@ def test_bad_auth(started_cluster):
|
||||
)
|
||||
],
|
||||
)
|
||||
zk_auth_failure_workaround()
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
|
||||
def test_bad_auth_13(started_cluster):
|
||||
auth_connection = get_fake_zk()
|
||||
with pytest.raises(InvalidACLError):
|
||||
print("Sending 12")
|
||||
@ -428,117 +549,145 @@ def test_bad_auth(started_cluster):
|
||||
)
|
||||
],
|
||||
)
|
||||
zk_auth_failure_workaround()
|
||||
zk_stop_and_close(auth_connection)
|
||||
|
||||
|
||||
def test_auth_snapshot(started_cluster):
|
||||
connection = get_fake_zk()
|
||||
connection.add_auth("digest", "user1:password1")
|
||||
connection = None
|
||||
connection1 = None
|
||||
connection2 = None
|
||||
|
||||
connection.create(
|
||||
"/test_snapshot_acl", b"data", acl=[make_acl("auth", "", all=True)]
|
||||
)
|
||||
try:
|
||||
connection = get_fake_zk()
|
||||
connection.add_auth("digest", "user1:password1")
|
||||
|
||||
connection1 = get_fake_zk()
|
||||
connection1.add_auth("digest", "user2:password2")
|
||||
|
||||
connection1.create(
|
||||
"/test_snapshot_acl1", b"data", acl=[make_acl("auth", "", all=True)]
|
||||
)
|
||||
|
||||
connection2 = get_fake_zk()
|
||||
|
||||
connection2.create("/test_snapshot_acl2", b"data")
|
||||
|
||||
for i in range(100):
|
||||
connection.create(
|
||||
f"/test_snapshot_acl/path{i}", b"data", acl=[make_acl("auth", "", all=True)]
|
||||
"/test_snapshot_acl", b"data", acl=[make_acl("auth", "", all=True)]
|
||||
)
|
||||
|
||||
node.restart_clickhouse()
|
||||
connection1 = get_fake_zk()
|
||||
connection1.add_auth("digest", "user2:password2")
|
||||
|
||||
connection = get_fake_zk()
|
||||
connection1.create(
|
||||
"/test_snapshot_acl1", b"data", acl=[make_acl("auth", "", all=True)]
|
||||
)
|
||||
|
||||
with pytest.raises(NoAuthError):
|
||||
connection.get("/test_snapshot_acl")
|
||||
connection2 = get_fake_zk()
|
||||
|
||||
connection.add_auth("digest", "user1:password1")
|
||||
connection2.create("/test_snapshot_acl2", b"data")
|
||||
|
||||
assert connection.get("/test_snapshot_acl")[0] == b"data"
|
||||
for i in range(100):
|
||||
connection.create(
|
||||
f"/test_snapshot_acl/path{i}",
|
||||
b"data",
|
||||
acl=[make_acl("auth", "", all=True)],
|
||||
)
|
||||
|
||||
with pytest.raises(NoAuthError):
|
||||
connection.get("/test_snapshot_acl1")
|
||||
node.restart_clickhouse()
|
||||
|
||||
assert connection.get("/test_snapshot_acl2")[0] == b"data"
|
||||
zk_stop_and_close(connection)
|
||||
connection = get_fake_zk()
|
||||
|
||||
for i in range(100):
|
||||
assert connection.get(f"/test_snapshot_acl/path{i}")[0] == b"data"
|
||||
with pytest.raises(NoAuthError):
|
||||
connection.get("/test_snapshot_acl")
|
||||
|
||||
connection1 = get_fake_zk()
|
||||
connection1.add_auth("digest", "user2:password2")
|
||||
connection.add_auth("digest", "user1:password1")
|
||||
|
||||
assert connection1.get("/test_snapshot_acl1")[0] == b"data"
|
||||
assert connection.get("/test_snapshot_acl")[0] == b"data"
|
||||
|
||||
with pytest.raises(NoAuthError):
|
||||
connection1.get("/test_snapshot_acl")
|
||||
with pytest.raises(NoAuthError):
|
||||
connection.get("/test_snapshot_acl1")
|
||||
|
||||
connection2 = get_fake_zk()
|
||||
assert connection2.get("/test_snapshot_acl2")[0] == b"data"
|
||||
with pytest.raises(NoAuthError):
|
||||
connection2.get("/test_snapshot_acl")
|
||||
assert connection.get("/test_snapshot_acl2")[0] == b"data"
|
||||
|
||||
with pytest.raises(NoAuthError):
|
||||
connection2.get("/test_snapshot_acl1")
|
||||
for i in range(100):
|
||||
assert connection.get(f"/test_snapshot_acl/path{i}")[0] == b"data"
|
||||
|
||||
zk_stop_and_close(connection1)
|
||||
connection1 = get_fake_zk()
|
||||
connection1.add_auth("digest", "user2:password2")
|
||||
|
||||
assert connection1.get("/test_snapshot_acl1")[0] == b"data"
|
||||
|
||||
with pytest.raises(NoAuthError):
|
||||
connection1.get("/test_snapshot_acl")
|
||||
|
||||
zk_stop_and_close(connection2)
|
||||
connection2 = get_fake_zk()
|
||||
assert connection2.get("/test_snapshot_acl2")[0] == b"data"
|
||||
with pytest.raises(NoAuthError):
|
||||
connection2.get("/test_snapshot_acl")
|
||||
|
||||
with pytest.raises(NoAuthError):
|
||||
connection2.get("/test_snapshot_acl1")
|
||||
finally:
|
||||
zk_stop_and_close(connection)
|
||||
zk_stop_and_close(connection1)
|
||||
zk_stop_and_close(connection2)
|
||||
|
||||
|
||||
@pytest.mark.parametrize(("get_zk"), [get_genuine_zk, get_fake_zk])
|
||||
def test_get_set_acl(started_cluster, get_zk):
|
||||
auth_connection = get_zk()
|
||||
auth_connection.add_auth("digest", "username1:secret1")
|
||||
auth_connection.add_auth("digest", "username2:secret2")
|
||||
auth_connection = None
|
||||
other_auth_connection = None
|
||||
try:
|
||||
auth_connection = get_zk()
|
||||
auth_connection.add_auth("digest", "username1:secret1")
|
||||
auth_connection.add_auth("digest", "username2:secret2")
|
||||
|
||||
auth_connection.create(
|
||||
"/test_set_get_acl", b"data", acl=[make_acl("auth", "", all=True)]
|
||||
)
|
||||
|
||||
acls, stat = auth_connection.get_acls("/test_set_get_acl")
|
||||
|
||||
assert stat.aversion == 0
|
||||
assert len(acls) == 2
|
||||
for acl in acls:
|
||||
assert acl.acl_list == ["ALL"]
|
||||
assert acl.id.scheme == "digest"
|
||||
assert acl.perms == 31
|
||||
assert acl.id.id in (
|
||||
"username1:eGncMdBgOfGS/TCojt51xWsWv/Y=",
|
||||
"username2:qgSSumukVlhftkVycylbHNvxhFU=",
|
||||
auth_connection.create(
|
||||
"/test_set_get_acl", b"data", acl=[make_acl("auth", "", all=True)]
|
||||
)
|
||||
|
||||
other_auth_connection = get_zk()
|
||||
other_auth_connection.add_auth("digest", "username1:secret1")
|
||||
other_auth_connection.add_auth("digest", "username3:secret3")
|
||||
other_auth_connection.set_acls(
|
||||
"/test_set_get_acl",
|
||||
acls=[
|
||||
make_acl(
|
||||
"auth", "", read=True, write=False, create=True, delete=True, admin=True
|
||||
acls, stat = auth_connection.get_acls("/test_set_get_acl")
|
||||
|
||||
assert stat.aversion == 0
|
||||
assert len(acls) == 2
|
||||
for acl in acls:
|
||||
assert acl.acl_list == ["ALL"]
|
||||
assert acl.id.scheme == "digest"
|
||||
assert acl.perms == 31
|
||||
assert acl.id.id in (
|
||||
"username1:eGncMdBgOfGS/TCojt51xWsWv/Y=",
|
||||
"username2:qgSSumukVlhftkVycylbHNvxhFU=",
|
||||
)
|
||||
],
|
||||
)
|
||||
|
||||
acls, stat = other_auth_connection.get_acls("/test_set_get_acl")
|
||||
|
||||
assert stat.aversion == 1
|
||||
assert len(acls) == 2
|
||||
for acl in acls:
|
||||
assert acl.acl_list == ["READ", "CREATE", "DELETE", "ADMIN"]
|
||||
assert acl.id.scheme == "digest"
|
||||
assert acl.perms == 29
|
||||
assert acl.id.id in (
|
||||
"username1:eGncMdBgOfGS/TCojt51xWsWv/Y=",
|
||||
"username3:CvWITOxxTwk+u6S5PoGlQ4hNoWI=",
|
||||
)
|
||||
|
||||
with pytest.raises(KazooException):
|
||||
other_auth_connection = get_zk()
|
||||
other_auth_connection.add_auth("digest", "username1:secret1")
|
||||
other_auth_connection.add_auth("digest", "username3:secret3")
|
||||
other_auth_connection.set_acls(
|
||||
"/test_set_get_acl", acls=[make_acl("auth", "", all=True)], version=0
|
||||
"/test_set_get_acl",
|
||||
acls=[
|
||||
make_acl(
|
||||
"auth",
|
||||
"",
|
||||
read=True,
|
||||
write=False,
|
||||
create=True,
|
||||
delete=True,
|
||||
admin=True,
|
||||
)
|
||||
],
|
||||
)
|
||||
|
||||
acls, stat = other_auth_connection.get_acls("/test_set_get_acl")
|
||||
|
||||
assert stat.aversion == 1
|
||||
assert len(acls) == 2
|
||||
for acl in acls:
|
||||
assert acl.acl_list == ["READ", "CREATE", "DELETE", "ADMIN"]
|
||||
assert acl.id.scheme == "digest"
|
||||
assert acl.perms == 29
|
||||
assert acl.id.id in (
|
||||
"username1:eGncMdBgOfGS/TCojt51xWsWv/Y=",
|
||||
"username3:CvWITOxxTwk+u6S5PoGlQ4hNoWI=",
|
||||
)
|
||||
|
||||
with pytest.raises(KazooException):
|
||||
other_auth_connection.set_acls(
|
||||
"/test_set_get_acl", acls=[make_acl("auth", "", all=True)], version=0
|
||||
)
|
||||
finally:
|
||||
zk_stop_and_close(auth_connection)
|
||||
zk_stop_and_close(other_auth_connection)
|
||||
|
@ -46,72 +46,85 @@ def start(node):
|
||||
|
||||
|
||||
def test_nodes_add(started_cluster):
|
||||
keeper_utils.wait_until_connected(cluster, node1)
|
||||
zk_conn = get_fake_zk(node1)
|
||||
zk_conn = None
|
||||
zk_conn2 = None
|
||||
zk_conn3 = None
|
||||
|
||||
for i in range(100):
|
||||
zk_conn.create("/test_two_" + str(i), b"somedata")
|
||||
try:
|
||||
keeper_utils.wait_until_connected(cluster, node1)
|
||||
zk_conn = get_fake_zk(node1)
|
||||
|
||||
p = Pool(3)
|
||||
node2.stop_clickhouse()
|
||||
node2.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_two_nodes_2.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper2.xml",
|
||||
)
|
||||
waiter = p.apply_async(start, (node2,))
|
||||
node1.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_two_nodes_1.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper1.xml",
|
||||
)
|
||||
node1.query("SYSTEM RELOAD CONFIG")
|
||||
waiter.wait()
|
||||
keeper_utils.wait_until_connected(cluster, node2)
|
||||
for i in range(100):
|
||||
zk_conn.create("/test_two_" + str(i), b"somedata")
|
||||
|
||||
zk_conn2 = get_fake_zk(node2)
|
||||
p = Pool(3)
|
||||
node2.stop_clickhouse()
|
||||
node2.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_two_nodes_2.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper2.xml",
|
||||
)
|
||||
waiter = p.apply_async(start, (node2,))
|
||||
node1.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_two_nodes_1.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper1.xml",
|
||||
)
|
||||
node1.query("SYSTEM RELOAD CONFIG")
|
||||
waiter.wait()
|
||||
keeper_utils.wait_until_connected(cluster, node2)
|
||||
|
||||
for i in range(100):
|
||||
assert zk_conn2.exists("/test_two_" + str(i)) is not None
|
||||
zk_conn2 = get_fake_zk(node2)
|
||||
|
||||
zk_conn = get_fake_zk(node1)
|
||||
for i in range(100):
|
||||
assert zk_conn2.exists("/test_two_" + str(i)) is not None
|
||||
|
||||
for i in range(100):
|
||||
zk_conn.create("/test_three_" + str(i), b"somedata")
|
||||
zk_conn.stop()
|
||||
zk_conn.close()
|
||||
|
||||
node3.stop_clickhouse()
|
||||
zk_conn = get_fake_zk(node1)
|
||||
|
||||
node3.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_three_nodes_3.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper3.xml",
|
||||
)
|
||||
waiter = p.apply_async(start, (node3,))
|
||||
node2.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_three_nodes_2.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper2.xml",
|
||||
)
|
||||
node1.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_three_nodes_1.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper1.xml",
|
||||
)
|
||||
for i in range(100):
|
||||
zk_conn.create("/test_three_" + str(i), b"somedata")
|
||||
|
||||
node1.query("SYSTEM RELOAD CONFIG")
|
||||
node2.query("SYSTEM RELOAD CONFIG")
|
||||
node3.stop_clickhouse()
|
||||
|
||||
waiter.wait()
|
||||
keeper_utils.wait_until_connected(cluster, node3)
|
||||
zk_conn3 = get_fake_zk(node3)
|
||||
node3.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_three_nodes_3.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper3.xml",
|
||||
)
|
||||
waiter = p.apply_async(start, (node3,))
|
||||
node2.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_three_nodes_2.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper2.xml",
|
||||
)
|
||||
node1.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_three_nodes_1.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper1.xml",
|
||||
)
|
||||
|
||||
for i in range(100):
|
||||
assert zk_conn3.exists("/test_three_" + str(i)) is not None
|
||||
node1.query("SYSTEM RELOAD CONFIG")
|
||||
node2.query("SYSTEM RELOAD CONFIG")
|
||||
|
||||
# configs which change endpoints of server should not be allowed
|
||||
node1.replace_in_config(
|
||||
"/etc/clickhouse-server/config.d/enable_keeper1.xml",
|
||||
"node3",
|
||||
"non_existing_node",
|
||||
)
|
||||
waiter.wait()
|
||||
keeper_utils.wait_until_connected(cluster, node3)
|
||||
zk_conn3 = get_fake_zk(node3)
|
||||
|
||||
node1.query("SYSTEM RELOAD CONFIG")
|
||||
time.sleep(2)
|
||||
assert node1.contains_in_log(
|
||||
"Config will be ignored because a server with ID 3 is already present in the cluster"
|
||||
)
|
||||
for i in range(100):
|
||||
assert zk_conn3.exists("/test_three_" + str(i)) is not None
|
||||
|
||||
# configs which change endpoints of server should not be allowed
|
||||
node1.replace_in_config(
|
||||
"/etc/clickhouse-server/config.d/enable_keeper1.xml",
|
||||
"node3",
|
||||
"non_existing_node",
|
||||
)
|
||||
|
||||
node1.query("SYSTEM RELOAD CONFIG")
|
||||
time.sleep(2)
|
||||
assert node1.contains_in_log(
|
||||
"Config will be ignored because a server with ID 3 is already present in the cluster"
|
||||
)
|
||||
finally:
|
||||
for zk in [zk_conn, zk_conn2, zk_conn3]:
|
||||
if zk:
|
||||
zk.stop()
|
||||
zk.close()
|
||||
|
@ -54,51 +54,62 @@ def get_fake_zk(node, timeout=30.0):
|
||||
|
||||
|
||||
def test_node_move(started_cluster):
|
||||
zk_conn = get_fake_zk(node1)
|
||||
zk_conn = None
|
||||
zk_conn2 = None
|
||||
zk_conn3 = None
|
||||
zk_conn4 = None
|
||||
|
||||
for i in range(100):
|
||||
zk_conn.create("/test_four_" + str(i), b"somedata")
|
||||
try:
|
||||
zk_conn = get_fake_zk(node1)
|
||||
|
||||
zk_conn2 = get_fake_zk(node2)
|
||||
zk_conn2.sync("/test_four_0")
|
||||
for i in range(100):
|
||||
zk_conn.create("/test_four_" + str(i), b"somedata")
|
||||
|
||||
zk_conn3 = get_fake_zk(node3)
|
||||
zk_conn3.sync("/test_four_0")
|
||||
zk_conn2 = get_fake_zk(node2)
|
||||
zk_conn2.sync("/test_four_0")
|
||||
|
||||
for i in range(100):
|
||||
assert zk_conn2.exists("test_four_" + str(i)) is not None
|
||||
assert zk_conn3.exists("test_four_" + str(i)) is not None
|
||||
zk_conn3 = get_fake_zk(node3)
|
||||
zk_conn3.sync("/test_four_0")
|
||||
|
||||
node4.stop_clickhouse()
|
||||
node4.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_node4_4.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper4.xml",
|
||||
)
|
||||
p = Pool(3)
|
||||
waiter = p.apply_async(start, (node4,))
|
||||
node1.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_node4_1.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper1.xml",
|
||||
)
|
||||
node2.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_node4_2.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper2.xml",
|
||||
)
|
||||
for i in range(100):
|
||||
assert zk_conn2.exists("test_four_" + str(i)) is not None
|
||||
assert zk_conn3.exists("test_four_" + str(i)) is not None
|
||||
|
||||
node1.query("SYSTEM RELOAD CONFIG")
|
||||
node2.query("SYSTEM RELOAD CONFIG")
|
||||
node4.stop_clickhouse()
|
||||
node4.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_node4_4.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper4.xml",
|
||||
)
|
||||
p = Pool(3)
|
||||
waiter = p.apply_async(start, (node4,))
|
||||
node1.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_node4_1.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper1.xml",
|
||||
)
|
||||
node2.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_node4_2.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper2.xml",
|
||||
)
|
||||
|
||||
waiter.wait()
|
||||
node1.query("SYSTEM RELOAD CONFIG")
|
||||
node2.query("SYSTEM RELOAD CONFIG")
|
||||
|
||||
zk_conn4 = get_fake_zk(node4)
|
||||
zk_conn4.sync("/test_four_0")
|
||||
waiter.wait()
|
||||
|
||||
for i in range(100):
|
||||
assert zk_conn4.exists("/test_four_" + str(i)) is not None
|
||||
zk_conn4 = get_fake_zk(node4)
|
||||
zk_conn4.sync("/test_four_0")
|
||||
|
||||
with pytest.raises(Exception):
|
||||
# Adding and removing nodes is async operation
|
||||
for i in range(10):
|
||||
zk_conn3 = get_fake_zk(node3)
|
||||
zk_conn3.sync("/test_four_0")
|
||||
time.sleep(i)
|
||||
for i in range(100):
|
||||
assert zk_conn4.exists("/test_four_" + str(i)) is not None
|
||||
|
||||
with pytest.raises(Exception):
|
||||
# Adding and removing nodes is async operation
|
||||
for i in range(10):
|
||||
zk_conn3 = get_fake_zk(node3)
|
||||
zk_conn3.sync("/test_four_0")
|
||||
time.sleep(i)
|
||||
finally:
|
||||
for zk in [zk_conn, zk_conn2, zk_conn3, zk_conn4]:
|
||||
if zk:
|
||||
zk.stop()
|
||||
zk.close()
|
||||
|
@ -40,74 +40,94 @@ def get_fake_zk(node, timeout=30.0):
|
||||
|
||||
|
||||
def test_nodes_remove(started_cluster):
|
||||
zk_conn = get_fake_zk(node1)
|
||||
|
||||
for i in range(100):
|
||||
zk_conn.create("/test_two_" + str(i), b"somedata")
|
||||
|
||||
zk_conn2 = get_fake_zk(node2)
|
||||
zk_conn2.sync("/test_two_0")
|
||||
|
||||
zk_conn3 = get_fake_zk(node3)
|
||||
zk_conn3.sync("/test_two_0")
|
||||
|
||||
for i in range(100):
|
||||
assert zk_conn2.exists("test_two_" + str(i)) is not None
|
||||
assert zk_conn3.exists("test_two_" + str(i)) is not None
|
||||
|
||||
node2.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_two_nodes_2.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper2.xml",
|
||||
)
|
||||
node1.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_two_nodes_1.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper1.xml",
|
||||
)
|
||||
|
||||
node1.query("SYSTEM RELOAD CONFIG")
|
||||
node2.query("SYSTEM RELOAD CONFIG")
|
||||
|
||||
zk_conn2 = get_fake_zk(node2)
|
||||
|
||||
for i in range(100):
|
||||
assert zk_conn2.exists("test_two_" + str(i)) is not None
|
||||
zk_conn2.create("/test_two_" + str(100 + i), b"otherdata")
|
||||
|
||||
zk_conn = get_fake_zk(node1)
|
||||
zk_conn.sync("/test_two_0")
|
||||
|
||||
for i in range(100):
|
||||
assert zk_conn.exists("test_two_" + str(i)) is not None
|
||||
assert zk_conn.exists("test_two_" + str(100 + i)) is not None
|
||||
zk_conn = None
|
||||
zk_conn2 = None
|
||||
zk_conn3 = None
|
||||
|
||||
try:
|
||||
zk_conn3 = get_fake_zk(node3)
|
||||
zk_conn3.sync("/test_two_0")
|
||||
time.sleep(0.1)
|
||||
except Exception:
|
||||
pass
|
||||
zk_conn = get_fake_zk(node1)
|
||||
|
||||
node3.stop_clickhouse()
|
||||
for i in range(100):
|
||||
zk_conn.create("/test_two_" + str(i), b"somedata")
|
||||
|
||||
node1.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_single_keeper1.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper1.xml",
|
||||
)
|
||||
|
||||
node1.query("SYSTEM RELOAD CONFIG")
|
||||
|
||||
zk_conn = get_fake_zk(node1)
|
||||
zk_conn.sync("/test_two_0")
|
||||
|
||||
for i in range(100):
|
||||
assert zk_conn.exists("test_two_" + str(i)) is not None
|
||||
assert zk_conn.exists("test_two_" + str(100 + i)) is not None
|
||||
|
||||
try:
|
||||
zk_conn2 = get_fake_zk(node2)
|
||||
zk_conn2.sync("/test_two_0")
|
||||
time.sleep(0.1)
|
||||
except Exception:
|
||||
pass
|
||||
|
||||
node2.stop_clickhouse()
|
||||
zk_conn3 = get_fake_zk(node3)
|
||||
zk_conn3.sync("/test_two_0")
|
||||
|
||||
for i in range(100):
|
||||
assert zk_conn2.exists("test_two_" + str(i)) is not None
|
||||
assert zk_conn3.exists("test_two_" + str(i)) is not None
|
||||
|
||||
node2.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_two_nodes_2.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper2.xml",
|
||||
)
|
||||
node1.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_keeper_two_nodes_1.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper1.xml",
|
||||
)
|
||||
|
||||
node1.query("SYSTEM RELOAD CONFIG")
|
||||
node2.query("SYSTEM RELOAD CONFIG")
|
||||
|
||||
zk_conn2.stop()
|
||||
zk_conn2.close()
|
||||
zk_conn2 = get_fake_zk(node2)
|
||||
|
||||
for i in range(100):
|
||||
assert zk_conn2.exists("test_two_" + str(i)) is not None
|
||||
zk_conn2.create("/test_two_" + str(100 + i), b"otherdata")
|
||||
|
||||
zk_conn.stop()
|
||||
zk_conn.close()
|
||||
zk_conn = get_fake_zk(node1)
|
||||
zk_conn.sync("/test_two_0")
|
||||
|
||||
for i in range(100):
|
||||
assert zk_conn.exists("test_two_" + str(i)) is not None
|
||||
assert zk_conn.exists("test_two_" + str(100 + i)) is not None
|
||||
|
||||
try:
|
||||
zk_conn3.stop()
|
||||
zk_conn3.close()
|
||||
zk_conn3 = get_fake_zk(node3)
|
||||
zk_conn3.sync("/test_two_0")
|
||||
time.sleep(0.1)
|
||||
except Exception:
|
||||
pass
|
||||
|
||||
node3.stop_clickhouse()
|
||||
|
||||
node1.copy_file_to_container(
|
||||
os.path.join(CONFIG_DIR, "enable_single_keeper1.xml"),
|
||||
"/etc/clickhouse-server/config.d/enable_keeper1.xml",
|
||||
)
|
||||
|
||||
node1.query("SYSTEM RELOAD CONFIG")
|
||||
|
||||
zk_conn.stop()
|
||||
zk_conn.close()
|
||||
zk_conn = get_fake_zk(node1)
|
||||
zk_conn.sync("/test_two_0")
|
||||
|
||||
for i in range(100):
|
||||
assert zk_conn.exists("test_two_" + str(i)) is not None
|
||||
assert zk_conn.exists("test_two_" + str(100 + i)) is not None
|
||||
|
||||
try:
|
||||
zk_conn2.stop()
|
||||
zk_conn2.close()
|
||||
zk_conn2 = get_fake_zk(node2)
|
||||
zk_conn2.sync("/test_two_0")
|
||||
time.sleep(0.1)
|
||||
except Exception:
|
||||
pass
|
||||
|
||||
node2.stop_clickhouse()
|
||||
finally:
|
||||
for zk in [zk_conn, zk_conn2, zk_conn3]:
|
||||
if zk:
|
||||
zk.stop()
|
||||
zk.close()
|
||||
|
@ -35,18 +35,23 @@ def started_cluster():
|
||||
|
||||
|
||||
def test_snapshot_on_exit(started_cluster):
|
||||
zk_conn = get_fake_zk(node1)
|
||||
zk_conn = None
|
||||
try:
|
||||
zk_conn = get_fake_zk(node1)
|
||||
zk_conn.create("/some_path", b"some_data")
|
||||
|
||||
zk_conn.create("/some_path", b"some_data")
|
||||
node1.stop_clickhouse()
|
||||
assert node1.contains_in_log("Created persistent snapshot")
|
||||
|
||||
node1.stop_clickhouse()
|
||||
assert node1.contains_in_log("Created persistent snapshot")
|
||||
node1.start_clickhouse()
|
||||
assert node1.contains_in_log("Loaded snapshot")
|
||||
|
||||
node1.start_clickhouse()
|
||||
assert node1.contains_in_log("Loaded snapshot")
|
||||
node2.stop_clickhouse()
|
||||
assert not node2.contains_in_log("Created persistent snapshot")
|
||||
|
||||
node2.stop_clickhouse()
|
||||
assert not node2.contains_in_log("Created persistent snapshot")
|
||||
|
||||
node2.start_clickhouse()
|
||||
assert node2.contains_in_log("No existing snapshots")
|
||||
node2.start_clickhouse()
|
||||
assert node2.contains_in_log("No existing snapshots")
|
||||
finally:
|
||||
if zk_conn:
|
||||
zk_conn.stop()
|
||||
zk_conn.close()
|
||||
|
@ -112,45 +112,57 @@ def get_genuine_zk(node, timeout=30.0):
|
||||
|
||||
|
||||
def test_snapshot_and_load(started_cluster):
|
||||
restart_and_clear_zookeeper(node1)
|
||||
genuine_connection = get_genuine_zk(node1)
|
||||
for node in [node1, node2, node3]:
|
||||
print("Stop and clear", node.name, "with dockerid", node.docker_id)
|
||||
stop_clickhouse(node)
|
||||
clear_clickhouse_data(node)
|
||||
genuine_connection = None
|
||||
fake_zks = []
|
||||
|
||||
for i in range(1000):
|
||||
genuine_connection.create("/test" + str(i), b"data")
|
||||
try:
|
||||
restart_and_clear_zookeeper(node1)
|
||||
genuine_connection = get_genuine_zk(node1)
|
||||
for node in [node1, node2, node3]:
|
||||
print("Stop and clear", node.name, "with dockerid", node.docker_id)
|
||||
stop_clickhouse(node)
|
||||
clear_clickhouse_data(node)
|
||||
|
||||
print("Data loaded to zookeeper")
|
||||
for i in range(1000):
|
||||
genuine_connection.create("/test" + str(i), b"data")
|
||||
|
||||
stop_zookeeper(node1)
|
||||
start_zookeeper(node1)
|
||||
stop_zookeeper(node1)
|
||||
print("Data loaded to zookeeper")
|
||||
|
||||
print("Data copied to node1")
|
||||
resulted_path = convert_zookeeper_data(node1)
|
||||
print("Resulted path", resulted_path)
|
||||
for node in [node2, node3]:
|
||||
print("Copy snapshot from", node1.name, "to", node.name)
|
||||
cluster.copy_file_from_container_to_container(
|
||||
node1, resulted_path, node, "/var/lib/clickhouse/coordination/snapshots"
|
||||
)
|
||||
stop_zookeeper(node1)
|
||||
start_zookeeper(node1)
|
||||
stop_zookeeper(node1)
|
||||
|
||||
print("Starting clickhouses")
|
||||
print("Data copied to node1")
|
||||
resulted_path = convert_zookeeper_data(node1)
|
||||
print("Resulted path", resulted_path)
|
||||
for node in [node2, node3]:
|
||||
print("Copy snapshot from", node1.name, "to", node.name)
|
||||
cluster.copy_file_from_container_to_container(
|
||||
node1, resulted_path, node, "/var/lib/clickhouse/coordination/snapshots"
|
||||
)
|
||||
|
||||
p = Pool(3)
|
||||
result = p.map_async(start_clickhouse, [node1, node2, node3])
|
||||
result.wait()
|
||||
print("Starting clickhouses")
|
||||
|
||||
print("Loading additional data")
|
||||
fake_zks = [get_fake_zk(node) for node in [node1, node2, node3]]
|
||||
for i in range(1000):
|
||||
fake_zk = random.choice(fake_zks)
|
||||
try:
|
||||
fake_zk.create("/test" + str(i + 1000), b"data")
|
||||
except Exception as ex:
|
||||
print("Got exception:" + str(ex))
|
||||
p = Pool(3)
|
||||
result = p.map_async(start_clickhouse, [node1, node2, node3])
|
||||
result.wait()
|
||||
|
||||
print("Final")
|
||||
fake_zks[0].create("/test10000", b"data")
|
||||
print("Loading additional data")
|
||||
fake_zks = [get_fake_zk(node) for node in [node1, node2, node3]]
|
||||
for i in range(1000):
|
||||
fake_zk = random.choice(fake_zks)
|
||||
try:
|
||||
fake_zk.create("/test" + str(i + 1000), b"data")
|
||||
except Exception as ex:
|
||||
print("Got exception:" + str(ex))
|
||||
|
||||
print("Final")
|
||||
fake_zks[0].create("/test10000", b"data")
|
||||
finally:
|
||||
for zk in fake_zks:
|
||||
if zk:
|
||||
zk.stop()
|
||||
zk.close()
|
||||
if genuine_connection:
|
||||
genuine_connection.stop()
|
||||
genuine_connection.close()
|
||||
|
@ -29,6 +29,8 @@ def get_fake_zk(nodename, timeout=30.0):
|
||||
|
||||
|
||||
def test_smoke():
|
||||
node1_zk = None
|
||||
|
||||
try:
|
||||
cluster.start()
|
||||
|
||||
@ -37,3 +39,7 @@ def test_smoke():
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
if node1_zk:
|
||||
node1_zk.stop()
|
||||
node1_zk.close()
|
||||
|
@ -54,11 +54,16 @@ def start(node):
|
||||
|
||||
def delete_with_retry(node_name, path):
|
||||
for _ in range(30):
|
||||
zk = None
|
||||
try:
|
||||
get_fake_zk(node_name).delete(path)
|
||||
zk = get_fake_zk(node_name)
|
||||
zk.delete(path)
|
||||
return
|
||||
except:
|
||||
time.sleep(0.5)
|
||||
finally:
|
||||
zk.stop()
|
||||
zk.close()
|
||||
raise Exception(f"Cannot delete {path} from node {node_name}")
|
||||
|
||||
|
||||
@ -89,9 +94,14 @@ def test_start_offline(started_cluster):
|
||||
p.map(start, [node1, node2, node3])
|
||||
delete_with_retry("node1", "/test_alive")
|
||||
|
||||
node1_zk.stop()
|
||||
node1_zk.close()
|
||||
|
||||
|
||||
def test_start_non_existing(started_cluster):
|
||||
p = Pool(3)
|
||||
node2_zk = None
|
||||
|
||||
try:
|
||||
node1.stop_clickhouse()
|
||||
node2.stop_clickhouse()
|
||||
@ -134,15 +144,23 @@ def test_start_non_existing(started_cluster):
|
||||
p.map(start, [node1, node2, node3])
|
||||
delete_with_retry("node2", "/test_non_exising")
|
||||
|
||||
if node2_zk:
|
||||
node2_zk.stop()
|
||||
node2_zk.close()
|
||||
|
||||
|
||||
def test_restart_third_node(started_cluster):
|
||||
node1_zk = get_fake_zk("node1")
|
||||
node1_zk.create("/test_restart", b"aaaa")
|
||||
try:
|
||||
node1_zk = get_fake_zk("node1")
|
||||
node1_zk.create("/test_restart", b"aaaa")
|
||||
|
||||
node3.restart_clickhouse()
|
||||
keeper_utils.wait_until_connected(cluster, node3)
|
||||
node3.restart_clickhouse()
|
||||
keeper_utils.wait_until_connected(cluster, node3)
|
||||
|
||||
assert node3.contains_in_log(
|
||||
"Connected to ZooKeeper (or Keeper) before internal Keeper start"
|
||||
)
|
||||
node1_zk.delete("/test_restart")
|
||||
assert node3.contains_in_log(
|
||||
"Connected to ZooKeeper (or Keeper) before internal Keeper start"
|
||||
)
|
||||
node1_zk.delete("/test_restart")
|
||||
finally:
|
||||
node1_zk.stop()
|
||||
node1_zk.close()
|
||||
|
@ -331,10 +331,12 @@ def test_bridge_dies_with_parent(ch_cluster):
|
||||
)
|
||||
logging.debug(f"Bridge is running, gdb output:\n{out}")
|
||||
|
||||
assert clickhouse_pid is None
|
||||
assert bridge_pid is None
|
||||
instance.start_clickhouse(20)
|
||||
instance.query("DROP DICTIONARY lib_dict_c")
|
||||
try:
|
||||
assert clickhouse_pid is None
|
||||
assert bridge_pid is None
|
||||
finally:
|
||||
instance.start_clickhouse(20)
|
||||
instance.query("DROP DICTIONARY lib_dict_c")
|
||||
|
||||
|
||||
def test_path_validation(ch_cluster):
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user