From a38317e1f5b55fb8a389378286733f88f2975c6a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Oct 2013 03:20:51 +0000 Subject: [PATCH] dbms: better exception messages [#METR-8224]. --- .../DB/Client/ConnectionPoolWithFailover.h | 6 +- dbms/include/DB/Core/Exception.h | 3 + dbms/include/DB/IO/RemoteWriteBuffer.h | 2 +- dbms/src/Client/Client.cpp | 4 +- dbms/src/Client/Connection.cpp | 77 +++++++++++-------- dbms/src/Core/Exception.cpp | 5 ++ .../BlockInputStreamFromRowInputStream.cpp | 5 +- dbms/src/IO/ReadHelpers.cpp | 2 +- dbms/src/Interpreters/Context.cpp | 2 +- .../Interpreters/InterpreterAlterQuery.cpp | 8 +- .../Interpreters/InterpreterSelectQuery.cpp | 2 + dbms/src/Interpreters/loadMetadata.cpp | 4 +- dbms/src/Parsers/formatAST.cpp | 2 +- dbms/src/Server/HTTPHandler.cpp | 2 +- dbms/src/Server/OLAPHTTPHandler.cpp | 2 +- dbms/src/Server/TCPHandler.cpp | 14 ++-- dbms/src/Storages/IStorage.cpp | 4 +- dbms/src/Storages/StorageChunkMerger.cpp | 6 +- dbms/src/Storages/StorageChunkRef.cpp | 2 +- 19 files changed, 87 insertions(+), 65 deletions(-) diff --git a/dbms/include/DB/Client/ConnectionPoolWithFailover.h b/dbms/include/DB/Client/ConnectionPoolWithFailover.h index 32a393705cc..3d9dd76f7e2 100644 --- a/dbms/include/DB/Client/ConnectionPoolWithFailover.h +++ b/dbms/include/DB/Client/ConnectionPoolWithFailover.h @@ -54,12 +54,12 @@ public: res->forceConnected(); return res; } - catch (const DB::Exception & e) + catch (const Exception & e) { if (e.code() != ErrorCodes::NETWORK_ERROR && e.code() != ErrorCodes::SOCKET_TIMEOUT) throw; - fail_message << "DB::Exception. Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what(); + fail_message << "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what(); } LOG_WARNING(log, "Connection failed at try №" @@ -71,7 +71,7 @@ public: } } - throw DB::Exception("All connection tries failed. Log: \n\n" + fail_messages.str() + "\n", + throw Exception("All connection tries failed. Log: \n\n" + fail_messages.str() + "\n", ErrorCodes::ALL_CONNECTION_TRIES_FAILED); } diff --git a/dbms/include/DB/Core/Exception.h b/dbms/include/DB/Core/Exception.h index e7177012608..443ad13c43d 100644 --- a/dbms/include/DB/Core/Exception.h +++ b/dbms/include/DB/Core/Exception.h @@ -28,6 +28,9 @@ public: Exception * clone() const; void rethrow() const; + /// Дописать к существующему сообщению что-нибудь ещё. + void addMessage(const std::string & arg); + const StackTrace & getStackTrace() const { return trace; } private: diff --git a/dbms/include/DB/IO/RemoteWriteBuffer.h b/dbms/include/DB/IO/RemoteWriteBuffer.h index 3679a2d5665..7278b11b34c 100644 --- a/dbms/include/DB/IO/RemoteWriteBuffer.h +++ b/dbms/include/DB/IO/RemoteWriteBuffer.h @@ -126,7 +126,7 @@ public: { impl->next(); } - catch (const DB::Exception & e) + catch (const Exception & e) { if (e.code() == ErrorCodes::CANNOT_WRITE_TO_OSTREAM) checkStatus(); /// Меняем сообщение об ошибке на более ясное. diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index 8d46373a0ed..194fda88d82 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -213,7 +213,7 @@ private: { return mainImpl(args); } - catch (const DB::Exception & e) + catch (const Exception & e) { std::string text = e.displayText(); @@ -399,7 +399,7 @@ private: if (!process(query)) break; } - catch (const DB::Exception & e) + catch (const Exception & e) { std::cerr << std::endl << "Exception on client:" << std::endl diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 6073b1ffa91..b64c4a0a616 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -268,47 +268,58 @@ bool Connection::poll(size_t timeout_microseconds) Connection::Packet Connection::receivePacket() { //LOG_TRACE(log, "Receiving packet (" << getServerAddress() << ")"); - - Packet res; - readVarUInt(res.type, *in); - switch (res.type) + try { - case Protocol::Server::Data: - res.block = receiveData(); - return res; + Packet res; + readVarUInt(res.type, *in); - case Protocol::Server::Exception: - res.exception = receiveException(); - return res; + switch (res.type) + { + case Protocol::Server::Data: + res.block = receiveData(); + return res; - case Protocol::Server::Progress: - res.progress = receiveProgress(); - return res; - - case Protocol::Server::ProfileInfo: - res.profile_info = receiveProfileInfo(); - return res; + case Protocol::Server::Exception: + res.exception = receiveException(); + return res; - case Protocol::Server::Totals: - /// Блок с тотальными значениями передаётся так же, как обычный блок данных. Разница только в идентификаторе пакета. - res.block = receiveData(); - return res; + case Protocol::Server::Progress: + res.progress = receiveProgress(); + return res; - case Protocol::Server::Extremes: - /// Аналогично. - res.block = receiveData(); - return res; + case Protocol::Server::ProfileInfo: + res.profile_info = receiveProfileInfo(); + return res; - case Protocol::Server::EndOfStream: - return res; + case Protocol::Server::Totals: + /// Блок с тотальными значениями передаётся так же, как обычный блок данных. Разница только в идентификаторе пакета. + res.block = receiveData(); + return res; - default: - /// Закроем соединение, чтобы не было рассинхронизации. - disconnect(); - throw Exception("Unknown packet " - + toString(res.type) - + " from server " + getServerAddress(), ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); + case Protocol::Server::Extremes: + /// Аналогично. + res.block = receiveData(); + return res; + + case Protocol::Server::EndOfStream: + return res; + + default: + /// Закроем соединение, чтобы не было рассинхронизации. + disconnect(); + throw Exception("Unknown packet " + + toString(res.type) + + " from server " + getServerAddress(), ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); + } + } + catch (Exception & e) + { + /// Дописываем в текст исключения адрес сервера, если надо. + if (e.code() != ErrorCodes::UNKNOWN_PACKET_FROM_SERVER) + e.addMessage("while receiving packet from " + getServerAddress()); + + throw; } } diff --git a/dbms/src/Core/Exception.cpp b/dbms/src/Core/Exception.cpp index 5987995cf69..28f41eb3e63 100644 --- a/dbms/src/Core/Exception.cpp +++ b/dbms/src/Core/Exception.cpp @@ -44,6 +44,11 @@ void Exception::rethrow() const throw *this; } +void Exception::addMessage(const std::string & arg) +{ + extendedMessage(arg); +} + void throwFromErrno(const std::string & s, int code, int e) { diff --git a/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.cpp b/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.cpp index 277bcfe299c..af46a124cd0 100644 --- a/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.cpp +++ b/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.cpp @@ -48,9 +48,10 @@ Block BlockInputStreamFromRowInputStream::readImpl() res.getByPosition(i).column->insert(row[i]); } } - catch (const DB::Exception & e) + catch (Exception & e) { - throw DB::Exception(e.message() + " (at row " + toString(total_rows + 1) + ")", e, e.code()); + e.addMessage("(at row " + toString(total_rows + 1) + ")"); + throw; } res.optimizeNestedArraysOffsets(); diff --git a/dbms/src/IO/ReadHelpers.cpp b/dbms/src/IO/ReadHelpers.cpp index 3ebe969c6ed..dcb42e86d54 100644 --- a/dbms/src/IO/ReadHelpers.cpp +++ b/dbms/src/IO/ReadHelpers.cpp @@ -125,7 +125,7 @@ void readEscapedString(DB::String & s, DB::ReadBuffer & buf) { ++buf.position(); if (buf.eof()) - throw DB::Exception("Cannot parse escape sequence", DB::ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE); + throw Exception("Cannot parse escape sequence", DB::ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE); s += DB::parseEscapeSequence(*buf.position()); ++buf.position(); } diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 84a1cc17b66..c222b65d46c 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -264,7 +264,7 @@ ASTPtr Context::getCreateQuery(const String & database_name, const String & tabl /// Распарсенный запрос должен заканчиваться на конец входных данных или на точку с запятой. if (!parse_res || (pos != end && *pos != ';')) - throw DB::Exception("Syntax error while parsing query from file " + metadata_path + ": failed at position " + throw Exception("Syntax error while parsing query from file " + metadata_path + ": failed at position " + toString(pos - begin) + ": " + std::string(pos, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, end - pos)) + ", expected " + (parse_res ? "end of query" : expected) + ".", diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index a5cfa03b59e..ed169651249 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -69,7 +69,7 @@ void InterpreterAlterQuery::execute() /// Проверяем, что колонка еще не существует if (std::find_if(columns_copy.begin(), columns_copy.end(), boost::bind(namesEqual, name_type.name, _1)) != columns_copy.end()) - throw DB::Exception("Wrong column name. Column already exists", DB::ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Wrong column name. Column already exists", DB::ErrorCodes::ILLEGAL_COLUMN); /// Проверяем опциональный аргумент AFTER ASTs::iterator insert_it = columns_copy.end(); @@ -78,7 +78,7 @@ void InterpreterAlterQuery::execute() const ASTIdentifier & col_after = dynamic_cast(*params.column); insert_it = std::find_if(columns_copy.begin(), columns_copy.end(), boost::bind(namesEqual, col_after.name, _1)) ; if (insert_it == columns_copy.end()) - throw DB::Exception("Wrong column name. Cannot find column to insert after", DB::ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Wrong column name. Cannot find column to insert after", DB::ErrorCodes::ILLEGAL_COLUMN); } columns_copy.insert(insert_it, params.name_type); } @@ -88,11 +88,11 @@ void InterpreterAlterQuery::execute() /// Проверяем, что поле не является ключевым if (identifier_names.find(drop_column.name) != identifier_names.end()) - throw DB::Exception("Cannot drop key column", DB::ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Cannot drop key column", DB::ErrorCodes::ILLEGAL_COLUMN); ASTs::iterator drop_it = std::find_if(columns_copy.begin(), columns_copy.end(), boost::bind(namesEqual, drop_column.name, _1)); if (drop_it == columns_copy.end()) - throw DB::Exception("Wrong column name. Cannot find column to drop", DB::ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Wrong column name. Cannot find column to drop", DB::ErrorCodes::ILLEGAL_COLUMN); else columns_copy.erase(drop_it); } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 885bb614f9a..473133d4d74 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -471,6 +471,8 @@ void InterpreterSelectQuery::executeAggregation(BlockInputStreams & streams, Exp AggregateDescriptions aggregates; query_analyzer->getAggregateInfo(key_names, aggregates); + /// TODO: Оптимизация для случая, когда есть LIMIT, но нет HAVING и ORDER BY. + bool separate_totals = to_stage > QueryProcessingStage::WithMergeableState; /// Если источников несколько, то выполняем параллельную агрегацию diff --git a/dbms/src/Interpreters/loadMetadata.cpp b/dbms/src/Interpreters/loadMetadata.cpp index cbbb1712470..87e41fbd5e5 100644 --- a/dbms/src/Interpreters/loadMetadata.cpp +++ b/dbms/src/Interpreters/loadMetadata.cpp @@ -33,7 +33,7 @@ static void executeCreateQuery(const String & query, Context & context, const St /// Распарсенный запрос должен заканчиваться на конец входных данных или на точку с запятой. if (!parse_res || (pos != end && *pos != ';')) - throw DB::Exception("Syntax error while executing query from file " + file_name + ": failed at position " + throw Exception("Syntax error while executing query from file " + file_name + ": failed at position " + toString(pos - begin) + ": " + std::string(pos, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, end - pos)) + ", expected " + (parse_res ? "end of query" : expected) + ".", @@ -131,7 +131,7 @@ void loadMetadata(Context & context) { executeCreateQuery(s, context, it.name(), tables[j]); } - catch (const DB::Exception & e) + catch (const Exception & e) { throw Exception("Cannot create table from metadata file " + tables[j] + ", error: " + e.displayText(), ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA); diff --git a/dbms/src/Parsers/formatAST.cpp b/dbms/src/Parsers/formatAST.cpp index 78102b955ee..6642467f764 100644 --- a/dbms/src/Parsers/formatAST.cpp +++ b/dbms/src/Parsers/formatAST.cpp @@ -68,7 +68,7 @@ void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, b DISPATCH(AlterQuery) DISPATCH(ShowProcesslistQuery) else - throw DB::Exception("Unknown element in AST: " + ast.getID() + " '" + std::string(ast.range.first, ast.range.second - ast.range.first) + "'", + throw Exception("Unknown element in AST: " + ast.getID() + " '" + std::string(ast.range.first, ast.range.second - ast.range.first) + "'", ErrorCodes::UNKNOWN_ELEMENT_IN_AST); #undef DISPATCH diff --git a/dbms/src/Server/HTTPHandler.cpp b/dbms/src/Server/HTTPHandler.cpp index a812d4d3135..4a864925037 100644 --- a/dbms/src/Server/HTTPHandler.cpp +++ b/dbms/src/Server/HTTPHandler.cpp @@ -167,7 +167,7 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne processQuery(request, response); LOG_INFO(log, "Done processing query"); } - catch (DB::Exception & e) + catch (Exception & e) { response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); std::stringstream s; diff --git a/dbms/src/Server/OLAPHTTPHandler.cpp b/dbms/src/Server/OLAPHTTPHandler.cpp index e0171ffa8ac..541ecc9969b 100644 --- a/dbms/src/Server/OLAPHTTPHandler.cpp +++ b/dbms/src/Server/OLAPHTTPHandler.cpp @@ -103,7 +103,7 @@ namespace DB processQuery(request, response); LOG_INFO(log, "Done processing query"); } - catch (DB::Exception & e) + catch (Exception & e) { response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); std::stringstream s; diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index decf160728c..cb7fbb933bd 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -43,7 +43,7 @@ void TCPHandler::runImpl() { receiveHello(); } - catch (const DB::Exception & e) /// Типично при неправильном имени пользователя, пароле, адресе. + catch (const Exception & e) /// Типично при неправильном имени пользователя, пароле, адресе. { try { @@ -61,7 +61,7 @@ void TCPHandler::runImpl() if (!connection_context.isDatabaseExist(default_database)) { Exception e("Database " + default_database + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE); - LOG_ERROR(log, "DB::Exception. Code: " << e.code() << ", e.displayText() = " << e.displayText() + LOG_ERROR(log, "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", Stack trace:\n\n" << e.getStackTrace().toString()); sendException(e); return; @@ -90,7 +90,7 @@ void TCPHandler::runImpl() /** Исключение во время выполнения запроса (его надо отдать по сети клиенту). * Клиент сможет его принять, если оно не произошло во время отправки другого пакета и клиент ещё не разорвал соединение. */ - SharedPtr exception; + SharedPtr exception; try { @@ -118,9 +118,9 @@ void TCPHandler::runImpl() state.reset(); } - catch (const DB::Exception & e) + catch (const Exception & e) { - LOG_ERROR(log, "DB::Exception. Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what() + LOG_ERROR(log, "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what() << ", Stack trace:\n\n" << e.getStackTrace().toString()); exception = e.clone(); @@ -621,9 +621,9 @@ void TCPHandler::run() LOG_INFO(log, "Done processing connection."); } - catch (DB::Exception & e) + catch (Exception & e) { - LOG_ERROR(log, "DB::Exception. Code: " << e.code() << ", e.displayText() = " << e.displayText() + LOG_ERROR(log, "Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", Stack trace:\n\n" << e.getStackTrace().toString()); } catch (Poco::Exception & e) diff --git a/dbms/src/Storages/IStorage.cpp b/dbms/src/Storages/IStorage.cpp index aa421eee527..66f088e6550 100644 --- a/dbms/src/Storages/IStorage.cpp +++ b/dbms/src/Storages/IStorage.cpp @@ -168,7 +168,7 @@ void IStorage::alter_columns(const ASTAlterQuery::Parameters & params, NamesAndT NamesAndTypesList::reverse_iterator reverse_insert_it = std::find_if(columns->rbegin(), columns->rend(), boost::bind(namesEqual, column_name, _1) ); if (reverse_insert_it == columns->rend()) - throw DB::Exception("Wrong column name. Cannot find column to insert after", DB::ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Wrong column name. Cannot find column to insert after", DB::ErrorCodes::ILLEGAL_COLUMN); else { /// base возвращает итератор уже смещенный на один элемент вправо @@ -206,7 +206,7 @@ void IStorage::alter_columns(const ASTAlterQuery::Parameters & params, NamesAndT if (column_it == columns->end()) { if (is_first) - throw DB::Exception("Wrong column name. Cannot find column to drop", DB::ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Wrong column name. Cannot find column to drop", DB::ErrorCodes::ILLEGAL_COLUMN); } else columns->erase(column_it); diff --git a/dbms/src/Storages/StorageChunkMerger.cpp b/dbms/src/Storages/StorageChunkMerger.cpp index 9522e6e3a01..454450e971f 100644 --- a/dbms/src/Storages/StorageChunkMerger.cpp +++ b/dbms/src/Storages/StorageChunkMerger.cpp @@ -172,9 +172,9 @@ void StorageChunkMerger::mergeThread() merged = maybeMergeSomething(); error = false; } - catch (const DB::Exception & e) + catch (const Exception & e) { - LOG_ERROR(log, "StorageChunkMerger at " << this_database << "." << name << " failed to merge: DB::Exception. Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what() + LOG_ERROR(log, "StorageChunkMerger at " << this_database << "." << name << " failed to merge: Code: " << e.code() << ", e.displayText() = " << e.displayText() << ", e.what() = " << e.what() << ", Stack trace:\n\n" << e.getStackTrace().toString()); } catch (const Poco::Exception & e) @@ -351,7 +351,7 @@ bool StorageChunkMerger::mergeChunks(const Storages & chunks) /// Распарсенный запрос должен заканчиваться на конец входных данных. if (!parse_res || pos != end) - throw DB::Exception("Syntax error while parsing create query made by ChunkMerger." + throw Exception("Syntax error while parsing create query made by ChunkMerger." " The query is \"" + create_query + "\"." + " Failed at position " + toString(pos - begin) + ": " + std::string(pos, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, end - pos)) diff --git a/dbms/src/Storages/StorageChunkRef.cpp b/dbms/src/Storages/StorageChunkRef.cpp index 535451b27cd..1f42a1ddb3f 100644 --- a/dbms/src/Storages/StorageChunkRef.cpp +++ b/dbms/src/Storages/StorageChunkRef.cpp @@ -49,7 +49,7 @@ void StorageChunkRef::dropImpl() { getSource().removeReference(); } - catch (const DB::Exception & e) + catch (const Exception & e) { if (e.code() != ErrorCodes::UNKNOWN_TABLE) throw;