Merge pull request #37460 from ClickHouse/memory-overcommit-improvement

Memory Overcommit: update defaults, exception message and add ProfileEvent
This commit is contained in:
Dmitry Novik 2022-06-01 17:06:33 +02:00 committed by GitHub
commit 7fbe91ca81
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 125 additions and 68 deletions

View File

@ -1745,13 +1745,3 @@ Possible values:
- Positive integer.
Default value: `10000`.
## global_memory_usage_overcommit_max_wait_microseconds {#global_memory_usage_overcommit_max_wait_microseconds}
Sets maximum waiting time for global overcommit tracker.
Possible values:
- Positive integer.
Default value: `200`.

View File

@ -4291,7 +4291,7 @@ Maximum time thread will wait for memory to be freed in the case of memory overc
If the timeout is reached and memory is not freed, an exception is thrown.
Read more about [memory overcommit](memory-overcommit.md).
Default value: `200`.
Default value: `5000000`.
## memory_overcommit_ratio_denominator_for_user

View File

@ -1100,8 +1100,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking);
auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker();
UInt64 max_overcommit_wait_time = config->getUInt64("global_memory_usage_overcommit_max_wait_microseconds", 200);
global_overcommit_tracker->setMaxWaitTime(max_overcommit_wait_time);
total_memory_tracker.setOvercommitTracker(global_overcommit_tracker);
// FIXME logging-related things need synchronization -- see the 'Logger * log' saved

View File

