Merge branch 'master' into new-block-for-functions-4

This commit is contained in:
Nikolai Kochetov 2020-10-21 16:16:58 +03:00
commit a0da588f06
19 changed files with 114 additions and 54 deletions

View File

@ -30,6 +30,8 @@ namespace ProfileEvents
static constexpr size_t log_peak_memory_usage_every = 1ULL << 30;
thread_local bool MemoryTracker::BlockerInThread::is_blocked = false;
MemoryTracker total_memory_tracker(nullptr, VariableContext::Global);
@ -56,13 +58,15 @@ MemoryTracker::~MemoryTracker()
void MemoryTracker::logPeakMemoryUsage() const
{
const auto * description = description_ptr.load(std::memory_order_relaxed);
LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak));
LOG_DEBUG(&Poco::Logger::get("MemoryTracker"),
"Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak));
}
void MemoryTracker::logMemoryUsage(Int64 current) const
{
const auto * description = description_ptr.load(std::memory_order_relaxed);
LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), "Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(current));
LOG_DEBUG(&Poco::Logger::get("MemoryTracker"),
"Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(current));
}
@ -71,7 +75,7 @@ void MemoryTracker::alloc(Int64 size)
if (size < 0)
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Negative size ({}) is passed to MemoryTracker. It is a bug.", size);
if (blocker.isCancelled())
if (BlockerInThread::isBlocked())
return;
/** Using memory_order_relaxed means that if allocations are done simultaneously,
@ -86,12 +90,15 @@ void MemoryTracker::alloc(Int64 size)
Int64 current_hard_limit = hard_limit.load(std::memory_order_relaxed);
Int64 current_profiler_limit = profiler_limit.load(std::memory_order_relaxed);
/// Cap the limit to the total_memory_tracker, since it may include some drift.
/// Cap the limit to the total_memory_tracker, since it may include some drift
/// for user-level memory tracker.
///
/// And since total_memory_tracker is reset to the process resident
/// memory peridically (in AsynchronousMetrics::update()), any limit can be
/// capped to it, to avoid possible drift.
if (unlikely(current_hard_limit && will_be > current_hard_limit))
if (unlikely(current_hard_limit
&& will_be > current_hard_limit
&& level == VariableContext::User))
{
Int64 total_amount = total_memory_tracker.get();
if (amount > total_amount)
@ -104,10 +111,8 @@ void MemoryTracker::alloc(Int64 size)
std::bernoulli_distribution fault(fault_probability);
if (unlikely(fault_probability && fault(thread_local_rng)))
{
free(size);
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
auto untrack_lock = blocker.cancel(); // NOLINT
BlockerInThread untrack_lock;
ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded);
std::stringstream message;
@ -118,12 +123,13 @@ void MemoryTracker::alloc(Int64 size)
<< " (attempt to allocate chunk of " << size << " bytes)"
<< ", maximum: " << formatReadableSizeWithBinarySuffix(current_hard_limit);
amount.fetch_sub(size, std::memory_order_relaxed);
throw DB::Exception(message.str(), DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED);
}
if (unlikely(current_profiler_limit && will_be > current_profiler_limit))
{
auto no_track = blocker.cancel();
BlockerInThread untrack_lock;
DB::TraceCollector::collect(DB::TraceType::Memory, StackTrace(), size);
setOrRaiseProfilerLimit((will_be + profiler_step - 1) / profiler_step * profiler_step);
}
@ -131,16 +137,14 @@ void MemoryTracker::alloc(Int64 size)
std::bernoulli_distribution sample(sample_probability);
if (unlikely(sample_probability && sample(thread_local_rng)))
{
auto no_track = blocker.cancel();
BlockerInThread untrack_lock;
DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), size);
}
if (unlikely(current_hard_limit && will_be > current_hard_limit))
{
free(size);
/// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc
auto no_track = blocker.cancel(); // NOLINT
BlockerInThread untrack_lock;
ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded);
std::stringstream message;
@ -151,6 +155,7 @@ void MemoryTracker::alloc(Int64 size)
<< " (attempt to allocate chunk of " << size << " bytes)"
<< ", maximum: " << formatReadableSizeWithBinarySuffix(current_hard_limit);
amount.fetch_sub(size, std::memory_order_relaxed);
throw DB::Exception(message.str(), DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED);
}
@ -177,13 +182,13 @@ void MemoryTracker::updatePeak(Int64 will_be)
void MemoryTracker::free(Int64 size)
{
if (blocker.isCancelled())
if (BlockerInThread::isBlocked())
return;
std::bernoulli_distribution sample(sample_probability);
if (unlikely(sample_probability && sample(thread_local_rng)))
{
auto no_track = blocker.cancel();
BlockerInThread untrack_lock;
DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), -size);
}
@ -298,11 +303,3 @@ namespace CurrentMemoryTracker
}
}
}
DB::SimpleActionLock getCurrentMemoryTrackerActionLock()
{
auto * memory_tracker = DB::CurrentThread::getMemoryTracker();
if (!memory_tracker)
return {};
return memory_tracker->blocker.cancel();
}

View File

@ -3,7 +3,6 @@
#include <atomic>
#include <common/types.h>
#include <Common/CurrentMetrics.h>
#include <Common/SimpleActionBlocker.h>
#include <Common/VariableContext.h>
@ -131,8 +130,18 @@ public:
/// Prints info about peak memory consumption into log.
void logPeakMemoryUsage() const;
/// To be able to temporarily stop memory tracker
DB::SimpleActionBlocker blocker;
/// To be able to temporarily stop memory tracking from current thread.
struct BlockerInThread
{
private:
BlockerInThread(const BlockerInThread &) = delete;
BlockerInThread & operator=(const BlockerInThread &) = delete;
static thread_local bool is_blocked;
public:
BlockerInThread() { is_blocked = true; }
~BlockerInThread() { is_blocked = false; }
static bool isBlocked() { return is_blocked; }
};
};
extern MemoryTracker total_memory_tracker;
@ -145,7 +154,3 @@ namespace CurrentMemoryTracker
void realloc(Int64 old_size, Int64 new_size);
void free(Int64 size);
}
/// Holding this object will temporarily disable memory tracking.
DB::SimpleActionLock getCurrentMemoryTrackerActionLock();

View File

@ -1659,7 +1659,7 @@ public:
if (!isUInt8(second_argument))
throw Exception{"Illegal type " + second_argument->getName()
+ " of second argument of function " + getName()
+ ", expected numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
+ ", expected UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
DataTypePtr element = DataTypeFactory::instance().get("IPv6");
return std::make_shared<DataTypeTuple>(DataTypes{element, element});
@ -1673,19 +1673,21 @@ public:
const auto & col_type_name_ip = arguments[0];
const ColumnPtr & column_ip = col_type_name_ip.column;
const auto * col_const_ip_in = checkAndGetColumnConst<ColumnFixedString>(column_ip.get());
const auto * col_ip_in = checkAndGetColumn<ColumnFixedString>(column_ip.get());
if (!col_ip_in)
if (!col_ip_in && !col_const_ip_in)
throw Exception("Illegal column " + arguments[0].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
if (col_ip_in->getN() != IPV6_BINARY_LENGTH)
throw Exception("Illegal type " + col_type_name_ip.type->getName() +
" of column " + col_ip_in->getName() +
" argument of function " + getName() +
", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if ((col_const_ip_in && col_const_ip_in->getValue<String>().size() != IPV6_BINARY_LENGTH) ||
(col_ip_in && col_ip_in->getN() != IPV6_BINARY_LENGTH))
throw Exception("Illegal type " + col_type_name_ip.type->getName() +
" of column " + column_ip->getName() +
" argument of function " + getName() +
", expected FixedString(" + toString(IPV6_BINARY_LENGTH) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
const auto & col_type_name_cidr = arguments[1];
const ColumnPtr & column_cidr = col_type_name_cidr.column;
@ -1698,8 +1700,6 @@ public:
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
const auto & vec_in = col_ip_in->getChars();
auto col_res_lower_range = ColumnFixedString::create(IPV6_BINARY_LENGTH);
auto col_res_upper_range = ColumnFixedString::create(IPV6_BINARY_LENGTH);
@ -1711,14 +1711,24 @@ public:
static constexpr UInt8 max_cidr_mask = IPV6_BINARY_LENGTH * 8;
const String col_const_ip_str = col_const_ip_in ? col_const_ip_in->getValue<String>() : "";
const UInt8 * col_const_ip_value = col_const_ip_in ? reinterpret_cast<const UInt8 *>(col_const_ip_str.c_str()) : nullptr;
for (size_t offset = 0; offset < input_rows_count; ++offset)
{
const size_t offset_ipv6 = offset * IPV6_BINARY_LENGTH;
const UInt8 * ip = col_const_ip_in
? col_const_ip_value
: &col_ip_in->getChars()[offset_ipv6];
UInt8 cidr = col_const_cidr_in
? col_const_cidr_in->getValue<UInt8>()
: col_cidr_in->getData()[offset];
cidr = std::min(cidr, max_cidr_mask);
applyCIDRMask(&vec_in[offset_ipv6], &vec_res_lower_range[offset_ipv6], &vec_res_upper_range[offset_ipv6], cidr);
applyCIDRMask(ip, &vec_res_lower_range[offset_ipv6], &vec_res_upper_range[offset_ipv6], cidr);
}
return ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});
@ -1763,7 +1773,7 @@ public:
if (!isUInt8(second_argument))
throw Exception{"Illegal type " + second_argument->getName()
+ " of second argument of function " + getName()
+ ", expected numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
+ ", expected UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
DataTypePtr element = DataTypeFactory::instance().get("IPv4");
return std::make_shared<DataTypeTuple>(DataTypes{element, element});
@ -1777,8 +1787,9 @@ public:
const auto & col_type_name_ip = arguments[0];
const ColumnPtr & column_ip = col_type_name_ip.column;
const auto * col_const_ip_in = checkAndGetColumnConst<ColumnUInt32>(column_ip.get());
const auto * col_ip_in = checkAndGetColumn<ColumnUInt32>(column_ip.get());
if (!col_ip_in)
if (!col_const_ip_in && !col_ip_in)
throw Exception("Illegal column " + arguments[0].column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
@ -1794,8 +1805,6 @@ public:
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
const auto & vec_in = col_ip_in->getData();
auto col_res_lower_range = ColumnUInt32::create();
auto col_res_upper_range = ColumnUInt32::create();
@ -1807,11 +1816,15 @@ public:
for (size_t i = 0; i < input_rows_count; ++i)
{
UInt32 ip = col_const_ip_in
? col_const_ip_in->getValue<UInt32>()
: col_ip_in->getData()[i];
UInt8 cidr = col_const_cidr_in
? col_const_cidr_in->getValue<UInt8>()
: col_cidr_in->getData()[i];
std::tie(vec_res_lower_range[i], vec_res_upper_range[i]) = applyCIDRMask(vec_in[i], cidr);
std::tie(vec_res_lower_range[i], vec_res_upper_range[i]) = applyCIDRMask(ip, cidr);
}
return ColumnTuple::create(Columns{std::move(col_res_lower_range), std::move(col_res_upper_range)});

View File

@ -413,6 +413,11 @@ BlockIO InterpreterInsertQuery::execute()
res.out = std::move(out_streams.at(0));
res.pipeline.addStorageHolder(table);
if (const auto * mv = dynamic_cast<const StorageMaterializedView *>(table.get()))
{
if (auto inner_table = mv->tryGetTargetTable())
res.pipeline.addStorageHolder(inner_table);
}
return res;
}

View File

@ -233,7 +233,7 @@ void SystemLog<LogElement>::add(const LogElement & element)
/// The size of allocation can be in order of a few megabytes.
/// But this should not be accounted for query memory usage.
/// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flacky.
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
MemoryTracker::BlockerInThread temporarily_disable_memory_tracker;
/// Should not log messages under mutex.
bool queue_is_half_full = false;

View File

@ -157,7 +157,7 @@ static void setExceptionStackTrace(QueryLogElement & elem)
{
/// Disable memory tracker for stack trace.
/// Because if exception is "Memory limit (for query) exceed", then we probably can't allocate another one string.
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
MemoryTracker::BlockerInThread temporarily_disable_memory_tracker;
try
{

View File

@ -408,7 +408,7 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks
/// Memory should not be limited during ATTACH TABLE query.
/// This is already true at the server startup but must be also ensured for manual table ATTACH.
/// Motivation: memory for index is shared between queries - not belong to the query itself.
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
MemoryTracker::BlockerInThread temporarily_disable_memory_tracker;
loadColumns(require_columns_checksums);
loadChecksums(require_columns_checksums);

View File

@ -212,7 +212,7 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Bloc
* And otherwise it will look like excessively growing memory consumption in context of query.
* (observed in long INSERT SELECTs)
*/
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
MemoryTracker::BlockerInThread temporarily_disable_memory_tracker;
/// Write index. The index contains Primary Key value for each `index_granularity` row.

