From 4e67e05835d2f0fd2e573e710093e1f1da1db63d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 14 Mar 2017 03:14:19 +0300 Subject: [PATCH] Fixed error [#CLICKHOUSE-2]. --- .../DB/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h | 2 +- dbms/src/Common/Collator.cpp | 2 +- dbms/src/Interpreters/ExpressionActions.cpp | 4 ++-- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 2 +- dbms/src/Interpreters/InterpreterAlterQuery.cpp | 2 +- dbms/src/Interpreters/InterpreterKillQueryQuery.cpp | 3 ++- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- dbms/src/Interpreters/ProcessList.cpp | 2 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp | 4 ++-- dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp | 2 +- 11 files changed, 15 insertions(+), 14 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index b8a37492680..5a5fc38e012 100644 --- a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -47,7 +47,7 @@ struct ReplicatedMergeTreeLogEntryData case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE"; case ReplicatedMergeTreeLogEntryData::ATTACH_PART: return "ATTACH_PART"; default: - throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR); + throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR); } } diff --git a/dbms/src/Common/Collator.cpp b/dbms/src/Common/Collator.cpp index 0b1e7d9010c..a51b18fd410 100644 --- a/dbms/src/Common/Collator.cpp +++ b/dbms/src/Common/Collator.cpp @@ -39,7 +39,7 @@ int Collator::compare(const char * str1, size_t length1, const char * str2, size UCollationResult compare_result = ucol_strcollIter(collator, &iter1, &iter2, &status); if (status != U_ZERO_ERROR) - throw DB::Exception("ICU collation comparison failed with error code: " + DB::toString(status), + throw DB::Exception("ICU collation comparison failed with error code: " + DB::toString(status), DB::ErrorCodes::COLLATION_COMPARISON_FAILED); /** Values of enum UCollationResult are equals to what exactly we need: diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 2d9844b2db9..397db2dcef7 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -534,7 +534,7 @@ void ExpressionActions::checkLimits(Block & block) const const Limits & limits = settings.limits; if (limits.max_temporary_columns && block.columns() > limits.max_temporary_columns) throw Exception("Too many temporary columns: " + block.dumpNames() - + ". Maximum: " + toString(limits.max_temporary_columns), + + ". Maximum: " + limits.max_temporary_columns.toString(), ErrorCodes::TOO_MUCH_TEMPORARY_COLUMNS); if (limits.max_temporary_non_const_columns) @@ -552,7 +552,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: " + toString(limits.max_temporary_non_const_columns), + + ". Maximum: " + limits.max_temporary_non_const_columns.toString(), ErrorCodes::TOO_MUCH_TEMPORARY_NON_CONST_COLUMNS); } } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index e104590c760..eebd1d1d689 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -706,7 +706,7 @@ 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) - throw Exception("Normalized AST is too deep. Maximum: " + toString(settings.limits.max_ast_depth), ErrorCodes::TOO_DEEP_AST); + throw Exception("Normalized AST is too deep. Maximum: " + settings.limits.max_ast_depth.toString(), ErrorCodes::TOO_DEEP_AST); if (finished_asts.count(ast)) { diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index a2e3c557af7..f6b7f2c8671 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -74,7 +74,7 @@ BlockIO InterpreterAlterQuery::execute() break; default: - throw Exception("Bad PartitionCommand::Type: " + toString(command.type), ErrorCodes::ARGUMENT_OUT_OF_BOUND); + throw Exception("Bad PartitionCommand::Type: " + toString(command.type), ErrorCodes::ARGUMENT_OUT_OF_BOUND); } } diff --git a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp index 9425e90fecc..972e2f95a1b 100644 --- a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB @@ -120,7 +121,7 @@ public: String getID() const override { - return "SynchronousQueryKiller_" + toString(this); + return "SynchronousQueryKiller_" + toString(intptr_t(this)); } Block readImpl() override diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 0b8c14f3951..ac68f4736d6 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -73,7 +73,7 @@ void InterpreterSelectQuery::init(BlockInputStreamPtr input, const Names & requi original_max_threads = settings.max_threads; if (settings.limits.max_subquery_depth && subquery_depth > settings.limits.max_subquery_depth) - throw Exception("Too deep subqueries. Maximum: " + toString(settings.limits.max_subquery_depth), + throw Exception("Too deep subqueries. Maximum: " + settings.limits.max_subquery_depth.toString(), ErrorCodes::TOO_DEEP_SUBQUERIES); if (is_first_select_inside_union_all) @@ -787,7 +787,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns() if (settings.limits.max_columns_to_read && required_columns.size() > settings.limits.max_columns_to_read) throw Exception("Limit for number of columns to read exceeded. " "Requested: " + toString(required_columns.size()) - + ", maximum: " + toString(settings.limits.max_columns_to_read), + + ", maximum: " + settings.limits.max_columns_to_read.toString(), ErrorCodes::TOO_MUCH_COLUMNS); size_t limit_length = 0; diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index a2673e34695..77e2e359aa9 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -48,7 +48,7 @@ ProcessList::EntryPtr ProcessList::insert( && user_process_list->second.queries.size() >= settings.max_concurrent_queries_for_user) throw Exception("Too much simultaneous queries for user " + client_info.current_user + ". Current: " + toString(user_process_list->second.queries.size()) - + ", maximum: " + toString(settings.max_concurrent_queries_for_user), + + ", maximum: " + settings.max_concurrent_queries_for_user.toString(), ErrorCodes::TOO_MUCH_SIMULTANEOUS_QUERIES); if (!client_info.current_query_id.empty()) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 05e146b9409..5b89e4ffef2 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -264,7 +264,7 @@ String MergeTreeData::MergingParams::getModeName() const case Graphite: return "Graphite"; default: - throw Exception("Unknown mode of operation for MergeTreeData: " + toString(mode), ErrorCodes::LOGICAL_ERROR); + throw Exception("Unknown mode of operation for MergeTreeData: " + toString(mode), ErrorCodes::LOGICAL_ERROR); } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp index 1cb7331950b..9979b710abe 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp @@ -594,7 +594,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMerger::mergePartsToTemporaryPart break; default: - throw Exception("Unknown mode of operation for MergeTreeData: " + toString(data.merging_params.mode), ErrorCodes::LOGICAL_ERROR); + throw Exception("Unknown mode of operation for MergeTreeData: " + toString(data.merging_params.mode), ErrorCodes::LOGICAL_ERROR); } String new_part_tmp_path = data.getFullPath() + "tmp_" + merged_name + "/"; @@ -993,7 +993,7 @@ MergeTreeData::PerShardDataParts MergeTreeDataMerger::reshardPartition( break; default: - throw Exception("Unknown mode of operation for MergeTreeData: " + toString(data.merging_params.mode), ErrorCodes::LOGICAL_ERROR); + throw Exception("Unknown mode of operation for MergeTreeData: " + toString(data.merging_params.mode), ErrorCodes::LOGICAL_ERROR); } merged_stream->readPrefix(); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index 6af7a1e6f40..0b189873c1f 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -47,7 +47,7 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const break; default: - throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR); + throw Exception("Unknown log entry type: " + DB::toString(type), ErrorCodes::LOGICAL_ERROR); } out << '\n';