@ -16,6 +16,7 @@
#include <cmath>
#include <random>
#include <cstdlib>
#include <string>
#ifdef MEMORY_TRACKER_DEBUG_CHECKS
@ -52,11 +53,37 @@ namespace DB
}
}
namespace
{
inline std::string_view toDescription(OvercommitResult result)
{
switch (result)
{
case OvercommitResult::NONE:
return "Memory overcommit isn't used. OvercommitTracker isn't set.";
case OvercommitResult::DISABLED:
return "Memory overcommit isn't used. Waiting time or orvercommit denominator are set to zero.";
case OvercommitResult::MEMORY_FREED:
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "OvercommitResult::MEMORY_FREED shouldn't be asked for description");
case OvercommitResult::SELECTED:
return "Query was selected to stop by OvercommitTracker.";
case OvercommitResult::TIMEOUTED:
return "Waiting timeout for memory to be freed is reached.";
case OvercommitResult::NOT_ENOUGH_FREED:
return "Memory overcommit has freed not enough memory.";
}
}
}
namespace ProfileEvents
{
extern const Event QueryMemoryLimitExceeded;
}
using namespace std::chrono_literals;
static constexpr size_t log_peak_memory_usage_every = 1ULL << 30;
MemoryTracker total_memory_tracker(nullptr, VariableContext::Global);
@ -189,11 +216,11 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT
if (unlikely(current_hard_limit && will_be > current_hard_limit) && memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded)
{
bool need_to_throw = true;
OvercommitResult overcommit_result = OvercommitResult::NONE;
if (auto * overcommit_tracker_ptr = overcommit_tracker.load(std::memory_order_relaxed); overcommit_tracker_ptr != nullptr && query_tracker != nullptr)
need_to_throw = overcommit_tracker_ptr->needToStopQuery(query_tracker, size);
overcommit_result = overcommit_tracker_ptr->needToStopQuery(query_tracker, size);
if (need_to_throw)
if (overcommit_result != OvercommitResult::MEMORY_FREED)
{
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global);
@ -201,12 +228,13 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT
const auto * description = description_ptr.load(std::memory_order_relaxed);
throw DB::Exception(
DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED,
"Memory limit{}{} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}",
"Memory limit{}{} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}. OvercommitTracker decision: {}.",
description ? " " : "",
description ? description : "",
formatReadableSizeWithBinarySuffix(will_be),
size,
formatReadableSizeWithBinarySuffix(current_hard_limit));
formatReadableSizeWithBinarySuffix(current_hard_limit),
toDescription(overcommit_result));
}
else
{
@ -337,6 +365,12 @@ OvercommitRatio MemoryTracker::getOvercommitRatio(Int64 limit)
}
void MemoryTracker::setOvercommitWaitingTime(UInt64 wait_time)
{
max_wait_time.store(wait_time * 1us, std::memory_order_relaxed);
}
void MemoryTracker::resetCounters()
{
amount.store(0, std::memory_order_relaxed);

View File

@ -1,6 +1,7 @@
#pragma once
#include <atomic>
#include <chrono>
#include <base/types.h>
#include <Common/CurrentMetrics.h>
#include <Common/VariableContext.h>
@ -73,6 +74,8 @@ private:
/// This description will be used as prefix into log messages (if isn't nullptr)
std::atomic<const char *> description_ptr = nullptr;
std::atomic<std::chrono::microseconds> max_wait_time;
std::atomic<OvercommitTracker *> overcommit_tracker = nullptr;
bool updatePeak(Int64 will_be, bool log_memory_usage);
@ -186,6 +189,13 @@ public:
OvercommitRatio getOvercommitRatio();
OvercommitRatio getOvercommitRatio(Int64 limit);
std::chrono::microseconds getOvercommitWaitingTime()
{
return max_wait_time.load(std::memory_order_relaxed);
}
void setOvercommitWaitingTime(UInt64 wait_time);
void setOvercommitTracker(OvercommitTracker * tracker) noexcept
{
overcommit_tracker.store(tracker, std::memory_order_relaxed);

View File

@ -2,15 +2,20 @@
#include <chrono>
#include <mutex>
#include <Common/ProfileEvents.h>
#include <Interpreters/ProcessList.h>
namespace ProfileEvents
{
extern const Event MemoryOvercommitWaitTimeMicroseconds;
}
using namespace std::chrono_literals;
constexpr std::chrono::microseconds ZERO_MICROSEC = 0us;
OvercommitTracker::OvercommitTracker(std::mutex & global_mutex_)
: max_wait_time(ZERO_MICROSEC)
, picked_tracker(nullptr)
: picked_tracker(nullptr)
, cancellation_state(QueryCancellationState::NONE)
, global_mutex(global_mutex_)
, freed_memory(0)
@ -18,13 +23,7 @@ OvercommitTracker::OvercommitTracker(std::mutex & global_mutex_)
, allow_release(true)
{}
void OvercommitTracker::setMaxWaitTime(UInt64 wait_time)
{
std::lock_guard guard(overcommit_m);
max_wait_time = wait_time * 1us;
}
bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount)
OvercommitResult OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount)
{
// NOTE: Do not change the order of locks
//
@ -35,8 +34,10 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount)
std::unique_lock<std::mutex> global_lock(global_mutex);
std::unique_lock<std::mutex> lk(overcommit_m);
auto max_wait_time = tracker->getOvercommitWaitingTime();
if (max_wait_time == ZERO_MICROSEC)
return true;
return OvercommitResult::DISABLED;
pickQueryToExclude();
assert(cancellation_state != QueryCancellationState::NONE);
@ -50,7 +51,7 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount)
// picked_tracker to be not null pointer.
assert(cancellation_state == QueryCancellationState::SELECTED);
cancellation_state = QueryCancellationState::NONE;
return true;
return OvercommitResult::DISABLED;
}
if (picked_tracker == tracker)
{
@ -58,17 +59,20 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount)
// It may happen even when current state is RUNNING, because
// ThreadStatus::~ThreadStatus may call MemoryTracker::alloc.
cancellation_state = QueryCancellationState::RUNNING;
return true;
return OvercommitResult::SELECTED;
}
allow_release = true;
required_memory += amount;
required_per_thread[tracker] = amount;
auto wait_start_time = std::chrono::system_clock::now();
bool timeout = !cv.wait_for(lk, max_wait_time, [this, tracker]()
{
return required_per_thread[tracker] == 0 || cancellation_state == QueryCancellationState::NONE;
});
auto wait_end_time = std::chrono::system_clock::now();
ProfileEvents::increment(ProfileEvents::MemoryOvercommitWaitTimeMicroseconds, (wait_end_time - wait_start_time) / 1us);
LOG_DEBUG(getLogger(), "Memory was{} freed within timeout", (timeout ? " not" : ""));
required_memory -= amount;
@ -84,7 +88,12 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount)
// As we don't need to free memory, we can continue execution of the selected query.
if (required_memory == 0 && cancellation_state == QueryCancellationState::SELECTED)
reset();
return timeout || still_need != 0;
if (timeout)
return OvercommitResult::TIMEOUTED;
if (still_need != 0)
return OvercommitResult::NOT_ENOUGH_FREED;
else
return OvercommitResult::MEMORY_FREED;
}
void OvercommitTracker::tryContinueQueryExecutionAfterFree(Int64 amount)

View File