View File

@ -48,7 +48,7 @@ const MarkInCompressedFile & MergeTreeMarksLoader::getMark(size_t row_index, siz
MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl()
{
/// Memory for marks must not be accounted as memory usage for query, because they are stored in shared cache.
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
MemoryTracker::BlockerInThread temporarily_disable_memory_tracker;
size_t file_size = disk->getFileSize(mrk_path);
size_t mark_size = index_granularity_info.getMarkSizeInBytes(columns_in_mark);

View File

@ -239,6 +239,7 @@ Pipe StorageBuffer::read(
}
pipe_from_dst.addTableLock(destination_lock);
pipe_from_dst.addStorageHolder(destination);
}
Pipe pipe_from_buffers;
@ -315,7 +316,7 @@ static void appendBlock(const Block & from, Block & to)
size_t old_rows = to.rows();
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
MemoryTracker::BlockerInThread temporarily_disable_memory_tracker;
try
{
@ -693,7 +694,7 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl
}
auto destination_metadata_snapshot = table->getInMemoryMetadataPtr();
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
MemoryTracker::BlockerInThread temporarily_disable_memory_tracker;
auto insert = std::make_shared<ASTInsertQuery>();
insert->table_id = destination_id;

View File

@ -82,6 +82,7 @@ Pipe StorageMaterializeMySQL::read(
}
Pipe pipe = nested_storage->read(require_columns_name, nested_metadata, query_info, context, processed_stage, max_block_size, num_streams);
pipe.addTableLock(lock);
if (!expressions->children.empty() && !pipe.empty())
{

View File

@ -124,6 +124,7 @@ Pipe StorageMaterializedView::read(
Pipe pipe = storage->read(column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams);
pipe.addTableLock(lock);
pipe.addStorageHolder(storage);
return pipe;
}

View File

@ -344,6 +344,7 @@ Pipe StorageMerge::createSources(
convertingSourceStream(header, metadata_snapshot, *modified_context, modified_query_info.query, pipe, processed_stage);
pipe.addTableLock(struct_lock);
pipe.addStorageHolder(storage);
pipe.addInterpreterContext(modified_context);
}

View File

@ -1,3 +1,5 @@
check invalid params
tests
4
3
2
@ -14,3 +16,5 @@
('192.168.5.2','192.168.5.2')
('0.0.0.0','0.255.255.255')
('240.0.0.0','255.255.255.255')
('240.0.0.0','255.255.255.255')
('248.0.0.0','255.255.255.255')

View File

@ -1,3 +1,9 @@
SELECT 'check invalid params';
SELECT IPv4CIDRToRange(1, 1); -- { serverError 43 }
SELECT IPv4CIDRToRange(toUInt32(1), 512); -- { serverError 43 }
SELECT 'tests';
DROP TABLE IF EXISTS ipv4_range;
CREATE TABLE ipv4_range(ip IPv4, cidr UInt8) ENGINE = Memory;
@ -16,7 +22,9 @@ WITH IPv4CIDRToRange(ip, cidr) as ip_range SELECT ip, cidr, IPv4NumToString(tupl
DROP TABLE ipv4_range;
SELECT IPv4CIDRToRange(toIPv4('192.168.5.2'), 0);
SELEcT IPv4CIDRToRange(toIPv4('255.255.255.255'), 8);
SELECT IPv4CIDRToRange(toIPv4('255.255.255.255'), 8);
SELECT IPv4CIDRToRange(toIPv4('192.168.5.2'), 32);
SELECT IPv4CIDRToRange(toIPv4('0.0.0.0'), 8);
SELECT IPv4CIDRToRange(toIPv4('255.0.0.0'), 4);
SELECT IPv4CIDRToRange(toIPv4('255.0.0.0'), toUInt8(4 + number)) FROM numbers(2);

View File

@ -1,3 +1,5 @@
check invalid params
tests
3
4
3
@ -16,3 +18,5 @@ ffff:: 4 ('f000::','ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff')
('::','ff:ffff:ffff:ffff:ffff:ffff:ffff:ffff')
('f000::','ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff')
1
('2001:db8:0:85a3::ac1f:8001','2001:db8:0:85a3::ac1f:8001')
('2001:db8:0:85a3::ac1f:8000','2001:db8:0:85a3::ac1f:8001')

View File

@ -1,3 +1,11 @@
SELECT 'check invalid params';
SELECT IPv6CIDRToRange(1, 1); -- { serverError 43 }
SELECT IPv6CIDRToRange('1234', 1); -- { serverError 43 }
SELECT IPv6CIDRToRange(toFixedString('1234', 10), 1); -- { serverError 43 }
SELECT IPv6CIDRToRange(toFixedString('1234', 16), toUInt16(1)); -- { serverError 43 }
SELECT 'tests';
DROP TABLE IF EXISTS ipv6_range;
CREATE TABLE ipv6_range(ip IPv6, cidr UInt8) ENGINE = Memory;
@ -23,3 +31,5 @@ SELECT IPv6CIDRToRange(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'
SELECT IPv6CIDRToRange(IPv6StringToNum('0000:0000:0000:0000:0000:0000:0000:0000'), 8);
SELECT IPv6CIDRToRange(IPv6StringToNum('ffff:0000:0000:0000:0000:0000:0000:0000'), 4);
SELECT IPv6CIDRToRange(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 128) = IPv6CIDRToRange(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), 200) ;
SELECT IPv6CIDRToRange(IPv6StringToNum('2001:0db8:0000:85a3:0000:0000:ac1f:8001'), toUInt8(128 - number)) FROM numbers(2);

View File

@ -0,0 +1,10 @@
#!/usr/bin/env bash
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
for _ in {1..10}; do
${CLICKHOUSE_CLIENT} --max_memory_usage '10G' --query "SELECT i FROM generateRandom('i Array(Int8)', 1, 1, 1048577) LIMIT 65536" 2>&1 | grep -v -P '^(Received exception from server|Code: 241)' ||:
done