Merged Limits to Settings [#CLICKHOUSE-2021]

This commit is contained in:
Alexey Milovidov 2018-03-11 03:15:26 +03:00
parent afbbd780e3
commit 5ff433b670
50 changed files with 288 additions and 586 deletions

View File

@ -371,6 +371,7 @@ namespace ErrorCodes
extern const int THERE_IS_NO_QUERY = 393;
extern const int QUERY_WAS_CANCELLED = 394;
extern const int FUNCTION_THROW_IF_VALUE_IS_NON_ZERO = 395;
extern const int TOO_MANY_ROWS_OR_BYTES = 396;
extern const int KEEPER_EXCEPTION = 999;

View File

@ -115,25 +115,8 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery)
rows_to_transfer += block.rows();
bytes_to_transfer += block.bytes();
if ((max_rows_to_transfer && rows_to_transfer > max_rows_to_transfer)
|| (max_bytes_to_transfer && bytes_to_transfer > max_bytes_to_transfer))
{
switch (transfer_overflow_mode)
{
case OverflowMode::THROW:
throw Exception("IN/JOIN external table size limit exceeded."
" Rows: " + toString(rows_to_transfer)
+ ", limit: " + toString(max_rows_to_transfer)
+ ". Bytes: " + toString(bytes_to_transfer)
+ ", limit: " + toString(max_bytes_to_transfer) + ".",
ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
case OverflowMode::BREAK:
done_with_table = true;
break;
default:
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
}
}
if (!network_transfer_limits.check(rows_to_transfer, bytes_to_transfer, "IN/JOIN external table", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED))
done_with_table = true;
}
if (done_with_set && done_with_join && done_with_table)

View File

@ -20,11 +20,9 @@ public:
CreatingSetsBlockInputStream(
const BlockInputStreamPtr & input,
const SubqueriesForSets & subqueries_for_sets_,
const Limits & limits)
const SizeLimits & network_transfer_limits)
: subqueries_for_sets(subqueries_for_sets_),
max_rows_to_transfer(limits.max_rows_to_transfer),
max_bytes_to_transfer(limits.max_bytes_to_transfer),
transfer_overflow_mode(limits.transfer_overflow_mode)
network_transfer_limits(network_transfer_limits)
{
for (auto & elem : subqueries_for_sets)
if (elem.second.source)
@ -48,9 +46,7 @@ private:
SubqueriesForSets subqueries_for_sets;
bool created = false;
size_t max_rows_to_transfer;
size_t max_bytes_to_transfer;
OverflowMode transfer_overflow_mode;
SizeLimits network_transfer_limits;
size_t rows_to_transfer = 0;
size_t bytes_to_transfer = 0;

View File

@ -8,12 +8,10 @@ namespace ErrorCodes
extern const int SET_SIZE_LIMIT_EXCEEDED;
}
DistinctBlockInputStream::DistinctBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, const Names & columns)
DistinctBlockInputStream::DistinctBlockInputStream(const BlockInputStreamPtr & input, const SizeLimits & set_size_limits, size_t limit_hint_, const Names & columns)
: columns_names(columns)
, limit_hint(limit_hint_)
, max_rows(limits.max_rows_in_distinct)
, max_bytes(limits.max_bytes_in_distinct)
, overflow_mode(limits.distinct_overflow_mode)
, set_size_limits(set_size_limits)
{
children.push_back(input);
}
@ -68,25 +66,8 @@ Block DistinctBlockInputStream::readImpl()
if (data.getTotalRowCount() == old_set_size)
continue;
if (!checkLimits())
{
switch (overflow_mode)
{
case OverflowMode::THROW:
throw Exception("DISTINCT-Set size limit exceeded."
" Rows: " + toString(data.getTotalRowCount()) +
", limit: " + toString(max_rows) +
". Bytes: " + toString(data.getTotalByteCount()) +
", limit: " + toString(max_bytes) + ".",
ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
case OverflowMode::BREAK:
return Block();
default:
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
}
}
if (!set_size_limits.check(data.getTotalRowCount(), data.getTotalByteCount(), "DISTINCT", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED))
return {};
for (auto & elem : block)
elem.column = elem.column->filter(filter, -1);
@ -95,14 +76,6 @@ Block DistinctBlockInputStream::readImpl()
}
}
bool DistinctBlockInputStream::checkLimits() const
{
if (max_rows && data.getTotalRowCount() > max_rows)
return false;
if (max_bytes && data.getTotalByteCount() > max_bytes)
return false;
return true;
}
template <typename Method>
void DistinctBlockInputStream::buildFilter(
@ -133,6 +106,7 @@ void DistinctBlockInputStream::buildFilter(
}
}
ColumnRawPtrs DistinctBlockInputStream::getKeyColumns(const Block & block) const
{
size_t columns = columns_names.empty() ? block.columns() : columns_names.size();

View File

@ -1,7 +1,6 @@
#pragma once
#include <DataStreams/IProfilingBlockInputStream.h>
#include <Interpreters/Limits.h>
#include <Interpreters/SetVariants.h>
namespace DB
@ -18,7 +17,7 @@ class DistinctBlockInputStream : public IProfilingBlockInputStream
{
public:
/// Empty columns_ means all collumns.
DistinctBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, const Names & columns);
DistinctBlockInputStream(const BlockInputStreamPtr & input, const SizeLimits & set_size_limits, size_t limit_hint_, const Names & columns);
String getName() const override { return "Distinct"; }
@ -28,8 +27,6 @@ protected:
Block readImpl() override;
private:
bool checkLimits() const;
ColumnRawPtrs getKeyColumns(const Block & block) const;
template <typename Method>
@ -49,9 +46,7 @@ private:
bool no_more_rows = false;
/// Restrictions on the maximum size of the output data.
size_t max_rows;
size_t max_bytes;
OverflowMode overflow_mode;
SizeLimits set_size_limits;
};
}

View File