@ -36,6 +36,16 @@ struct OvercommitRatio
class MemoryTracker;
enum class OvercommitResult
{
NONE,
DISABLED,
MEMORY_FREED,
SELECTED,
TIMEOUTED,
NOT_ENOUGH_FREED,
};
enum class QueryCancellationState
{
NONE = 0, // Hard limit is not reached, there is no selected query to kill.
@ -52,9 +62,7 @@ enum class QueryCancellationState
// is killed to free memory.
struct OvercommitTracker : boost::noncopyable
{
void setMaxWaitTime(UInt64 wait_time);
bool needToStopQuery(MemoryTracker * tracker, Int64 amount);
OvercommitResult needToStopQuery(MemoryTracker * tracker, Int64 amount);
void tryContinueQueryExecutionAfterFree(Int64 amount);
@ -72,8 +80,6 @@ protected:
std::mutex overcommit_m;
std::condition_variable cv;
std::chrono::microseconds max_wait_time;
// Specifies memory tracker of the chosen to stop query.
// If soft limit is not set, all the queries which reach hard limit must stop.
// This case is represented as picked tracker pointer is set to nullptr and

View File

@ -199,6 +199,7 @@
M(RealTimeMicroseconds, "Total (wall clock) time spent in processing (queries and other tasks) threads (not that this is a sum).") \
M(UserTimeMicroseconds, "Total time spent in processing (queries and other tasks) threads executing CPU instructions in user space. This include time CPU pipeline was stalled due to cache misses, branch mispredictions, hyper-threading, etc.") \
M(SystemTimeMicroseconds, "Total time spent in processing (queries and other tasks) threads executing CPU instructions in OS kernel space. This include time CPU pipeline was stalled due to cache misses, branch mispredictions, hyper-threading, etc.") \
M(MemoryOvercommitWaitTimeMicroseconds, "Total time spent in waiting for memory to be freed in OvercommitTracker.") \
M(SoftPageFaults, "") \
M(HardPageFaults, "") \
\

View File

@ -40,15 +40,17 @@ static constexpr UInt64 WAIT_TIME = 4'000'000;
template <typename T>
void free_not_continue_test(T & overcommit_tracker)
{
overcommit_tracker.setMaxWaitTime(WAIT_TIME);
static constexpr size_t THREADS = 5;
std::vector<MemoryTracker> trackers(THREADS);
for (auto & tracker : trackers)
tracker.setOvercommitWaitingTime(WAIT_TIME);
std::atomic<int> need_to_stop = 0;
std::vector<std::thread> threads;
threads.reserve(THREADS);
MemoryTracker picked;
picked.setOvercommitWaitingTime(WAIT_TIME);
overcommit_tracker.setCandidate(&picked);
for (size_t i = 0; i < THREADS; ++i)
@ -56,7 +58,7 @@ void free_not_continue_test(T & overcommit_tracker)
threads.push_back(std::thread(
[&, i]()
{
if (overcommit_tracker.needToStopQuery(&trackers[i], 100))
if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED)
++need_to_stop;
}
));
@ -96,15 +98,16 @@ TEST(OvercommitTracker, GlobalFreeNotContinue)
template <typename T>
void free_continue_test(T & overcommit_tracker)
{
overcommit_tracker.setMaxWaitTime(WAIT_TIME);
static constexpr size_t THREADS = 5;
std::vector<MemoryTracker> trackers(THREADS);
for (auto & tracker : trackers)
tracker.setOvercommitWaitingTime(WAIT_TIME);
std::atomic<int> need_to_stop = 0;
std::vector<std::thread> threads;
threads.reserve(THREADS);
MemoryTracker picked;
picked.setOvercommitWaitingTime(WAIT_TIME);
overcommit_tracker.setCandidate(&picked);
for (size_t i = 0; i < THREADS; ++i)
@ -112,7 +115,7 @@ void free_continue_test(T & overcommit_tracker)
threads.push_back(std::thread(
[&, i]()
{
if (overcommit_tracker.needToStopQuery(&trackers[i], 100))
if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED)
++need_to_stop;
}
));
@ -152,15 +155,16 @@ TEST(OvercommitTracker, GlobalFreeContinue)
template <typename T>
void free_continue_and_alloc_test(T & overcommit_tracker)
{
overcommit_tracker.setMaxWaitTime(WAIT_TIME);
static constexpr size_t THREADS = 5;
std::vector<MemoryTracker> trackers(THREADS);
for (auto & tracker : trackers)
tracker.setOvercommitWaitingTime(WAIT_TIME);
std::atomic<int> need_to_stop = 0;
std::vector<std::thread> threads;
threads.reserve(THREADS);
MemoryTracker picked;
picked.setOvercommitWaitingTime(WAIT_TIME);
overcommit_tracker.setCandidate(&picked);
for (size_t i = 0; i < THREADS; ++i)
@ -168,7 +172,7 @@ void free_continue_and_alloc_test(T & overcommit_tracker)
threads.push_back(std::thread(
[&, i]()
{
if (overcommit_tracker.needToStopQuery(&trackers[i], 100))
if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED)
++need_to_stop;
}
));
@ -179,9 +183,10 @@ void free_continue_and_alloc_test(T & overcommit_tracker)
[&]()
{
MemoryTracker failed;
failed.setOvercommitWaitingTime(WAIT_TIME);
std::this_thread::sleep_for(1000ms);
overcommit_tracker.tryContinueQueryExecutionAfterFree(5000);
stopped_next = overcommit_tracker.needToStopQuery(&failed, 100);
stopped_next = overcommit_tracker.needToStopQuery(&failed, 100) != OvercommitResult::MEMORY_FREED;
}
).join();
@ -212,15 +217,16 @@ TEST(OvercommitTracker, GlobalFreeContinueAndAlloc)
template <typename T>
void free_continue_and_alloc_2_test(T & overcommit_tracker)
{
overcommit_tracker.setMaxWaitTime(WAIT_TIME);
static constexpr size_t THREADS = 5;
std::vector<MemoryTracker> trackers(THREADS);
for (auto & tracker : trackers)
tracker.setOvercommitWaitingTime(WAIT_TIME);
std::atomic<int> need_to_stop = 0;
std::vector<std::thread> threads;
threads.reserve(THREADS);
MemoryTracker picked;
picked.setOvercommitWaitingTime(WAIT_TIME);
overcommit_tracker.setCandidate(&picked);
for (size_t i = 0; i < THREADS; ++i)
@ -228,7 +234,7 @@ void free_continue_and_alloc_2_test(T & overcommit_tracker)
threads.push_back(std::thread(
[&, i]()
{
if (overcommit_tracker.needToStopQuery(&trackers[i], 100))
if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED)
++need_to_stop;
}
));
@ -239,9 +245,10 @@ void free_continue_and_alloc_2_test(T & overcommit_tracker)
[&]()
{
MemoryTracker failed;
failed.setOvercommitWaitingTime(WAIT_TIME);
std::this_thread::sleep_for(1000ms);
overcommit_tracker.tryContinueQueryExecutionAfterFree(5000);
stopped_next = overcommit_tracker.needToStopQuery(&failed, 100);
stopped_next = overcommit_tracker.needToStopQuery(&failed, 100) != OvercommitResult::MEMORY_FREED;
}
));
@ -280,15 +287,16 @@ TEST(OvercommitTracker, GlobalFreeContinueAndAlloc2)
template <typename T>
void free_continue_and_alloc_3_test(T & overcommit_tracker)
{
overcommit_tracker.setMaxWaitTime(WAIT_TIME);
static constexpr size_t THREADS = 5;
std::vector<MemoryTracker> trackers(THREADS);
for (auto & tracker : trackers)
tracker.setOvercommitWaitingTime(WAIT_TIME);
std::atomic<int> need_to_stop = 0;
std::vector<std::thread> threads;
threads.reserve(THREADS);
MemoryTracker picked;
picked.setOvercommitWaitingTime(WAIT_TIME);
overcommit_tracker.setCandidate(&picked);
for (size_t i = 0; i < THREADS; ++i)
@ -296,7 +304,7 @@ void free_continue_and_alloc_3_test(T & overcommit_tracker)
threads.push_back(std::thread(
[&, i]()
{
if (overcommit_tracker.needToStopQuery(&trackers[i], 100))
if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED)
++need_to_stop;
}
));
@ -307,9 +315,10 @@ void free_continue_and_alloc_3_test(T & overcommit_tracker)
[&]()
{
MemoryTracker failed;
failed.setOvercommitWaitingTime(WAIT_TIME);
std::this_thread::sleep_for(1000ms);
overcommit_tracker.tryContinueQueryExecutionAfterFree(5000);
stopped_next = overcommit_tracker.needToStopQuery(&failed, 100);
stopped_next = overcommit_tracker.needToStopQuery(&failed, 100) != OvercommitResult::MEMORY_FREED;
}
));
@ -348,15 +357,16 @@ TEST(OvercommitTracker, GlobalFreeContinueAndAlloc3)
template <typename T>
void free_continue_2_test(T & overcommit_tracker)
{
overcommit_tracker.setMaxWaitTime(WAIT_TIME);
static constexpr size_t THREADS = 5;
std::vector<MemoryTracker> trackers(THREADS);
for (auto & tracker : trackers)
tracker.setOvercommitWaitingTime(WAIT_TIME);
std::atomic<int> need_to_stop = 0;
std::vector<std::thread> threads;
threads.reserve(THREADS);
MemoryTracker picked;
picked.setOvercommitWaitingTime(WAIT_TIME);
overcommit_tracker.setCandidate(&picked);
for (size_t i = 0; i < THREADS; ++i)
@ -364,7 +374,7 @@ void free_continue_2_test(T & overcommit_tracker)
threads.push_back(std::thread(
[&, i]()
{
if (overcommit_tracker.needToStopQuery(&trackers[i], 100))
if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED)
++need_to_stop;
}
));
@ -404,18 +414,18 @@ TEST(OvercommitTracker, GlobalFreeContinue2)
template <typename T>
void query_stop_not_continue_test(T & overcommit_tracker)
{
overcommit_tracker.setMaxWaitTime(WAIT_TIME);
std::atomic<int> need_to_stop = 0;
MemoryTracker picked;
picked.setOvercommitWaitingTime(WAIT_TIME);
overcommit_tracker.setCandidate(&picked);
MemoryTracker another;
another.setOvercommitWaitingTime(WAIT_TIME);
auto thread = std::thread(
[&]()
{
if (overcommit_tracker.needToStopQuery(&another, 100))
if (overcommit_tracker.needToStopQuery(&another, 100) != OvercommitResult::MEMORY_FREED)
++need_to_stop;
}
);

View File

@ -372,7 +372,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \
M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \
\
M(UInt64, memory_usage_overcommit_max_wait_microseconds, 200, "Maximum time thread will wait for memory to be freed in the case of memory overcommit on user level. If timeout is reached and memory is not freed, exception is thrown.", 0) \
M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.", 0) \
\
M(UInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \
M(UInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.", 0) \

