diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index df96b5704ed..6c07a5e3422 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -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; diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp index 83c3db48509..b183226e4c9 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp @@ -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) diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.h b/dbms/src/DataStreams/CreatingSetsBlockInputStream.h index d3cf53c034b..dc34866a60c 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.h +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.h @@ -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; diff --git a/dbms/src/DataStreams/DistinctBlockInputStream.cpp b/dbms/src/DataStreams/DistinctBlockInputStream.cpp index 50949b2f0cc..848fe923db4 100644 --- a/dbms/src/DataStreams/DistinctBlockInputStream.cpp +++ b/dbms/src/DataStreams/DistinctBlockInputStream.cpp @@ -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 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(); diff --git a/dbms/src/DataStreams/DistinctBlockInputStream.h b/dbms/src/DataStreams/DistinctBlockInputStream.h index 60a92e0399a..3ab915cb964 100644 --- a/dbms/src/DataStreams/DistinctBlockInputStream.h +++ b/dbms/src/DataStreams/DistinctBlockInputStream.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include 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 @@ -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; }; } diff --git a/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp b/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp index e9e5d69d047..a14d32794f3 100644 --- a/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp @@ -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 bool DistinctSortedBlockInputStream::buildFilter( diff --git a/dbms/src/DataStreams/DistinctSortedBlockInputStream.h b/dbms/src/DataStreams/DistinctSortedBlockInputStream.h index 33c6cef7774..5673e1376b0 100644 --- a/dbms/src/DataStreams/DistinctSortedBlockInputStream.h +++ b/dbms/src/DataStreams/DistinctSortedBlockInputStream.h @@ -1,9 +1,9 @@ #pragma once #include -#include #include + 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; }; } diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp index 1f9354a34e1..306afb5955b 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp @@ -1,6 +1,3 @@ -#include -#include - #include #include #include @@ -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); } diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.h b/dbms/src/DataStreams/IProfilingBlockInputStream.h index 3bff72da8ec..442c451faa3 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.h +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.h @@ -2,13 +2,15 @@ #include -#include - #include #include +#include + +#include #include + 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); diff --git a/dbms/src/DataStreams/SizeLimits.cpp b/dbms/src/DataStreams/SizeLimits.cpp new file mode 100644 index 00000000000..63164552120 --- /dev/null +++ b/dbms/src/DataStreams/SizeLimits.cpp @@ -0,0 +1,33 @@ +#include +#include +#include +#include + + +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; +} + +} diff --git a/dbms/src/DataStreams/SizeLimits.h b/dbms/src/DataStreams/SizeLimits.h new file mode 100644 index 00000000000..66373a179ab --- /dev/null +++ b/dbms/src/DataStreams/SizeLimits.h @@ -0,0 +1,37 @@ +#pragma once + +#include + + +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; +}; + +} diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index a498acf4c74..5919a296085 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -15,10 +15,10 @@ #include #include +#include #include #include -#include #include #include diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index 89acf7417f4..4d304aff64b 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -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( @@ -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::vectorgetUserNetworkThrottler(); /// 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( - 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()) diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 57084136cbd..6fddcbc2f05 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -936,12 +936,6 @@ Settings Context::getSettings() const } -Limits Context::getLimits() const -{ - return settings.limits; -} - - void Context::setSettings(const Settings & settings_) { settings = settings_; diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index ea32e38d85c..bab3be07add 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -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); diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 455c6bf2c17..1c60b6d766c 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -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."); diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 19f56e3c387..14fdb99090c 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -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); } } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 8f2915e0d9b..c3c663d59ad 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -687,8 +687,8 @@ static std::shared_ptr 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(settings.limits); + SetPtr set = std::make_shared(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(settings.limits); + SetPtr set = std::make_shared(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(settings.limits); + SetPtr set = std::make_shared(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_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()); diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 700f07929bb..2b6d333d1f6 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -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) { diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index 1f3b6c59bc0..d99d9052d6b 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -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)) diff --git a/dbms/src/Interpreters/InterpreterFactory.cpp b/dbms/src/Interpreters/InterpreterFactory.cpp index 4361166cb91..9d30c586129 100644 --- a/dbms/src/Interpreters/InterpreterFactory.cpp +++ b/dbms/src/Interpreters/InterpreterFactory.cpp @@ -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); } diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 9a448c58c3a..b9ac74554d1 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -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)) { diff --git a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp index 673e78e1e2e..1710f881fe4 100644 --- a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -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); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index ad7e2cdcf1a..2d645dc7dd5 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -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( 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(stream, settings.limits, limit_for_distinct, columns); + stream = std::make_shared(stream, limits, limit_for_distinct, columns); else - stream = std::make_shared(stream, settings.limits, limit_for_distinct, columns); + stream = std::make_shared(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(pipeline.firstStream(), subqueries_for_sets, settings.limits); + pipeline.firstStream() = std::make_shared( + pipeline.firstStream(), subqueries_for_sets, + SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode)); } diff --git a/dbms/src/Interpreters/InterpreterSetQuery.cpp b/dbms/src/Interpreters/InterpreterSetQuery.cpp index 29894bf8f3e..00620ffb4cc 100644 --- a/dbms/src/Interpreters/InterpreterSetQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSetQuery.cpp @@ -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) { diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 18e1e056a03..eecb108e1a5 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -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); } diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 20df3829af9..ee7e3b1e0ad 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -13,6 +13,7 @@ #include #include +#include #include @@ -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; diff --git a/dbms/src/Interpreters/Limits.h b/dbms/src/Interpreters/Limits.h deleted file mode 100644 index e04d1ec7795..00000000000 --- a/dbms/src/Interpreters/Limits.h +++ /dev/null @@ -1,185 +0,0 @@ -#pragma once - -#include -#include -#include - - -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, read_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \ - \ - M(SettingUInt64, max_rows_to_group_by, 0, "") \ - M(SettingOverflowMode, 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, 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, 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, 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, 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, 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, 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, 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 - } -}; - - -} diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index e3009d59749..208373cddba 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -75,7 +75,7 @@ ProcessList::EntryPtr ProcessList::insert( res = std::make_shared(*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(settings.limits.max_network_bandwidth_for_user, 0, + user_process_list.user_throttler = std::make_shared(settings.max_network_bandwidth_for_user, 0, "Network bandwidth limit for a user exceeded."); } diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index 63b7024cb73..dd944cd1cb4 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -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 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) diff --git a/dbms/src/Interpreters/Set.h b/dbms/src/Interpreters/Set.h index bd2a7363ad4..1c64bacab00 100644 --- a/dbms/src/Interpreters/Set.h +++ b/dbms/src/Interpreters/Set.h @@ -1,10 +1,12 @@ #pragma once #include +#include #include -#include -#include +#include +#include #include +#include #include #include @@ -28,11 +30,9 @@ using FunctionBasePtr = std::shared_ptr; 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()) { } @@ -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 * 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; diff --git a/dbms/src/Interpreters/Settings.cpp b/dbms/src/Interpreters/Settings.cpp index c501120413f..e6a8544be1b 100644 --- a/dbms/src/Interpreters/Settings.cpp +++ b/dbms/src/Interpreters/Settings.cpp @@ -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); diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 19e81766564..79d76850088 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -4,7 +4,6 @@ #include #include -#include #include @@ -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, read_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \ + \ + M(SettingUInt64, max_rows_to_group_by, 0, "") \ + M(SettingOverflowMode, 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, 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, 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, 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, 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, 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, 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, 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}; diff --git a/dbms/src/Interpreters/SettingsCommon.h b/dbms/src/Interpreters/SettingsCommon.h index 68838b43b07..c9a0632bdd2 100644 --- a/dbms/src/Interpreters/SettingsCommon.h +++ b/dbms/src/Interpreters/SettingsCommon.h @@ -7,6 +7,8 @@ #include #include +#include + #include #include @@ -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 struct SettingOverflowMode diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 5d5cb1bc10a..963727fc481 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -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 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 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); diff --git a/dbms/src/Server/Benchmark.cpp b/dbms/src/Server/Benchmark.cpp index 989ab07aa0f..2a75e187f23 100644 --- a/dbms/src/Server/Benchmark.cpp +++ b/dbms/src/Server/Benchmark.cpp @@ -440,12 +440,9 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ("database", value()->default_value("default"), "") ("stacktrace", "print stack traces of exceptions") - #define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value (), "Settings.h") - #define DECLARE_LIMIT(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value (), "Limits.h") + #define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value (), 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()); APPLY_FOR_SETTINGS(EXTRACT_SETTING) - APPLY_FOR_LIMITS(EXTRACT_SETTING) #undef EXTRACT_SETTING Benchmark benchmark( diff --git a/dbms/src/Server/Client.cpp b/dbms/src/Server/Client.cpp index 78820fc142a..661d91e68d7 100644 --- a/dbms/src/Server/Client.cpp +++ b/dbms/src/Server/Client.cpp @@ -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 (), "Settings.h") -#define DECLARE_LIMIT(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value (), "Limits.h") +#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value (), 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(), "the maximum speed of data exchange over the network for the client in bytes per second.") ("compression", boost::program_options::value(), "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()); APPLY_FOR_SETTINGS(EXTRACT_SETTING) - APPLY_FOR_LIMITS(EXTRACT_SETTING) #undef EXTRACT_SETTING /// Save received data into the internal config. diff --git a/dbms/src/Server/ClusterCopier.cpp b/dbms/src/Server/ClusterCopier.cpp index ec89ff2c99c..78eaf7b02d4 100644 --- a/dbms/src/Server/ClusterCopier.cpp +++ b/dbms/src/Server/ClusterCopier.cpp @@ -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; diff --git a/dbms/src/Server/HTTPHandler.cpp b/dbms/src/Server/HTTPHandler.cpp index 555d4b5f3ba..e06a3f0b885 100644 --- a/dbms/src/Server/HTTPHandler.cpp +++ b/dbms/src/Server/HTTPHandler.cpp @@ -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") diff --git a/dbms/src/Server/LocalServer.cpp b/dbms/src/Server/LocalServer.cpp index 8ebfdb22e12..aec3f4152f1 100644 --- a/dbms/src/Server/LocalServer.cpp +++ b/dbms/src/Server/LocalServer.cpp @@ -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("") .repeatable(false).binding(*name)); - - for (const char ** name = limits_names; *name; ++name) - _options.addOption(Poco::Util::Option(*name, "", "Limits.h").required(false).argument("") - .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 } diff --git a/dbms/src/Server/PerformanceTest.cpp b/dbms/src/Server/PerformanceTest.cpp index 82518df2fe3..fce45775b35 100644 --- a/dbms/src/Server/PerformanceTest.cpp +++ b/dbms/src/Server/PerformanceTest.cpp @@ -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 diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 1155d4c6362..daba8ea0dad 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -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(settings.limits.max_network_bandwidth, settings.limits.max_network_bytes, + throttler = std::make_shared(settings.max_network_bandwidth, settings.max_network_bytes, "Network bandwidth limit for a query exceeded."); } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp index bc56bd61fe4..cf9e6eef15c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp @@ -648,7 +648,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMerger::mergePartsToTemporaryPart } if (deduplicate && merged_stream->isGroupedOutput()) - merged_stream = std::make_shared(merged_stream, Limits(), 0 /*limit_hint*/, Names()); + merged_stream = std::make_shared(merged_stream, SizeLimits(), 0 /*limit_hint*/, Names()); auto compression_settings = data.context.chooseCompressionSettings( merge_entry->total_size_bytes_compressed, diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index b6d19da1d89..fb7c1735ced 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -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(Block()), prewhere_subqueries, settings.limits).read(); + CreatingSetsBlockInputStream(std::make_shared(Block()), prewhere_subqueries, + SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode)).read(); } RangesInDataParts parts_with_ranges; diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index f2a4128e514..6b9ec92ce75 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -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."); diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index de75e61f563..846c18d7055 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -1,11 +1,10 @@ #include #include #include -#include #include #include -#include +#include /// toLower namespace DB @@ -49,7 +48,7 @@ StorageJoin::StorageJoin( /// NOTE StorageJoin doesn't use join_use_nulls setting. - join = std::make_shared(key_names, key_names, false /* use_nulls */, Limits(), kind, strictness); + join = std::make_shared(key_names, key_names, false /* use_nulls */, SizeLimits(), kind, strictness); join->setSampleBlock(getSampleBlock().sortColumns()); restore(); } diff --git a/dbms/src/Storages/StorageSet.cpp b/dbms/src/Storages/StorageSet.cpp index bdbfca46d57..1303aa6271f 100644 --- a/dbms/src/Storages/StorageSet.cpp +++ b/dbms/src/Storages/StorageSet.cpp @@ -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(Limits{})) + set(std::make_shared(SizeLimits())) { restore(); } diff --git a/dbms/src/Storages/System/StorageSystemSettings.cpp b/dbms/src/Storages/System/StorageSystemSettings.cpp index 58c3eb504b9..6099883eeb6 100644 --- a/dbms/src/Storages/System/StorageSystemSettings.cpp +++ b/dbms/src/Storages/System/StorageSystemSettings.cpp @@ -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(getSampleBlock().cloneWithColumns(std::move(res_columns)))); } diff --git a/dbms/src/TableFunctions/TableFunctionFactory.cpp b/dbms/src/TableFunctions/TableFunctionFactory.cpp index a3355fd32e9..b6188ee5967 100644 --- a/dbms/src/TableFunctions/TableFunctionFactory.cpp +++ b/dbms/src/TableFunctions/TableFunctionFactory.cpp @@ -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);