@ -8,13 +8,12 @@ namespace ErrorCodes
extern const int SET_SIZE_LIMIT_EXCEEDED;
}
DistinctSortedBlockInputStream::DistinctSortedBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, const Names & columns)
DistinctSortedBlockInputStream::DistinctSortedBlockInputStream(
const BlockInputStreamPtr & input, const SizeLimits & set_size_limits, size_t limit_hint_, const Names & columns)
: description(input->getSortDescription())
, columns_names(columns)
, limit_hint(limit_hint_)
, max_rows(limits.max_rows_in_distinct)
, max_bytes(limits.max_bytes_in_distinct)
, overflow_mode(limits.distinct_overflow_mode)
, set_size_limits(set_size_limits)
{
children.push_back(input);
}
@ -62,25 +61,8 @@ Block DistinctSortedBlockInputStream::readImpl()
if (!has_new_data)
continue;
if (!checkLimits())
{
switch (overflow_mode)
{
case OverflowMode::THROW:
throw Exception("DISTINCT-Set size limit exceeded."
" Rows: " + toString(data.getTotalRowCount()) +
", limit: " + toString(max_rows) +
". Bytes: " + toString(data.getTotalByteCount()) +
", limit: " + toString(max_bytes) + ".",
ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
case OverflowMode::BREAK:
return Block();
default:
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
}
}
if (!set_size_limits.check(data.getTotalRowCount(), data.getTotalByteCount(), "DISTINCT", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED))
return {};
prev_block.block = block;
prev_block.clearing_hint_columns = std::move(clearing_hint_columns);
@ -93,14 +75,6 @@ Block DistinctSortedBlockInputStream::readImpl()
}
}
bool DistinctSortedBlockInputStream::checkLimits() const
{
if (max_rows && data.getTotalRowCount() > max_rows)
return false;
if (max_bytes && data.getTotalByteCount() > max_bytes)
return false;
return true;
}
template <typename Method>
bool DistinctSortedBlockInputStream::buildFilter(

View File

@ -1,9 +1,9 @@
#pragma once
#include <DataStreams/IProfilingBlockInputStream.h>
#include <Interpreters/Limits.h>
#include <Interpreters/SetVariants.h>
namespace DB
{
@ -21,7 +21,7 @@ class DistinctSortedBlockInputStream : public IProfilingBlockInputStream
{
public:
/// Empty columns_ means all collumns.
DistinctSortedBlockInputStream(const BlockInputStreamPtr & input, const Limits & limits, size_t limit_hint_, const Names & columns);
DistinctSortedBlockInputStream(const BlockInputStreamPtr & input, const SizeLimits & set_size_limits, size_t limit_hint_, const Names & columns);
String getName() const override { return "DistinctSorted"; }
@ -31,8 +31,6 @@ protected:
Block readImpl() override;
private:
bool checkLimits() const;
ColumnRawPtrs getKeyColumns(const Block & block) const;
/// When clearing_columns changed, we can clean HashSet to memory optimization
/// clearing_columns is a left-prefix of SortDescription exists in key_columns
@ -64,9 +62,7 @@ private:
size_t limit_hint;
/// Restrictions on the maximum size of the output data.
size_t max_rows;
size_t max_bytes;
OverflowMode overflow_mode;
SizeLimits set_size_limits;
};
}

View File

@ -1,6 +1,3 @@
#include <iomanip>
#include <random>
#include <Interpreters/Quota.h>
#include <Interpreters/ProcessList.h>
#include <DataStreams/IProfilingBlockInputStream.h>
@ -13,6 +10,7 @@ namespace ErrorCodes
{
extern const int TOO_MANY_ROWS;
extern const int TOO_MANY_BYTES;
extern const int TOO_MANY_ROWS_OR_BYTES;
extern const int TIMEOUT_EXCEEDED;
extern const int TOO_SLOW;
extern const int LOGICAL_ERROR;
@ -57,7 +55,7 @@ Block IProfilingBlockInputStream::read()
if (enabled_extremes)
updateExtremes(res);
if (!checkDataSizeLimits())
if (limits.mode == LIMITS_CURRENT && !limits.size_limits.check(info.rows, info.bytes, "result", ErrorCodes::TOO_MANY_ROWS_OR_BYTES))
limit_exceeded_need_break = true;
if (quota != nullptr)
@ -192,30 +190,6 @@ static bool handleOverflowMode(OverflowMode mode, const String & message, int co
}
};
bool IProfilingBlockInputStream::checkDataSizeLimits()
{
if (limits.mode == LIMITS_CURRENT)
{
/// Check current stream limitations (i.e. max_result_{rows,bytes})
if (limits.max_rows_to_read && info.rows > limits.max_rows_to_read)
return handleOverflowMode(limits.read_overflow_mode,
std::string("Limit for result rows")
+ " exceeded: read " + toString(info.rows)
+ " rows, maximum: " + toString(limits.max_rows_to_read),
ErrorCodes::TOO_MANY_ROWS);
if (limits.max_bytes_to_read && info.bytes > limits.max_bytes_to_read)
return handleOverflowMode(limits.read_overflow_mode,
std::string("Limit for result bytes (uncompressed)")
+ " exceeded: read " + toString(info.bytes)
+ " bytes, maximum: " + toString(limits.max_bytes_to_read),
ErrorCodes::TOO_MANY_BYTES);
}
return true;
}
bool IProfilingBlockInputStream::checkTimeLimits()
{
@ -276,29 +250,29 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value)
*/
if (limits.mode == LIMITS_TOTAL
&& ((limits.max_rows_to_read && total_rows_estimate > limits.max_rows_to_read)
|| (limits.max_bytes_to_read && progress.bytes > limits.max_bytes_to_read)))
&& ((limits.size_limits.max_rows && total_rows_estimate > limits.size_limits.max_rows)
|| (limits.size_limits.max_bytes && progress.bytes > limits.size_limits.max_bytes)))
{
switch (limits.read_overflow_mode)
switch (limits.size_limits.overflow_mode)
{
case OverflowMode::THROW:
{
if (limits.max_rows_to_read && total_rows_estimate > limits.max_rows_to_read)
if (limits.size_limits.max_rows && total_rows_estimate > limits.size_limits.max_rows)
throw Exception("Limit for rows to read exceeded: " + toString(total_rows_estimate)
+ " rows read (or to read), maximum: " + toString(limits.max_rows_to_read),
+ " rows read (or to read), maximum: " + toString(limits.size_limits.max_rows),
ErrorCodes::TOO_MANY_ROWS);
else
throw Exception("Limit for (uncompressed) bytes to read exceeded: " + toString(progress.bytes)
+ " bytes read, maximum: " + toString(limits.max_bytes_to_read),
+ " bytes read, maximum: " + toString(limits.size_limits.max_bytes),
ErrorCodes::TOO_MANY_BYTES);
break;
}
case OverflowMode::BREAK:
{
/// For `break`, we will stop only if so many lines were actually read, and not just supposed to be read.
if ((limits.max_rows_to_read && progress.rows > limits.max_rows_to_read)
|| (limits.max_bytes_to_read && progress.bytes > limits.max_bytes_to_read))
/// For `break`, we will stop only if so many rows were actually read, and not just supposed to be read.
if ((limits.size_limits.max_rows && progress.rows > limits.size_limits.max_rows)
|| (limits.size_limits.max_bytes && progress.bytes > limits.size_limits.max_bytes))
{
cancel(false);
}

View File

@ -2,13 +2,15 @@
#include <IO/Progress.h>
#include <Interpreters/Limits.h>
#include <DataStreams/BlockStreamProfileInfo.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataStreams/SizeLimits.h>
#include <Interpreters/SettingsCommon.h>
#include <atomic>
namespace DB
{
@ -151,10 +153,7 @@ public:
{
LimitsMode mode = LIMITS_CURRENT;
/// If it is zero, corresponding limit check isn't performed.
size_t max_rows_to_read = 0;
size_t max_bytes_to_read = 0;
OverflowMode read_overflow_mode = OverflowMode::THROW;
SizeLimits size_limits;
Poco::Timespan max_execution_time = 0;
OverflowMode timeout_overflow_mode = OverflowMode::THROW;
@ -238,7 +237,6 @@ private:
/** Check constraints and quotas.
* But only those that can be tested within each separate source.
*/
bool checkDataSizeLimits();
bool checkTimeLimits();
void checkQuota(Block & block);

View File

@ -0,0 +1,33 @@
#include <DataStreams/SizeLimits.h>
#include <Common/formatReadable.h>
#include <Common/Exception.h>
#include <string>
namespace DB
{
bool SizeLimits::check(UInt64 rows, UInt64 bytes, const char * what, int exception_code) const
{
if (max_rows && rows > max_rows)
{
if (overflow_mode == OverflowMode::THROW)
throw Exception("Limit for " + std::string(what) + " exceeded, max rows: " + formatReadableQuantity(max_rows)
+ ", current rows: " + formatReadableQuantity(rows), exception_code);
else
return false;
}
if (max_bytes && bytes > max_bytes)
{
if (overflow_mode == OverflowMode::THROW)
throw Exception("Limit for " + std::string(what) + " exceeded, max bytes: " + formatReadableSizeWithBinarySuffix(max_bytes)
+ ", current bytes: " + formatReadableSizeWithBinarySuffix(bytes), exception_code);
else
return false;
}
return true;
}
}

View File

@ -0,0 +1,37 @@
#pragma once
#include <common/Types.h>
namespace DB
{
/// What to do if the limit is exceeded.
enum class OverflowMode
{
THROW = 0, /// Throw exception.
BREAK = 1, /// Abort query execution, return what is.
/** Only for GROUP BY: do not add new rows to the set,
* but continue to aggregate for keys that are already in the set.
*/
ANY = 2,
};
struct SizeLimits
{
/// If it is zero, corresponding limit check isn't performed.
UInt64 max_rows = 0;
UInt64 max_bytes = 0;
OverflowMode overflow_mode = OverflowMode::THROW;
SizeLimits() {}
SizeLimits(UInt64 max_rows, UInt64 max_bytes, OverflowMode overflow_mode)
: max_rows(max_rows), max_bytes(max_bytes), overflow_mode(overflow_mode) {}
/// Check limits. If exceeded, return false or throw an exception, depending on overflow_mode.
bool check(UInt64 rows, UInt64 bytes, const char * what, int exception_code) const;
};
}

View File

@ -15,10 +15,10 @@
#include <common/ThreadPool.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataStreams/SizeLimits.h>
#include <Interpreters/AggregateDescription.h>
#include <Interpreters/AggregationCommon.h>
#include <Interpreters/Limits.h>
#include <Interpreters/Compiler.h>
#include <Columns/ColumnString.h>

View File

@ -219,7 +219,7 @@ Cluster::Cluster(Poco::Util::AbstractConfiguration & config, const Settings & se
settings.distributed_connections_pool_size,
address.host_name, address.port, address.resolved_address,
address.default_database, address.user, address.password,
ConnectionTimeouts::getTCPTimeouts(settings).getSaturated(settings.limits.max_execution_time),
ConnectionTimeouts::getTCPTimeouts(settings).getSaturated(settings.max_execution_time),
"server", Protocol::Compression::Enable, Protocol::Encryption::Disable);
info.pool = std::make_shared<ConnectionPoolWithFailover>(
@ -302,7 +302,7 @@ Cluster::Cluster(Poco::Util::AbstractConfiguration & config, const Settings & se
settings.distributed_connections_pool_size,
replica.host_name, replica.port, replica.resolved_address,
replica.default_database, replica.user, replica.password,
ConnectionTimeouts::getTCPTimeouts(settings).getSaturated(settings.limits.max_execution_time),
ConnectionTimeouts::getTCPTimeouts(settings).getSaturated(settings.max_execution_time),
"server", Protocol::Compression::Enable, Protocol::Encryption::Disable);
remote_replicas_pools.emplace_back(replica_pool);
@ -366,7 +366,7 @@ Cluster::Cluster(const Settings & settings, const std::vector<std::vector<String
settings.distributed_connections_pool_size,
replica.host_name, replica.port, replica.resolved_address,
replica.default_database, replica.user, replica.password,
ConnectionTimeouts::getHTTPTimeouts(settings).getSaturated(settings.limits.max_execution_time),
ConnectionTimeouts::getHTTPTimeouts(settings).getSaturated(settings.max_execution_time),
"server", Protocol::Compression::Enable, Protocol::Encryption::Disable);
all_replicas.emplace_back(replica_pool);
remote_replicas.emplace_back(replica_pool);

View File

@ -23,38 +23,38 @@ BlockInputStreams executeQuery(
const std::string query = queryToString(query_ast);
Settings new_settings = settings;
new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.limits.max_execution_time);
new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.max_execution_time);
/// Does not matter on remote servers, because queries are sent under different user.
new_settings.max_concurrent_queries_for_user = 0;
new_settings.limits.max_memory_usage_for_user = 0;
new_settings.max_memory_usage_for_user = 0;
/// This setting is really not for user and should not be sent to remote server.
new_settings.limits.max_memory_usage_for_all_queries = 0;
new_settings.max_memory_usage_for_all_queries = 0;
/// Set as unchanged to avoid sending to remote server.
new_settings.max_concurrent_queries_for_user.changed = false;
new_settings.limits.max_memory_usage_for_user.changed = false;
new_settings.limits.max_memory_usage_for_all_queries.changed = false;
new_settings.max_memory_usage_for_user.changed = false;
new_settings.max_memory_usage_for_all_queries.changed = false;
Context new_context(context);
new_context.setSettings(new_settings);
ThrottlerPtr user_level_throttler;
if (settings.limits.max_network_bandwidth_for_user)
if (settings.max_network_bandwidth_for_user)
if (auto process_list_element = context.getProcessListElement())
user_level_throttler = process_list_element->getUserNetworkThrottler();
/// Network bandwidth limit, if needed.
ThrottlerPtr throttler;
if (settings.limits.max_network_bandwidth || settings.limits.max_network_bytes)
if (settings.max_network_bandwidth || settings.max_network_bytes)
{
throttler = std::make_shared<Throttler>(
settings.limits.max_network_bandwidth,
settings.limits.max_network_bytes,
settings.max_network_bandwidth,
settings.max_network_bytes,
"Limit for bytes to send or receive over network exceeded.",
user_level_throttler);
}
else if (settings.limits.max_network_bandwidth_for_user)
else if (settings.max_network_bandwidth_for_user)
throttler = user_level_throttler;
for (const auto & shard_info : cluster->getShardsInfo())

View File

@ -936,12 +936,6 @@ Settings Context::getSettings() const
}
Limits Context::getLimits() const
{
return settings.limits;
}
void Context::setSettings(const Settings & settings_)
{
settings = settings_;

View File

@ -212,8 +212,6 @@ public:
Settings getSettings() const;
void setSettings(const Settings & settings_);
Limits getLimits() const;
/// Set a setting by name.
void setSetting(const String & name, const Field & value);

View File

@ -233,7 +233,7 @@ DDLWorker::DDLWorker(const std::string & zk_root_dir, Context & context_, const
context.setSetting("profile", config->getString(prefix + ".profile"));
}
if (context.getSettingsRef().limits.readonly)
if (context.getSettingsRef().readonly)
{
LOG_WARNING(log, "Distributed DDL worker is run with readonly settings, it will not be able to execute DDL queries"
<< " Set apropriate system_profile or distributed_ddl.profile to fix this.");

View File

@ -499,20 +499,19 @@ std::string ExpressionAction::toString() const
void ExpressionActions::checkLimits(Block & block) const
{
const Limits & limits = settings.limits;
if (limits.max_temporary_columns && block.columns() > limits.max_temporary_columns)
if (settings.max_temporary_columns && block.columns() > settings.max_temporary_columns)
throw Exception("Too many temporary columns: " + block.dumpNames()
+ ". Maximum: " + limits.max_temporary_columns.toString(),
+ ". Maximum: " + settings.max_temporary_columns.toString(),
ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS);
if (limits.max_temporary_non_const_columns)
if (settings.max_temporary_non_const_columns)
{
size_t non_const_columns = 0;
for (size_t i = 0, size = block.columns(); i < size; ++i)
if (block.safeGetByPosition(i).column && !block.safeGetByPosition(i).column->isColumnConst())
++non_const_columns;
if (non_const_columns > limits.max_temporary_non_const_columns)
if (non_const_columns > settings.max_temporary_non_const_columns)
{
std::stringstream list_of_non_const_columns;
for (size_t i = 0, size = block.columns(); i < size; ++i)
@ -520,7 +519,7 @@ void ExpressionActions::checkLimits(Block & block) const
list_of_non_const_columns << "\n" << block.safeGetByPosition(i).name;
throw Exception("Too many temporary non-const columns:" + list_of_non_const_columns.str()
+ ". Maximum: " + limits.max_temporary_non_const_columns.toString(),
+ ". Maximum: " + settings.max_temporary_non_const_columns.toString(),
ErrorCodes::TOO_MANY_TEMPORARY_NON_CONST_COLUMNS);
}
}

View File

@ -687,8 +687,8 @@ static std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
*/
Context subquery_context = context;
Settings subquery_settings = context.getSettings();
subquery_settings.limits.max_result_rows = 0;
subquery_settings.limits.max_result_bytes = 0;
subquery_settings.max_result_rows = 0;
subquery_settings.max_result_bytes = 0;
/// The calculation of `extremes` does not make sense and is not necessary (if you do it, then the `extremes` of the subquery can be taken instead of the whole query).
subquery_settings.extremes = 0;
subquery_context.setSettings(subquery_settings);
@ -936,7 +936,7 @@ void ExpressionAnalyzer::normalizeTree()
try
{
ast->checkSize(settings.limits.max_expanded_ast_elements);
ast->checkSize(settings.max_expanded_ast_elements);
}
catch (Exception & e)
{
@ -952,9 +952,9 @@ void ExpressionAnalyzer::normalizeTree()
void ExpressionAnalyzer::normalizeTreeImpl(
ASTPtr & ast, MapOfASTs & finished_asts, SetOfASTs & current_asts, std::string current_alias, size_t level)
{
if (level > settings.limits.max_ast_depth)
if (level > settings.max_ast_depth)
throw Exception("Normalized AST is too deep. Maximum: "
+ settings.limits.max_ast_depth.toString(), ErrorCodes::TOO_DEEP_AST);
+ settings.max_ast_depth.toString(), ErrorCodes::TOO_DEEP_AST);
if (finished_asts.count(ast))
{
@ -1210,7 +1210,7 @@ void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast)
{
Context subquery_context = context;
Settings subquery_settings = context.getSettings();
subquery_settings.limits.max_result_rows = 1;
subquery_settings.max_result_rows = 1;
subquery_settings.extremes = 0;
subquery_context.setSettings(subquery_settings);
@ -1465,7 +1465,7 @@ void ExpressionAnalyzer::tryMakeSetFromSubquery(const ASTPtr & subquery_or_table
{
BlockIO res = interpretSubquery(subquery_or_table_name, context, subquery_depth + 1, {})->execute();
SetPtr set = std::make_shared<Set>(settings.limits);
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode));
while (Block block = res.in->read())
{
@ -1571,7 +1571,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
return;
}
SetPtr set = std::make_shared<Set>(settings.limits);
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode));
/** The following happens for GLOBAL INs:
* - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1,
@ -1714,7 +1714,7 @@ void ExpressionAnalyzer::makeExplicitSet(const ASTFunction * node, const Block &
elements_ast = exp_list;
}
SetPtr set = std::make_shared<Set>(settings.limits);
SetPtr set = std::make_shared<Set>(SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode));
set->createFromAST(set_element_types, elements_ast, context, create_ordered_set);
prepared_sets[arg.get()] = std::move(set);
}
@ -2390,7 +2390,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
{
JoinPtr join = std::make_shared<Join>(
join_key_names_left, join_key_names_right,
settings.join_use_nulls, settings.limits,
settings.join_use_nulls, SizeLimits(settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode),
join_params.kind, join_params.strictness);
Names required_joined_columns(join_key_names_right.begin(), join_key_names_right.end());

View File

@ -589,7 +589,7 @@ void InterpreterCreateQuery::checkAccess(const ASTCreateQuery & create)
return;
const Settings & settings = context.getSettingsRef();
auto readonly = settings.limits.readonly;
auto readonly = settings.readonly;
if (!readonly)
{

View File

@ -192,7 +192,7 @@ BlockIO InterpreterDropQuery::execute()
void InterpreterDropQuery::checkAccess(const ASTDropQuery & drop)
{
const Settings & settings = context.getSettingsRef();
auto readonly = settings.limits.readonly;
auto readonly = settings.readonly;
/// It's allowed to drop temporary tables.
if (!readonly || (drop.database.empty() && context.tryGetExternalTable(drop.table) && readonly >= 2))

View File

@ -49,7 +49,7 @@ namespace ErrorCodes
static void throwIfReadOnly(Context & context)
{
if (context.getSettingsRef().limits.readonly)
if (context.getSettingsRef().readonly)
throw Exception("Cannot execute query in readonly mode", ErrorCodes::READONLY);
}

View File

@ -144,7 +144,7 @@ BlockIO InterpreterInsertQuery::execute()
void InterpreterInsertQuery::checkAccess(const ASTInsertQuery & query)
{
const Settings & settings = context.getSettingsRef();
auto readonly = settings.limits.readonly;
auto readonly = settings.readonly;
if (!readonly || (query.database.empty() && context.tryGetExternalTable(query.table) && readonly >= 2))
{

View File

@ -85,7 +85,7 @@ static QueryDescriptors extractQueriesExceptMeAndCheckAccess(const Block & proce
if (my_client.current_query_id == query_id && my_client.current_user == user)
continue;
if (context.getSettingsRef().limits.readonly && my_client.current_user != user)
if (context.getSettingsRef().readonly && my_client.current_user != user)
{
throw Exception("Readonly user " + my_client.current_user + " attempts to kill query created by " + user,
ErrorCodes::READONLY);

View File

@ -109,8 +109,8 @@ void InterpreterSelectQuery::init(const Names & required_result_column_names)
initSettings();
const Settings & settings = context.getSettingsRef();
if (settings.limits.max_subquery_depth && subquery_depth > settings.limits.max_subquery_depth)
throw Exception("Too deep subqueries. Maximum: " + settings.limits.max_subquery_depth.toString(),
if (settings.max_subquery_depth && subquery_depth > settings.max_subquery_depth)
throw Exception("Too deep subqueries. Maximum: " + settings.max_subquery_depth.toString(),
ErrorCodes::TOO_DEEP_SUBQUERIES);
max_streams = settings.max_threads;
@ -348,8 +348,8 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
bool aggregate_overflow_row =
expressions.need_aggregate &&
query.group_by_with_totals &&
settings.limits.max_rows_to_group_by &&
settings.limits.group_by_overflow_mode == OverflowMode::ANY &&
settings.max_rows_to_group_by &&
settings.group_by_overflow_mode == OverflowMode::ANY &&
settings.totals_mode != TotalsMode::AFTER_HAVING_EXCLUSIVE;
/// Do I need to immediately finalize the aggregate functions after the aggregation?
@ -555,8 +555,8 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline
*/
Context subquery_context = context;
Settings subquery_settings = context.getSettings();
subquery_settings.limits.max_result_rows = 0;
subquery_settings.limits.max_result_bytes = 0;
subquery_settings.max_result_rows = 0;
subquery_settings.max_result_bytes = 0;
/// The calculation of extremes does not make sense and is not necessary (if you do it, then the extremes of the subquery can be taken for whole query).
subquery_settings.extremes = 0;
subquery_context.setSettings(subquery_settings);
@ -573,10 +573,10 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline
/// Limitation on the number of columns to read.
/// It's not applied in 'dry_run' mode, because the query could be analyzed without removal of unnecessary columns.
if (!dry_run && settings.limits.max_columns_to_read && required_columns.size() > settings.limits.max_columns_to_read)
if (!dry_run && settings.max_columns_to_read && required_columns.size() > settings.max_columns_to_read)
throw Exception("Limit for number of columns to read exceeded. "
"Requested: " + toString(required_columns.size())
+ ", maximum: " + settings.limits.max_columns_to_read.toString(),
+ ", maximum: " + settings.max_columns_to_read.toString(),
ErrorCodes::TOO_MANY_COLUMNS);
size_t limit_length = 0;
@ -685,13 +685,11 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline
{
IProfilingBlockInputStream::LocalLimits limits;
limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL;
limits.max_rows_to_read = settings.limits.max_rows_to_read;
limits.max_bytes_to_read = settings.limits.max_bytes_to_read;
limits.read_overflow_mode = settings.limits.read_overflow_mode;
limits.max_execution_time = settings.limits.max_execution_time;
limits.timeout_overflow_mode = settings.limits.timeout_overflow_mode;
limits.min_execution_speed = settings.limits.min_execution_speed;
limits.timeout_before_checking_execution_speed = settings.limits.timeout_before_checking_execution_speed;
limits.size_limits = SizeLimits(settings.max_rows_to_read, settings.max_bytes_to_read, settings.read_overflow_mode);
limits.max_execution_time = settings.max_execution_time;
limits.timeout_overflow_mode = settings.timeout_overflow_mode;
limits.min_execution_speed = settings.min_execution_speed;
limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed;
QuotaForIntervals & quota = context.getQuota();
@ -756,14 +754,14 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre
* 1. Parallel aggregation is done, and the results should be merged in parallel.
* 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way.
*/
bool allow_to_use_two_level_group_by = pipeline.streams.size() > 1 || settings.limits.max_bytes_before_external_group_by != 0;
bool allow_to_use_two_level_group_by = pipeline.streams.size() > 1 || settings.max_bytes_before_external_group_by != 0;
Aggregator::Params params(header, keys, aggregates,
overflow_row, settings.limits.max_rows_to_group_by, settings.limits.group_by_overflow_mode,
overflow_row, settings.max_rows_to_group_by, settings.group_by_overflow_mode,
settings.compile ? &context.getCompiler() : nullptr, settings.min_count_to_compile,
allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0),
allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0),
settings.limits.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
context.getTemporaryPath());
/// If there are several sources, then we perform parallel aggregation
@ -928,9 +926,7 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline)
/// Limits on sorting
IProfilingBlockInputStream::LocalLimits limits;
limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL;
limits.max_rows_to_read = settings.limits.max_rows_to_sort;
limits.max_bytes_to_read = settings.limits.max_bytes_to_sort;
limits.read_overflow_mode = settings.limits.sort_overflow_mode;
limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode);
sorting_stream->setLimits(limits);
stream = sorting_stream;
@ -942,7 +938,7 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline)
/// Merge the sorted blocks.
pipeline.firstStream() = std::make_shared<MergeSortingBlockInputStream>(
pipeline.firstStream(), order_descr, settings.max_block_size, limit,
settings.limits.max_bytes_before_external_sort, context.getTemporaryPath());
settings.max_bytes_before_external_sort, context.getTemporaryPath());
}
@ -998,10 +994,12 @@ void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_or
pipeline.transform([&](auto & stream)
{
SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode);
if (stream->isGroupedOutput())
stream = std::make_shared<DistinctSortedBlockInputStream>(stream, settings.limits, limit_for_distinct, columns);
stream = std::make_shared<DistinctSortedBlockInputStream>(stream, limits, limit_for_distinct, columns);
else
stream = std::make_shared<DistinctBlockInputStream>(stream, settings.limits, limit_for_distinct, columns);
stream = std::make_shared<DistinctBlockInputStream>(stream, limits, limit_for_distinct, columns);
});
}
}
@ -1137,7 +1135,9 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline
const Settings & settings = context.getSettingsRef();
executeUnion(pipeline);
pipeline.firstStream() = std::make_shared<CreatingSetsBlockInputStream>(pipeline.firstStream(), subqueries_for_sets, settings.limits);
pipeline.firstStream() = std::make_shared<CreatingSetsBlockInputStream>(
pipeline.firstStream(), subqueries_for_sets,
SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode));
}

