diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp index 0a58050339a..bca32fb342b 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp @@ -104,7 +104,7 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) bool done_with_table = !subquery.table; if (done_with_set && done_with_join && done_with_table) - throw Exception("Logical error: nothing to do with subquery", ErrorCodes::LOGICAL_ERROR); + return; if (table_out) table_out->writePrefix(); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 592ec7f89b2..ebd83f0e01a 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -168,13 +168,13 @@ ExpressionAnalyzer::ExpressionAnalyzer( const NamesAndTypesList & source_columns_, const Names & required_result_columns_, size_t subquery_depth_, - bool do_global_, + bool dry_run_, const SubqueriesForSets & subqueries_for_set_) : ast(ast_), context(context_), settings(context.getSettings()), subquery_depth(subquery_depth_), source_columns(source_columns_), required_result_columns(required_result_columns_.begin(), required_result_columns_.end()), storage(storage_), - do_global(do_global_), subqueries_for_sets(subqueries_for_set_) + dry_run(dry_run_), subqueries_for_sets(subqueries_for_set_) { select_query = typeid_cast(ast.get()); @@ -613,7 +613,8 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables() findExternalTables(ast); /// Converts GLOBAL subqueries to external tables; Puts them into the external_tables dictionary: name -> StoragePtr. - initGlobalSubqueries(ast); + if (!dry_run) + initGlobalSubqueries(ast); } @@ -630,14 +631,13 @@ void ExpressionAnalyzer::initGlobalSubqueries(ASTPtr & ast) if (ASTFunction * node = typeid_cast(ast.get())) { /// For GLOBAL IN. - if (do_global && (node->name == "globalIn" || node->name == "globalNotIn")) + if (node->name == "globalIn" || node->name == "globalNotIn") addExternalStorage(node->arguments->children.at(1)); } else if (ASTTablesInSelectQueryElement * node = typeid_cast(ast.get())) { /// For GLOBAL JOIN. - if (do_global && node->table_join - && static_cast(*node->table_join).locality == ASTTableJoin::Locality::Global) + if (node->table_join && static_cast(*node->table_join).locality == ASTTableJoin::Locality::Global) addExternalStorage(node->table_expression); } } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 9afacc675d7..7cf90b31c1c 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -103,7 +103,7 @@ public: const NamesAndTypesList & source_columns_ = {}, const Names & required_result_columns_ = {}, size_t subquery_depth_ = 0, - bool do_global_ = false, + bool dry_run_ = false, const SubqueriesForSets & subqueries_for_set_ = {}); /// Does the expression have aggregate functions or a GROUP BY or HAVING section. @@ -196,8 +196,8 @@ private: NamesAndTypesList aggregation_keys; AggregateDescriptions aggregate_descriptions; - /// Do I need to prepare for execution global subqueries when analyzing the query. - bool do_global; + /// Don't prepare global subqueries and don't fill sets for index. + bool dry_run; SubqueriesForSets subqueries_for_sets; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 26c96a6a98a..7aee172943e 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -145,7 +145,7 @@ void InterpreterSelectQuery::init(const Names & required_result_column_names) table_lock = storage->lockStructure(false, __PRETTY_FUNCTION__); query_analyzer = std::make_unique( - query_ptr, context, storage, source_columns, required_result_column_names, subquery_depth, !only_analyze); + query_ptr, context, storage, source_columns, required_result_column_names, subquery_depth, only_analyze); if (!only_analyze) { diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index b8240d957a7..3d61538f68e 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -187,8 +187,6 @@ struct Settings M(SettingSeconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout") \ M(SettingBool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown") \ M(SettingBool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.") \ - M(SettingUInt64, use_index_for_in_with_subqueries_max_values, 100000, "Don't use index of a table for filtering by right hand size of the IN operator if the size of set is larger than specified threshold. This allows to avoid performance degradation and higher memory usage due to preparation of additional data structures.") \ - \ 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.") \