mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Fixed error [#CLICKHOUSE-2].
This commit is contained in:
parent
d877718bb6
commit
4e67e05835
@ -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<int>(type), ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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<int>(status),
|
||||
DB::ErrorCodes::COLLATION_COMPARISON_FAILED);
|
||||
|
||||
/** Values of enum UCollationResult are equals to what exactly we need:
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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))
|
||||
{
|
||||
|
@ -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<int>(command.type), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <DB/DataStreams/OneBlockInputStream.h>
|
||||
#include <thread>
|
||||
#include <iostream>
|
||||
#include <cstddef>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -120,7 +121,7 @@ public:
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
return "SynchronousQueryKiller_" + toString(this);
|
||||
return "SynchronousQueryKiller_" + toString(intptr_t(this));
|
||||
}
|
||||
|
||||
Block readImpl() override
|
||||
|
@ -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;
|
||||
|
@ -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())
|
||||
|
@ -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<int>(mode), ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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<int>(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<int>(data.merging_params.mode), ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
merged_stream->readPrefix();
|
||||
|
@ -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<int>(type), ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
out << '\n';
|
||||
|
Loading…
Reference in New Issue
Block a user