View File

@ -35,7 +35,7 @@ void InterpreterSetQuery::checkAccess(const ASTSetQuery & ast)
*/
const Settings & settings = context.getSettingsRef();
auto readonly = settings.limits.readonly;
auto readonly = settings.readonly;
for (const auto & change : ast.changes)
{

View File

@ -31,15 +31,13 @@ namespace ErrorCodes
Join::Join(const Names & key_names_left_, const Names & key_names_right_, bool use_nulls_,
const Limits & limits, ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_)
const SizeLimits & limits, ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_)
: kind(kind_), strictness(strictness_),
key_names_left(key_names_left_),
key_names_right(key_names_right_),
use_nulls(use_nulls_),
log(&Logger::get("Join")),
max_rows(limits.max_rows_in_join),
max_bytes(limits.max_bytes_in_join),
overflow_mode(limits.join_overflow_mode)
limits(limits)
{
}
@ -243,16 +241,6 @@ size_t Join::getTotalByteCount() const
}
bool Join::checkSizeLimits() const
{
if (max_rows && getTotalRowCount() > max_rows)
return false;
if (max_bytes && getTotalByteCount() > max_bytes)
return false;
return true;
}
static void convertColumnToNullable(ColumnWithTypeAndName & column)
{
column.type = makeNullable(column.type);
@ -522,27 +510,7 @@ bool Join::insertFromBlock(const Block & block)
}
}
if (!checkSizeLimits())
{
switch (overflow_mode)
{
case OverflowMode::THROW:
throw Exception("Join size limit exceeded."
" Rows: " + toString(getTotalRowCount()) +
", limit: " + toString(max_rows) +
". Bytes: " + toString(getTotalByteCount()) +
", limit: " + toString(max_bytes) + ".",
ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
case OverflowMode::BREAK:
return false;
default:
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
}
}
return true;
return limits.check(getTotalRowCount(), getTotalByteCount(), "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
}

View File

@ -13,6 +13,7 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <DataStreams/SizeLimits.h>
#include <DataStreams/IBlockInputStream.h>
@ -159,9 +160,6 @@ struct JoinKeyGetterHashed
struct Limits;
/** Data structure for implementation of JOIN.
* It is just a hash table: keys -> rows of joined ("right") table.
* Additionally, CROSS JOIN is supported: instead of hash table, it use just set of blocks without keys.
@ -222,7 +220,7 @@ class Join
{
public:
Join(const Names & key_names_left_, const Names & key_names_right_, bool use_nulls_,
const Limits & limits, ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_);
const SizeLimits & limits, ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_);
bool empty() { return type == Type::EMPTY; }
@ -394,9 +392,7 @@ private:
Poco::Logger * log;
/// Limits for maximum map size.
size_t max_rows;
size_t max_bytes;
OverflowMode overflow_mode;
SizeLimits limits;
Block totals;
@ -409,8 +405,6 @@ private:
void init(Type type_);
bool checkSizeLimits() const;
/// Throw an exception if blocks have different types of key columns.
void checkTypesOfKeys(const Block & block_left, const Block & block_right) const;

View File

@ -1,185 +0,0 @@
#pragma once
#include <Core/Defines.h>
#include <Core/Field.h>
#include <Interpreters/SettingsCommon.h>
namespace DB
{
/** Limits during query execution are part of the settings.
* Used to provide a more safe execution of queries from the user interface.
* Basically, limits are checked for each block (not every row). That is, the limits can be slightly violated.
* Almost all limits apply only to SELECTs.
* Almost all limits apply to each stream individually.
*/
struct Limits
{
#define APPLY_FOR_LIMITS(M) \
M(SettingUInt64, max_rows_to_read, 0, "Limit on read rows from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.") \
M(SettingUInt64, max_bytes_to_read, 0, "Limit on read bytes (after decompression) from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.") \
M(SettingOverflowMode<false>, read_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
\
M(SettingUInt64, max_rows_to_group_by, 0, "") \
M(SettingOverflowMode<true>, group_by_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
M(SettingUInt64, max_bytes_before_external_group_by, 0, "") \
\
M(SettingUInt64, max_rows_to_sort, 0, "") \
M(SettingUInt64, max_bytes_to_sort, 0, "") \
M(SettingOverflowMode<false>, sort_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
M(SettingUInt64, max_bytes_before_external_sort, 0, "") \
\
M(SettingUInt64, max_result_rows, 0, "Limit on result size in rows. Also checked for intermediate data sent from remote servers.") \
M(SettingUInt64, max_result_bytes, 0, "Limit on result size in bytes (uncompressed). Also checked for intermediate data sent from remote servers.") \
M(SettingOverflowMode<false>, result_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
\
/* TODO: Check also when merging and finalizing aggregate functions. */ \
M(SettingSeconds, max_execution_time, 0, "") \
M(SettingOverflowMode<false>, timeout_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
\
M(SettingUInt64, min_execution_speed, 0, "In rows per second.") \
M(SettingSeconds, timeout_before_checking_execution_speed, 0, "Check that the speed is not too low after the specified time has elapsed.") \
\
M(SettingUInt64, max_columns_to_read, 0, "") \
M(SettingUInt64, max_temporary_columns, 0, "") \
M(SettingUInt64, max_temporary_non_const_columns, 0, "") \
\
M(SettingUInt64, max_subquery_depth, 100, "") \
M(SettingUInt64, max_pipeline_depth, 1000, "") \
M(SettingUInt64, max_ast_depth, 1000, "Maximum depth of query syntax tree. Checked after parsing.") \
M(SettingUInt64, max_ast_elements, 50000, "Maximum size of query syntax tree in number of nodes. Checked after parsing.") \
M(SettingUInt64, max_expanded_ast_elements, 500000, "Maximum size of query syntax tree in number of nodes after expansion of aliases and the asterisk.") \
\
M(SettingUInt64, readonly, 0, "0 - everything is allowed. 1 - only read requests. 2 - only read requests, as well as changing settings, except for the 'readonly' setting.") \
\
M(SettingUInt64, max_rows_in_set, 0, "Maximum size of the set (in number of elements) resulting from the execution of the IN section.") \
M(SettingUInt64, max_bytes_in_set, 0, "Maximum size of the set (in bytes in memory) resulting from the execution of the IN section.") \
M(SettingOverflowMode<false>, set_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
\
M(SettingUInt64, max_rows_in_join, 0, "Maximum size of the hash table for JOIN (in number of rows).") \
M(SettingUInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).") \
M(SettingOverflowMode<false>, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
\
M(SettingUInt64, max_rows_to_transfer, 0, "Maximum size (in rows) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.") \
M(SettingUInt64, max_bytes_to_transfer, 0, "Maximum size (in uncompressed bytes) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.") \
M(SettingOverflowMode<false>, transfer_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
\
M(SettingUInt64, max_rows_in_distinct, 0, "Maximum number of elements during execution of DISTINCT.") \
M(SettingUInt64, max_bytes_in_distinct, 0, "Maximum total size of state (in uncompressed bytes) in memory for the execution of DISTINCT.") \
M(SettingOverflowMode<false>, distinct_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
\
M(SettingUInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.") \
M(SettingUInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.") \
M(SettingUInt64, max_memory_usage_for_all_queries, 0, "Maximum memory usage for processing all concurrently running queries on the server. Zero means unlimited.") \
\
M(SettingUInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second. Zero means unlimited.") \
M(SettingUInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.") \
M(SettingUInt64, max_network_bandwidth_for_user, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries for the user. Zero means unlimited.")
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
TYPE NAME {DEFAULT};
APPLY_FOR_LIMITS(DECLARE)
#undef DECLARE
/// Set setting by name.
bool trySet(const String & name, const Field & value)
{
#define TRY_SET(TYPE, NAME, DEFAULT, DESCRIPTION) \
else if (name == #NAME) NAME.set(value);
if (false) {}
APPLY_FOR_LIMITS(TRY_SET)
else
return false;
return true;
#undef TRY_SET
}
/// Set the setting by name. Read the binary serialized value from the buffer (for server-to-server interaction).
bool trySet(const String & name, ReadBuffer & buf)
{
#define TRY_SET(TYPE, NAME, DEFAULT, DESCRIPTION) \
else if (name == #NAME) NAME.set(buf);
if (false) {}
APPLY_FOR_LIMITS(TRY_SET)
else
return false;
return true;
#undef TRY_SET
}
/// Skip the binary-serialized value from the buffer.
bool tryIgnore(const String & name, ReadBuffer & buf)
{
#define TRY_IGNORE(TYPE, NAME, DEFAULT, DESCRIPTION) \
else if (name == #NAME) decltype(NAME)(DEFAULT).set(buf);
if (false) {}
APPLY_FOR_LIMITS(TRY_IGNORE)
else
return false;
return true;
#undef TRY_IGNORE
}
/** Set the setting by name. Read the value in text form from a string (for example, from a config, or from a URL parameter).
*/
bool trySet(const String & name, const String & value)
{
#define TRY_SET(TYPE, NAME, DEFAULT, DESCRIPTION) \
else if (name == #NAME) NAME.set(value);
if (false) {}
APPLY_FOR_LIMITS(TRY_SET)
else
return false;
return true;
#undef TRY_SET
}
bool tryGet(const String & name, String & value) const
{
#define TRY_GET(TYPE, NAME, DEFAULT, DESCRIPTION) \
else if (name == #NAME) { value = NAME.toString(); return true; }
if (false) {}
APPLY_FOR_LIMITS(TRY_GET)
return false;
#undef TRY_GET
}
private:
friend struct Settings;
/// Write all the settings to the buffer. (Unlike the corresponding method in Settings, the empty line on the end is not written).
void serialize(WriteBuffer & buf) const
{
#define WRITE(TYPE, NAME, DEFAULT, DESCRIPTION) \
if (NAME.changed) \
{ \
writeStringBinary(#NAME, buf); \
NAME.write(buf); \
}
APPLY_FOR_LIMITS(WRITE)
#undef WRITE
}
};
}

View File

@ -75,7 +75,7 @@ ProcessList::EntryPtr ProcessList::insert(
res = std::make_shared<Entry>(*this, cont.emplace(cont.end(),
query_, client_info,
settings.limits.max_memory_usage, settings.memory_tracker_fault_probability,
settings.max_memory_usage, settings.memory_tracker_fault_probability,
priorities.insert(settings.priority)));
ProcessListForUser & user_process_list = user_to_queries[client_info.current_user];
@ -88,7 +88,7 @@ ProcessList::EntryPtr ProcessList::insert(
/// setting from one query effectively sets values for all other queries.
/// Track memory usage for all simultaneously running queries from single user.
user_process_list.user_memory_tracker.setOrRaiseLimit(settings.limits.max_memory_usage_for_user);
user_process_list.user_memory_tracker.setOrRaiseLimit(settings.max_memory_usage_for_user);
user_process_list.user_memory_tracker.setDescription("(for user)");
current_memory_tracker->setNext(&user_process_list.user_memory_tracker);
@ -96,14 +96,14 @@ ProcessList::EntryPtr ProcessList::insert(
/// You should specify this value in configuration for default profile,
/// not for specific users, sessions or queries,
/// because this setting is effectively global.
total_memory_tracker.setOrRaiseLimit(settings.limits.max_memory_usage_for_all_queries);
total_memory_tracker.setOrRaiseLimit(settings.max_memory_usage_for_all_queries);
total_memory_tracker.setDescription("(total)");
user_process_list.user_memory_tracker.setNext(&total_memory_tracker);
}
if (settings.limits.max_network_bandwidth_for_user && !user_process_list.user_throttler)
if (settings.max_network_bandwidth_for_user && !user_process_list.user_throttler)
{
user_process_list.user_throttler = std::make_shared<Throttler>(settings.limits.max_network_bandwidth_for_user, 0,
user_process_list.user_throttler = std::make_shared<Throttler>(settings.max_network_bandwidth_for_user, 0,
"Network bandwidth limit for a user exceeded.");
}

View File

@ -37,15 +37,6 @@ namespace ErrorCodes
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
}
bool Set::checkSetSizeLimits() const
{
if (max_rows && data.getTotalRowCount() > max_rows)
return false;
if (max_bytes && data.getTotalByteCount() > max_bytes)
return false;
return true;
}
template <typename Method>
void NO_INLINE Set::insertFromBlockImpl(
@ -189,27 +180,7 @@ bool Set::insertFromBlock(const Block & block, bool fill_set_elements)
}
}
if (!checkSetSizeLimits())
{
switch (overflow_mode)
{
case OverflowMode::THROW:
throw Exception("IN-set size exceeded."
" Rows: " + toString(data.getTotalRowCount()) +
", limit: " + toString(max_rows) +
". Bytes: " + toString(data.getTotalByteCount()) +
", limit: " + toString(max_bytes) + ".",
ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
case OverflowMode::BREAK:
return false;
default:
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
}
}
return true;
return limits.check(getTotalRowCount(), getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
}
static Field extractValueFromNode(ASTPtr & node, const IDataType & type, const Context & context)

View File

@ -1,10 +1,12 @@
#pragma once
#include <shared_mutex>
#include <Core/Block.h>
#include <Columns/ColumnArray.h>
#include <DataStreams/IBlockInputStream.h>
#include <Interpreters/Limits.h>
#include <DataStreams/SizeLimits.h>
#include <DataTypes/IDataType.h>
#include <Interpreters/SetVariants.h>
#include <Interpreters/Context.h>
#include <Parsers/IAST.h>
#include <Storages/MergeTree/BoolMask.h>
@ -28,11 +30,9 @@ using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
class Set
{
public:
Set(const Limits & limits) :
Set(const SizeLimits & limits) :
log(&Logger::get("Set")),
max_rows(limits.max_rows_in_set),
max_bytes(limits.max_bytes_in_set),
overflow_mode(limits.set_overflow_mode),
limits(limits),
set_elements(std::make_unique<SetElements>())
{
}
@ -87,9 +87,7 @@ private:
Logger * log;
/// Limitations on the maximum size of the set
size_t max_rows;
size_t max_bytes;
OverflowMode overflow_mode;
SizeLimits limits;
/// If there is an array on the left side of IN. We check that at least one element of the array presents in the set.
void executeArray(const ColumnArray * key_column, ColumnUInt8::Container & vec_res, bool negative) const;
@ -101,9 +99,6 @@ private:
bool negative,
const PaddedPODArray<UInt8> * null_map) const;
/// Check whether the permissible sizes of keys set reached
bool checkSetSizeLimits() const;
/// Vector of elements of `Set`.
/// It is necessary for the index to work on the primary key in the IN statement.
SetElementsPtr set_elements;

View File

@ -20,7 +20,7 @@ void Settings::set(const String & name, const Field & value)
if (false) {}
APPLY_FOR_SETTINGS(TRY_SET)
else if (!limits.trySet(name, value))
else
throw Exception("Unknown setting " + name, ErrorCodes::UNKNOWN_SETTING);
#undef TRY_SET
@ -34,7 +34,7 @@ void Settings::set(const String & name, ReadBuffer & buf)
if (false) {}
APPLY_FOR_SETTINGS(TRY_SET)
else if (!limits.trySet(name, buf))
else
throw Exception("Unknown setting " + name, ErrorCodes::UNKNOWN_SETTING);
#undef TRY_SET
@ -48,7 +48,7 @@ void Settings::ignore(const String & name, ReadBuffer & buf)
if (false) {}
APPLY_FOR_SETTINGS(TRY_IGNORE)
else if (!limits.tryIgnore(name, buf))
else
throw Exception("Unknown setting " + name, ErrorCodes::UNKNOWN_SETTING);
#undef TRY_IGNORE
@ -63,7 +63,7 @@ void Settings::set(const String & name, const String & value)
if (false) {}
APPLY_FOR_SETTINGS(TRY_SET)
else if (!limits.trySet(name, value))
else
throw Exception("Unknown setting " + name, ErrorCodes::UNKNOWN_SETTING);
#undef TRY_SET
@ -77,12 +77,7 @@ String Settings::get(const String & name) const
if (false) {}
APPLY_FOR_SETTINGS(GET)
else
{
String value;
if (!limits.tryGet(name, value))
throw Exception("Unknown setting " + name, ErrorCodes::UNKNOWN_SETTING);
return value;
}
throw Exception("Unknown setting " + name, ErrorCodes::UNKNOWN_SETTING);
#undef GET
}
@ -95,7 +90,7 @@ bool Settings::tryGet(const String & name, String & value) const
if (false) {}
APPLY_FOR_SETTINGS(TRY_GET)
else
return limits.tryGet(name, value);
return false;
#undef TRY_GET
}
@ -140,7 +135,7 @@ void Settings::loadSettingsFromConfig(const String & path, const Poco::Util::Abs
/// If the `check_readonly` flag is set, `readonly` is set in the preferences, but some changes have occurred - throw an exception.
void Settings::deserialize(ReadBuffer & buf)
{
auto before_readonly = limits.readonly;
auto before_readonly = readonly;
while (true)
{
@ -171,8 +166,6 @@ void Settings::serialize(WriteBuffer & buf) const
APPLY_FOR_SETTINGS(WRITE)
limits.serialize(buf);
/// An empty string is a marker for the end of the settings.
writeStringBinary("", buf);

View File

@ -4,7 +4,6 @@
#include <Core/Defines.h>
#include <Core/Field.h>
#include <Interpreters/Limits.h>
#include <Interpreters/SettingsCommon.h>
@ -180,10 +179,74 @@ struct Settings
M(SettingBool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.") \
M(SettingBool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.") \
M(SettingUInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.") \
/// Possible limits for query execution.
Limits limits;
\
\
/** Limits during query execution are part of the settings. \
* Used to provide a more safe execution of queries from the user interface. \
* Basically, limits are checked for each block (not every row). That is, the limits can be slightly violated. \
* Almost all limits apply only to SELECTs. \
* Almost all limits apply to each stream individually. \
*/ \
\
M(SettingUInt64, max_rows_to_read, 0, "Limit on read rows from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.") \
M(SettingUInt64, max_bytes_to_read, 0, "Limit on read bytes (after decompression) from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.") \
M(SettingOverflowMode<false>, read_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
\
M(SettingUInt64, max_rows_to_group_by, 0, "") \
M(SettingOverflowMode<true>, group_by_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
M(SettingUInt64, max_bytes_before_external_group_by, 0, "") \
\
M(SettingUInt64, max_rows_to_sort, 0, "") \
M(SettingUInt64, max_bytes_to_sort, 0, "") \
M(SettingOverflowMode<false>, sort_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
M(SettingUInt64, max_bytes_before_external_sort, 0, "") \
\
M(SettingUInt64, max_result_rows, 0, "Limit on result size in rows. Also checked for intermediate data sent from remote servers.") \
M(SettingUInt64, max_result_bytes, 0, "Limit on result size in bytes (uncompressed). Also checked for intermediate data sent from remote servers.") \
M(SettingOverflowMode<false>, result_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
\
/* TODO: Check also when merging and finalizing aggregate functions. */ \
M(SettingSeconds, max_execution_time, 0, "") \
M(SettingOverflowMode<false>, timeout_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
\
M(SettingUInt64, min_execution_speed, 0, "In rows per second.") \
M(SettingSeconds, timeout_before_checking_execution_speed, 0, "Check that the speed is not too low after the specified time has elapsed.") \
\
M(SettingUInt64, max_columns_to_read, 0, "") \
M(SettingUInt64, max_temporary_columns, 0, "") \
M(SettingUInt64, max_temporary_non_const_columns, 0, "") \
\
M(SettingUInt64, max_subquery_depth, 100, "") \
M(SettingUInt64, max_pipeline_depth, 1000, "") \
M(SettingUInt64, max_ast_depth, 1000, "Maximum depth of query syntax tree. Checked after parsing.") \
M(SettingUInt64, max_ast_elements, 50000, "Maximum size of query syntax tree in number of nodes. Checked after parsing.") \
M(SettingUInt64, max_expanded_ast_elements, 500000, "Maximum size of query syntax tree in number of nodes after expansion of aliases and the asterisk.") \
\
M(SettingUInt64, readonly, 0, "0 - everything is allowed. 1 - only read requests. 2 - only read requests, as well as changing settings, except for the 'readonly' setting.") \
\
M(SettingUInt64, max_rows_in_set, 0, "Maximum size of the set (in number of elements) resulting from the execution of the IN section.") \
M(SettingUInt64, max_bytes_in_set, 0, "Maximum size of the set (in bytes in memory) resulting from the execution of the IN section.") \
M(SettingOverflowMode<false>, set_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
\
M(SettingUInt64, max_rows_in_join, 0, "Maximum size of the hash table for JOIN (in number of rows).") \
M(SettingUInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).") \
M(SettingOverflowMode<false>, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
\
M(SettingUInt64, max_rows_to_transfer, 0, "Maximum size (in rows) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.") \
M(SettingUInt64, max_bytes_to_transfer, 0, "Maximum size (in uncompressed bytes) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.") \
M(SettingOverflowMode<false>, transfer_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
\
M(SettingUInt64, max_rows_in_distinct, 0, "Maximum number of elements during execution of DISTINCT.") \
M(SettingUInt64, max_bytes_in_distinct, 0, "Maximum total size of state (in uncompressed bytes) in memory for the execution of DISTINCT.") \
M(SettingOverflowMode<false>, distinct_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \
\
M(SettingUInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.") \
M(SettingUInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.") \
M(SettingUInt64, max_memory_usage_for_all_queries, 0, "Maximum memory usage for processing all concurrently running queries on the server. Zero means unlimited.") \
\
M(SettingUInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second. Zero means unlimited.") \
M(SettingUInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.") \
M(SettingUInt64, max_network_bandwidth_for_user, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries for the user. Zero means unlimited.")
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
TYPE NAME {DEFAULT};

View File

@ -7,6 +7,8 @@
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/FieldVisitors.h>
#include <DataStreams/SizeLimits.h>
#include <IO/CompressedStream.h>
#include <IO/ReadHelpers.h>
@ -466,15 +468,6 @@ struct SettingTotalsMode
}
};
/// What to do if the limit is exceeded.
enum class OverflowMode
{
THROW = 0, /// Throw exception.
BREAK = 1, /// Abort query execution, return what is.
ANY = 2, /** Only for GROUP BY: do not add new rows to the set,
* but continue to aggregate for keys that are already in the set.
*/
};
template <bool enable_mode_any>
struct SettingOverflowMode

View File

@ -41,12 +41,12 @@ namespace ErrorCodes
}
static void checkLimits(const IAST & ast, const Limits & limits)
static void checkASTSizeLimits(const IAST & ast, const Settings & settings)
{
if (limits.max_ast_depth)
ast.checkDepth(limits.max_ast_depth);
if (limits.max_ast_elements)
ast.checkSize(limits.max_ast_elements);
if (settings.max_ast_depth)
ast.checkDepth(settings.max_ast_depth);
if (settings.max_ast_elements)
ast.checkSize(settings.max_ast_elements);
}
@ -190,7 +190,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
logQuery(query.substr(0, settings.log_queries_cut_to_length), context);
/// Check the limits.
checkLimits(*ast, settings.limits);
checkASTSizeLimits(*ast, settings);
QuotaForIntervals & quota = context.getQuota();
@ -233,9 +233,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
{
IProfilingBlockInputStream::LocalLimits limits;
limits.mode = IProfilingBlockInputStream::LIMITS_CURRENT;
limits.max_rows_to_read = settings.limits.max_result_rows;
limits.max_bytes_to_read = settings.limits.max_result_bytes;
limits.read_overflow_mode = settings.limits.result_overflow_mode;
limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode);
stream->setLimits(limits);
stream->setQuota(quota);

View File

@ -440,12 +440,9 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
("database", value<std::string>()->default_value("default"), "")
("stacktrace", "print stack traces of exceptions")
#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value<std::string> (), "Settings.h")
#define DECLARE_LIMIT(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value<std::string> (), "Limits.h")
#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value<std::string> (), DESCRIPTION)
APPLY_FOR_SETTINGS(DECLARE_SETTING)
APPLY_FOR_LIMITS(DECLARE_LIMIT)
#undef DECLARE_SETTING
#undef DECLARE_LIMIT
;
boost::program_options::variables_map options;
@ -467,7 +464,6 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
if (options.count(#NAME)) \
settings.set(#NAME, options[#NAME].as<std::string>());
APPLY_FOR_SETTINGS(EXTRACT_SETTING)
APPLY_FOR_LIMITS(EXTRACT_SETTING)
#undef EXTRACT_SETTING
Benchmark benchmark(

View File

@ -193,11 +193,6 @@ private:
APPLY_FOR_SETTINGS(EXTRACT_SETTING)
#undef EXTRACT_SETTING
#define EXTRACT_LIMIT(TYPE, NAME, DEFAULT, DESCRIPTION) \
if (config().has(#NAME) && !context.getSettingsRef().limits.NAME.changed) \
context.setSetting(#NAME, config().getString(#NAME));
APPLY_FOR_LIMITS(EXTRACT_LIMIT)
#undef EXTRACT_LIMIT
}
@ -1273,8 +1268,7 @@ public:
}
}
#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value<std::string> (), "Settings.h")
#define DECLARE_LIMIT(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value<std::string> (), "Limits.h")
#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value<std::string> (), DESCRIPTION)
/// Main commandline options related to client functionality and all parameters from Settings.
boost::program_options::options_description main_description("Main options");
@ -1302,10 +1296,8 @@ public:
("max_client_network_bandwidth", boost::program_options::value<int>(), "the maximum speed of data exchange over the network for the client in bytes per second.")
("compression", boost::program_options::value<bool>(), "enable or disable compression")
APPLY_FOR_SETTINGS(DECLARE_SETTING)
APPLY_FOR_LIMITS(DECLARE_LIMIT)
;
#undef DECLARE_SETTING
#undef DECLARE_LIMIT
/// Commandline options related to external tables.
boost::program_options::options_description external_description("External tables options");
@ -1369,7 +1361,6 @@ public:
if (options.count(#NAME)) \
context.setSetting(#NAME, options[#NAME].as<std::string>());
APPLY_FOR_SETTINGS(EXTRACT_SETTING)
APPLY_FOR_LIMITS(EXTRACT_SETTING)
#undef EXTRACT_SETTING
/// Save received data into the internal config.

View File

@ -682,7 +682,7 @@ void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration & c
/// Override important settings
settings_pull.load_balancing = LoadBalancing::NEAREST_HOSTNAME;
settings_pull.limits.readonly = 1;
settings_pull.readonly = 1;
settings_pull.max_threads = 1;
settings_pull.max_block_size = std::min(8192UL, settings_pull.max_block_size.value);
settings_pull.preferred_block_size_bytes = 0;

View File

@ -451,24 +451,22 @@ void HTTPHandler::processQuery(
/// In theory if initially readonly = 0, the client can change any setting and then set readonly
/// to some other value.
auto & limits = context.getSettingsRef().limits;
auto & settings = context.getSettingsRef();
/// Only readonly queries are allowed for HTTP GET requests.
if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET)
{
if (limits.readonly == 0)
limits.readonly = 2;
if (settings.readonly == 0)
settings.readonly = 2;
}
auto readonly_before_query = limits.readonly;
auto readonly_before_query = settings.readonly;
NameSet reserved_param_names{"query", "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace",
"buffer_size", "wait_end_of_query",
"session_id", "session_timeout", "session_check"
};
const Settings & settings = context.getSettingsRef();
for (auto it = params.begin(); it != params.end(); ++it)
{
if (it->first == "database")

View File

@ -161,19 +161,9 @@ void LocalServer::defineOptions(Poco::Util::OptionSet& _options)
#undef DECLARE_SETTING
nullptr};
static const char * limits_names[] = {
#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) #NAME,
APPLY_FOR_LIMITS(DECLARE_SETTING)
#undef DECLARE_SETTING
nullptr};
for (const char ** name = settings_names; *name; ++name)
_options.addOption(Poco::Util::Option(*name, "", "Settings.h").required(false).argument("<value>")
.repeatable(false).binding(*name));
for (const char ** name = limits_names; *name; ++name)
_options.addOption(Poco::Util::Option(*name, "", "Limits.h").required(false).argument("<value>")
.repeatable(false).binding(*name));
}
@ -187,12 +177,6 @@ void LocalServer::applyOptions()
context->setSetting(#NAME, config().getString(#NAME));
APPLY_FOR_SETTINGS(EXTRACT_SETTING)
#undef EXTRACT_SETTING
#define EXTRACT_LIMIT(TYPE, NAME, DEFAULT, DESCRIPTION) \
if (config().has(#NAME) && !context->getSettingsRef().limits.NAME.changed) \
context->setSetting(#NAME, config().getString(#NAME));
APPLY_FOR_LIMITS(EXTRACT_LIMIT)
#undef EXTRACT_LIMIT
}

View File

@ -848,7 +848,6 @@ private:
settings.set(#NAME, settings_to_apply[#NAME]);
APPLY_FOR_SETTINGS(EXTRACT_SETTING)
APPLY_FOR_LIMITS(EXTRACT_SETTING)
#undef EXTRACT_SETTING

View File

@ -276,9 +276,9 @@ void DistributedBlockOutputStream::writeSync(const Block & block)
pool.emplace(remote_jobs_count + local_jobs_count);
query_string = queryToString(query_ast);
if (!throttler && (settings.limits.max_network_bandwidth || settings.limits.max_network_bytes))
if (!throttler && (settings.max_network_bandwidth || settings.max_network_bytes))
{
throttler = std::make_shared<Throttler>(settings.limits.max_network_bandwidth, settings.limits.max_network_bytes,
throttler = std::make_shared<Throttler>(settings.max_network_bandwidth, settings.max_network_bytes,
"Network bandwidth limit for a query exceeded.");
}
}

View File

@ -648,7 +648,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMerger::mergePartsToTemporaryPart
}
if (deduplicate && merged_stream->isGroupedOutput())
merged_stream = std::make_shared<DistinctSortedBlockInputStream>(merged_stream, Limits(), 0 /*limit_hint*/, Names());
merged_stream = std::make_shared<DistinctSortedBlockInputStream>(merged_stream, SizeLimits(), 0 /*limit_hint*/, Names());
auto compression_settings = data.context.chooseCompressionSettings(
merge_entry->total_size_bytes_compressed,

View File

@ -508,7 +508,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
* They are done before the execution of the pipeline; they can not be interrupted; during the computation, packets of progress are not sent.
*/
if (!prewhere_subqueries.empty())
CreatingSetsBlockInputStream(std::make_shared<NullBlockInputStream>(Block()), prewhere_subqueries, settings.limits).read();
CreatingSetsBlockInputStream(std::make_shared<NullBlockInputStream>(Block()), prewhere_subqueries,
SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode)).read();
}
RangesInDataParts parts_with_ranges;

View File

@ -328,7 +328,7 @@ BlockOutputStreamPtr StorageBuffer::write(const ASTPtr & /*query*/, const Settin
void StorageBuffer::startup()
{
if (context.getSettingsRef().limits.readonly)
if (context.getSettingsRef().readonly)
{
LOG_WARNING(log, "Storage " << getName() << " is run with readonly settings, it will not be able to insert data."
<< " Set apropriate system_profile to fix this.");

View File

@ -1,11 +1,10 @@
#include <Storages/StorageJoin.h>
#include <Storages/StorageFactory.h>
#include <Interpreters/Join.h>
#include <Interpreters/Limits.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
#include <Poco/String.h>
#include <Poco/String.h> /// toLower
namespace DB
@ -49,7 +48,7 @@ StorageJoin::StorageJoin(
/// NOTE StorageJoin doesn't use join_use_nulls setting.
join = std::make_shared<Join>(key_names, key_names, false /* use_nulls */, Limits(), kind, strictness);
join = std::make_shared<Join>(key_names, key_names, false /* use_nulls */, SizeLimits(), kind, strictness);
join->setSampleBlock(getSampleBlock().sortColumns());
restore();
}

View File

@ -112,7 +112,7 @@ StorageSet::StorageSet(
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_)
: StorageSetOrJoinBase{path_, name_, columns_, materialized_columns_, alias_columns_, column_defaults_},
set(std::make_shared<Set>(Limits{}))
set(std::make_shared<Set>(SizeLimits()))
{
restore();
}

View File

@ -46,14 +46,6 @@ BlockInputStreams StorageSystemSettings::read(
APPLY_FOR_SETTINGS(ADD_SETTING)
#undef ADD_SETTING
#define ADD_LIMIT(TYPE, NAME, DEFAULT, DESCRIPTION) \
res_columns[0]->insert(String(#NAME)); \
res_columns[1]->insert(settings.limits.NAME.toString()); \
res_columns[2]->insert(UInt64(settings.limits.NAME.changed)); \
res_columns[3]->insert(String(DESCRIPTION));
APPLY_FOR_LIMITS(ADD_LIMIT)
#undef ADD_LIMIT
return BlockInputStreams(1, std::make_shared<OneBlockInputStream>(getSampleBlock().cloneWithColumns(std::move(res_columns))));
}

View File

@ -27,7 +27,7 @@ TableFunctionPtr TableFunctionFactory::get(
const std::string & name,
const Context & context) const
{
if (context.getSettings().limits.readonly == 1) /** For example, for readonly = 2 - allowed. */
if (context.getSettings().readonly == 1) /** For example, for readonly = 2 - allowed. */
throw Exception("Table functions are forbidden in readonly mode", ErrorCodes::READONLY);
auto it = functions.find(name);