();
}
@@ -1339,7 +1347,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optionalinput_order_info && query_info.input_order_info)
- throw Exception("InputOrderInfo is set for projection and for query", ErrorCodes::LOGICAL_ERROR);
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "InputOrderInfo is set for projection and for query");
InputOrderInfoPtr input_order_info_for_order;
if (!expressions.need_aggregate)
input_order_info_for_order = query_info.projection ? query_info.projection->input_order_info : query_info.input_order_info;
@@ -1382,7 +1390,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional
hasWindow())
- throw Exception(
- "Window functions does not support processing from WithMergeableStateAfterAggregation",
- ErrorCodes::NOT_IMPLEMENTED);
+ throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Window functions does not support processing from WithMergeableStateAfterAggregation");
}
else if (expressions.need_aggregate)
{
@@ -2242,7 +2248,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
}
if (!max_block_size)
- throw Exception("Setting 'max_block_size' cannot be zero", ErrorCodes::PARAMETER_OUT_OF_BOUND);
+ throw Exception(ErrorCodes::PARAMETER_OUT_OF_BOUND, "Setting 'max_block_size' cannot be zero");
storage_limits.emplace_back(local_limits);
@@ -2256,7 +2262,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
/// Subquery.
ASTPtr subquery = extractTableExpression(query, 0);
if (!subquery)
- throw Exception("Subquery expected", ErrorCodes::LOGICAL_ERROR);
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Subquery expected");
interpreter_subquery = std::make_unique(
subquery, getSubqueryContext(context),
@@ -2367,7 +2373,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc
}
}
else
- throw Exception("Logical error in InterpreterSelectQuery: nowhere to read", ErrorCodes::LOGICAL_ERROR);
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error in InterpreterSelectQuery: nowhere to read");
/// Specify the number of threads only if it wasn't specified in storage.
///
@@ -2945,7 +2951,7 @@ void InterpreterSelectQuery::executeLimit(QueryPlan & query_plan)
if (query.limit_with_ties)
{
if (!query.orderBy())
- throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR);
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "LIMIT WITH TIES without ORDER BY");
order_descr = getSortDescription(query, context);
}
diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp
index bf384fa5d86..e3954f2a197 100644
--- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp
+++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp
@@ -54,7 +54,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(
size_t num_children = ast->list_of_selects->children.size();
if (!num_children)
- throw Exception("Logical error: no children in ASTSelectWithUnionQuery", ErrorCodes::LOGICAL_ERROR);
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no children in ASTSelectWithUnionQuery");
/// Note that we pass 'required_result_column_names' to first SELECT.
/// And for the rest, we pass names at the corresponding positions of 'required_result_column_names' in the result of first SELECT,
@@ -81,11 +81,9 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(
= getCurrentChildResultHeader(ast->list_of_selects->children.at(query_num), required_result_column_names);
if (full_result_header_for_current_select.columns() != full_result_header.columns())
- throw Exception("Different number of columns in UNION ALL elements:\n"
- + full_result_header.dumpNames()
- + "\nand\n"
- + full_result_header_for_current_select.dumpNames() + "\n",
- ErrorCodes::UNION_ALL_RESULT_STRUCTURES_MISMATCH);
+ throw Exception(ErrorCodes::UNION_ALL_RESULT_STRUCTURES_MISMATCH,
+ "Different number of columns in UNION ALL elements:\n{}\nand\n{}\n",
+ full_result_header.dumpNames(), full_result_header_for_current_select.dumpNames());
required_result_column_names_for_other_selects[query_num].reserve(required_result_column_names.size());
for (const auto & pos : positions_of_required_result_columns)
@@ -213,11 +211,9 @@ Block InterpreterSelectWithUnionQuery::getCommonHeaderForUnion(const Blocks & he
for (size_t query_num = 1; query_num < num_selects; ++query_num)
{
if (headers[query_num].columns() != num_columns)
- throw Exception("Different number of columns in UNION ALL elements:\n"
- + common_header.dumpNames()
- + "\nand\n"
- + headers[query_num].dumpNames() + "\n",
- ErrorCodes::UNION_ALL_RESULT_STRUCTURES_MISMATCH);
+ throw Exception(ErrorCodes::UNION_ALL_RESULT_STRUCTURES_MISMATCH,
+ "Different number of columns in UNION ALL elements:\n{}\nand\n{}\n",
+ common_header.dumpNames(), headers[query_num].dumpNames());
}
std::vector columns(num_selects);
diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp
index eff31b168bd..5e1b74681fe 100644
--- a/src/Interpreters/InterpreterShowCreateQuery.cpp
+++ b/src/Interpreters/InterpreterShowCreateQuery.cpp
@@ -76,14 +76,16 @@ QueryPipeline InterpreterShowCreateQuery::executeImpl()
else if ((show_query = query_ptr->as()))
{
if (show_query->temporary)
- throw Exception("Temporary databases are not possible.", ErrorCodes::SYNTAX_ERROR);
+ throw Exception(ErrorCodes::SYNTAX_ERROR, "Temporary databases are not possible.");
show_query->setDatabase(getContext()->resolveDatabase(show_query->getDatabase()));
getContext()->checkAccess(AccessType::SHOW_DATABASES, show_query->getDatabase());
create_query = DatabaseCatalog::instance().getDatabase(show_query->getDatabase())->getCreateDatabaseQuery();
}
if (!create_query)
- throw Exception("Unable to show the create query of " + show_query->getTable() + ". Maybe it was created by the system.", ErrorCodes::THERE_IS_NO_QUERY);
+ throw Exception(ErrorCodes::THERE_IS_NO_QUERY,
+ "Unable to show the create query of {}. Maybe it was created by the system.",
+ show_query->getTable());
if (!getContext()->getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil)
{
diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp
index a6cea66df84..4e0dfdc9236 100644
--- a/src/Interpreters/InterpreterShowTablesQuery.cpp
+++ b/src/Interpreters/InterpreterShowTablesQuery.cpp
@@ -105,7 +105,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
}
if (query.temporary && !query.from.empty())
- throw Exception("The `FROM` and `TEMPORARY` cannot be used together in `SHOW TABLES`", ErrorCodes::SYNTAX_ERROR);
+ throw Exception(ErrorCodes::SYNTAX_ERROR, "The `FROM` and `TEMPORARY` cannot be used together in `SHOW TABLES`");
String database = getContext()->resolveDatabase(query.from);
DatabaseCatalog::instance().assertDatabaseExists(database);
@@ -131,7 +131,7 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
if (query.temporary)
{
if (query.dictionaries)
- throw Exception("Temporary dictionaries are not possible.", ErrorCodes::SYNTAX_ERROR);
+ throw Exception(ErrorCodes::SYNTAX_ERROR, "Temporary dictionaries are not possible.");
rewritten_query << "is_temporary";
}
else
diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp
index a82a11e7c97..1d0b299295f 100644
--- a/src/Interpreters/InterpreterSystemQuery.cpp
+++ b/src/Interpreters/InterpreterSystemQuery.cpp
@@ -142,7 +142,7 @@ AccessType getRequiredAccessType(StorageActionBlockType action_type)
else if (action_type == ActionLocks::PartsMove)
return AccessType::SYSTEM_MOVES;
else
- throw Exception("Unknown action type: " + std::to_string(action_type), ErrorCodes::LOGICAL_ERROR);
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown action type: {}", std::to_string(action_type));
}
constexpr std::string_view table_is_not_replicated = "Table {} is not replicated";
@@ -320,6 +320,10 @@ BlockIO InterpreterSystemQuery::execute()
getContext()->checkAccess(AccessType::SYSTEM_DROP_MMAP_CACHE);
system_context->dropMMappedFileCache();
break;
+ case Type::DROP_QUERY_RESULT_CACHE:
+ getContext()->checkAccess(AccessType::SYSTEM_DROP_QUERY_RESULT_CACHE);
+ getContext()->dropQueryResultCache();
+ break;
#if USE_EMBEDDED_COMPILER
case Type::DROP_COMPILED_EXPRESSION_CACHE:
getContext()->checkAccess(AccessType::SYSTEM_DROP_COMPILED_EXPRESSION_CACHE);
@@ -435,7 +439,7 @@ BlockIO InterpreterSystemQuery::execute()
SymbolIndex::reload();
break;
#else
- throw Exception("SYSTEM RELOAD SYMBOLS is not supported on current platform", ErrorCodes::NOT_IMPLEMENTED);
+ throw Exception(ErrorCodes::NOT_IMPLEMENTED, "SYSTEM RELOAD SYMBOLS is not supported on current platform");
#endif
}
case Type::STOP_MERGES:
@@ -553,7 +557,7 @@ BlockIO InterpreterSystemQuery::execute()
break;
}
default:
- throw Exception("Unknown type of SYSTEM query", ErrorCodes::BAD_ARGUMENTS);
+ throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown type of SYSTEM query");
}
return result;
@@ -676,7 +680,7 @@ void InterpreterSystemQuery::restartReplicas(ContextMutablePtr system_context)
void InterpreterSystemQuery::dropReplica(ASTSystemQuery & query)
{
if (query.replica.empty())
- throw Exception("Replica name is empty", ErrorCodes::BAD_ARGUMENTS);
+ throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica name is empty");
if (!table_id.empty())
{
@@ -731,10 +735,13 @@ void InterpreterSystemQuery::dropReplica(ASTSystemQuery & query)
ReplicatedTableStatus status;
storage_replicated->getStatus(status);
if (status.zookeeper_path == query.replica_zk_path)
- throw Exception("There is a local table " + storage_replicated->getStorageID().getNameForLogs() +
- ", which has the same table path in ZooKeeper. Please check the path in query. "
- "If you want to drop replica of this table, use `DROP TABLE` "
- "or `SYSTEM DROP REPLICA 'name' FROM db.table`", ErrorCodes::TABLE_WAS_NOT_DROPPED);
+ throw Exception(ErrorCodes::TABLE_WAS_NOT_DROPPED,
+ "There is a local table {}, which has the same table path in ZooKeeper. "
+ "Please check the path in query. "
+ "If you want to drop replica "
+ "of this table, use `DROP TABLE` "
+ "or `SYSTEM DROP REPLICA 'name' FROM db.table`",
+ storage_replicated->getStorageID().getNameForLogs());
}
}
}
@@ -744,18 +751,17 @@ void InterpreterSystemQuery::dropReplica(ASTSystemQuery & query)
bool looks_like_table_path = zookeeper->exists(query.replica_zk_path + "/replicas") ||
zookeeper->exists(query.replica_zk_path + "/dropped");
if (!looks_like_table_path)
- throw Exception("Specified path " + query.replica_zk_path + " does not look like a table path",
- ErrorCodes::TABLE_WAS_NOT_DROPPED);
+ throw Exception(ErrorCodes::TABLE_WAS_NOT_DROPPED, "Specified path {} does not look like a table path",
+ query.replica_zk_path);
if (zookeeper->exists(remote_replica_path + "/is_active"))
- throw Exception("Can't remove replica: " + query.replica + ", because it's active",
- ErrorCodes::TABLE_WAS_NOT_DROPPED);
+ throw Exception(ErrorCodes::TABLE_WAS_NOT_DROPPED, "Can't remove replica: {}, because it's active", query.replica);
StorageReplicatedMergeTree::dropReplica(zookeeper, query.replica_zk_path, query.replica, log);
LOG_INFO(log, "Dropped replica {}", remote_replica_path);
}
else
- throw Exception("Invalid query", ErrorCodes::LOGICAL_ERROR);
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid query");
}
bool InterpreterSystemQuery::dropReplicaImpl(ASTSystemQuery & query, const StoragePtr & table)
@@ -770,15 +776,15 @@ bool InterpreterSystemQuery::dropReplicaImpl(ASTSystemQuery & query, const Stora
/// Do not allow to drop local replicas and active remote replicas
if (query.replica == status.replica_name)
- throw Exception("We can't drop local replica, please use `DROP TABLE` "
- "if you want to clean the data and drop this replica", ErrorCodes::TABLE_WAS_NOT_DROPPED);
+ throw Exception(ErrorCodes::TABLE_WAS_NOT_DROPPED,
+ "We can't drop local replica, please use `DROP TABLE` if you want "
+ "to clean the data and drop this replica");
/// NOTE it's not atomic: replica may become active after this check, but before dropReplica(...)
/// However, the main use case is to drop dead replica, which cannot become active.
/// This check prevents only from accidental drop of some other replica.
if (zookeeper->exists(status.zookeeper_path + "/replicas/" + query.replica + "/is_active"))
- throw Exception("Can't drop replica: " + query.replica + ", because it's active",
- ErrorCodes::TABLE_WAS_NOT_DROPPED);
+ throw Exception(ErrorCodes::TABLE_WAS_NOT_DROPPED, "Can't drop replica: {}, because it's active", query.replica);
storage_replicated->dropReplica(zookeeper, status.zookeeper_path, query.replica, log);
LOG_TRACE(log, "Dropped replica {} of {}", query.replica, table->getStorageID().getNameForLogs());
@@ -789,7 +795,7 @@ bool InterpreterSystemQuery::dropReplicaImpl(ASTSystemQuery & query, const Stora
void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query)
{
if (query.replica.empty())
- throw Exception("Replica name is empty", ErrorCodes::BAD_ARGUMENTS);
+ throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica name is empty");
auto check_not_local_replica = [](const DatabaseReplicated * replicated, const ASTSystemQuery & query)
{
@@ -852,7 +858,7 @@ void InterpreterSystemQuery::dropDatabaseReplica(ASTSystemQuery & query)
LOG_INFO(log, "Dropped replica {} of Replicated database with path {}", query.replica, query.replica_zk_path);
}
else
- throw Exception("Invalid query", ErrorCodes::LOGICAL_ERROR);
+ throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid query");
}
void InterpreterSystemQuery::syncReplica()
@@ -928,13 +934,13 @@ void InterpreterSystemQuery::flushDistributed(ASTSystemQuery &)
if (auto * storage_distributed = dynamic_cast(DatabaseCatalog::instance().getTable(table_id, getContext()).get()))
storage_distributed->flushClusterNodesAllData(getContext());
else
- throw Exception("Table " + table_id.getNameForLogs() + " is not distributed", ErrorCodes::BAD_ARGUMENTS);
+ throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table {} is not distributed", table_id.getNameForLogs());
}
[[noreturn]] void InterpreterSystemQuery::restartDisk(String &)
{
getContext()->checkAccess(AccessType::SYSTEM_RESTART_DISK);
- throw Exception("SYSTEM RESTART DISK is not supported", ErrorCodes::NOT_IMPLEMENTED);
+ throw Exception(ErrorCodes::NOT_IMPLEMENTED, "SYSTEM RESTART DISK is not supported");
}
@@ -956,6 +962,7 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster()
case Type::DROP_DNS_CACHE:
case Type::DROP_MARK_CACHE:
case Type::DROP_MMAP_CACHE:
+ case Type::DROP_QUERY_RESULT_CACHE:
#if USE_EMBEDDED_COMPILER
case Type::DROP_COMPILED_EXPRESSION_CACHE:
#endif
diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp
index 9d153c2a9d2..b2086831e4e 100644
--- a/src/Interpreters/InterpreterWatchQuery.cpp
+++ b/src/Interpreters/InterpreterWatchQuery.cpp
@@ -66,12 +66,12 @@ QueryPipelineBuilder InterpreterWatchQuery::buildQueryPipeline()
auto storage_name = storage->getName();
if (storage_name == "LiveView"
&& !getContext()->getSettingsRef().allow_experimental_live_view)
- throw Exception("Experimental LIVE VIEW feature is not enabled (the setting 'allow_experimental_live_view')",
- ErrorCodes::SUPPORT_IS_DISABLED);
+ throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
+ "Experimental LIVE VIEW feature is not enabled (the setting 'allow_experimental_live_view')");
else if (storage_name == "WindowView"
&& !getContext()->getSettingsRef().allow_experimental_window_view)
- throw Exception("Experimental WINDOW VIEW feature is not enabled (the setting 'allow_experimental_window_view')",
- ErrorCodes::SUPPORT_IS_DISABLED);
+ throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
+ "Experimental WINDOW VIEW feature is not enabled (the setting 'allow_experimental_window_view')");
/// List of columns to read to execute the query.
Names required_columns = storage->getInMemoryMetadataPtr()->getColumns().getNamesOfPhysical();
@@ -82,10 +82,8 @@ QueryPipelineBuilder InterpreterWatchQuery::buildQueryPipeline()
/// Limitation on the number of columns to read.
if (settings.max_columns_to_read && required_columns.size() > settings.max_columns_to_read)
- throw Exception("Limit for number of columns to read exceeded. "
- "Requested: " + std::to_string(required_columns.size())
- + ", maximum: " + settings.max_columns_to_read.toString(),
- ErrorCodes::TOO_MANY_COLUMNS);
+ throw Exception(ErrorCodes::TOO_MANY_COLUMNS, "Limit for number of columns to read exceeded. "
+ "Requested: {}, maximum: {}", required_columns.size(), settings.max_columns_to_read.toString());
size_t max_block_size = settings.max_block_size;
size_t max_streams = 1;
diff --git a/src/Interpreters/InterserverCredentials.cpp b/src/Interpreters/InterserverCredentials.cpp
index 6e36b06f9cc..094b58789a8 100644
--- a/src/Interpreters/InterserverCredentials.cpp
+++ b/src/Interpreters/InterserverCredentials.cpp
@@ -14,10 +14,11 @@ std::unique_ptr
InterserverCredentials::make(const Poco::Util::AbstractConfiguration & config, const std::string & root_tag)
{
if (config.has("user") && !config.has("password"))
- throw Exception("Configuration parameter interserver_http_credentials.password can't be empty", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
+ throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Configuration parameter interserver_http_credentials.password can't be empty");
if (!config.has("user") && config.has("password"))
- throw Exception("Configuration parameter interserver_http_credentials.user can't be empty if user specified", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
+ throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG,
+ "Configuration parameter interserver_http_credentials.user can't be empty if user specified");
/// They both can be empty
auto user = config.getString(root_tag + ".user", "");
diff --git a/src/Interpreters/InterserverIOHandler.h b/src/Interpreters/InterserverIOHandler.h
index 69b742db2ec..375c6ee9ca5 100644
--- a/src/Interpreters/InterserverIOHandler.h
+++ b/src/Interpreters/InterserverIOHandler.h
@@ -7,11 +7,11 @@
#include
#include
#include
+#include
#include
#include
#include