View File

@ -225,6 +225,8 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as
if (settings.memory_tracker_fault_probability)
thread_group->memory_tracker.setFaultProbability(settings.memory_tracker_fault_probability);
thread_group->memory_tracker.setOvercommitWaitingTime(settings.memory_usage_overcommit_max_wait_microseconds);
/// NOTE: Do not set the limit for thread-level memory tracker since it could show unreal values
/// since allocation and deallocation could happen in different threads
}
@ -244,7 +246,6 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as
user_process_list.user_memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage_for_user);
user_process_list.user_memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator_for_user);
user_process_list.user_memory_tracker.setDescription("(for user)");
user_process_list.user_overcommit_tracker.setMaxWaitTime(settings.memory_usage_overcommit_max_wait_microseconds);
if (!user_process_list.user_throttler)
{

View File

@ -1,4 +1,3 @@
<clickhouse>
<max_server_memory_usage>50000000</max_server_memory_usage>
<global_memory_usage_overcommit_max_wait_microseconds>500</global_memory_usage_overcommit_max_wait_microseconds>
</clickhouse>

View File

@ -18,8 +18,8 @@ def start_cluster():
cluster.shutdown()
TEST_QUERY_A = "SELECT number FROM numbers(1000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator_for_user=1"
TEST_QUERY_B = "SELECT number FROM numbers(1000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator_for_user=2"
TEST_QUERY_A = "SELECT number FROM numbers(1000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator_for_user=1, memory_usage_overcommit_max_wait_microseconds=500"
TEST_QUERY_B = "SELECT number FROM numbers(1000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator_for_user=2, memory_usage_overcommit_max_wait_microseconds=500"
def test_overcommited_is_killed():
@ -45,9 +45,8 @@ def test_overcommited_is_killed():
if err == "":
finished = True
assert (
overcommited_killed and finished
), "no overcommited task was killed or all tasks are killed"
assert overcommited_killed, "no overcommited task was killed"
assert finished, "all tasks are killed"
node.query("DROP USER IF EXISTS A")
node.query("DROP USER IF EXISTS B")