未验证 提交 2c2d75cc 编写于 作者: A alexey-milovidov 提交者: GitHub

Don't allow exceptions without code (#3645)

* Don't allow exceptions without code #3633

* Don't allow exceptions without code #3633
上级 6288f0f0
......@@ -42,10 +42,8 @@ namespace DB
namespace ErrorCodes
{
extern const int POCO_EXCEPTION;
extern const int STD_EXCEPTION;
extern const int UNKNOWN_EXCEPTION;
extern const int BAD_ARGUMENTS;
extern const int EMPTY_DATA_PASSED;
}
class Benchmark
......@@ -170,7 +168,7 @@ private:
}
if (queries.empty())
throw Exception("Empty list of queries.");
throw Exception("Empty list of queries.", ErrorCodes::EMPTY_DATA_PASSED);
std::cerr << "Loaded " << queries.size() << " queries.\n";
}
......
......@@ -86,9 +86,6 @@ namespace DB
namespace ErrorCodes
{
extern const int POCO_EXCEPTION;
extern const int STD_EXCEPTION;
extern const int UNKNOWN_EXCEPTION;
extern const int NETWORK_ERROR;
extern const int NO_DATA_TO_INSERT;
extern const int BAD_ARGUMENTS;
......
......@@ -49,10 +49,10 @@ namespace DB
{
namespace ErrorCodes
{
extern const int POCO_EXCEPTION;
extern const int STD_EXCEPTION;
extern const int UNKNOWN_EXCEPTION;
extern const int NOT_IMPLEMENTED;
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
extern const int FILE_DOESNT_EXIST;
}
static String pad(size_t padding)
......@@ -156,7 +156,7 @@ struct StopConditionsSet
else if (key == "average_speed_not_changing_for_ms")
average_speed_not_changing_for_ms.value = stop_conditions_view->getUInt64(key);
else
throw DB::Exception("Met unkown stop condition: " + key);
throw DB::Exception("Met unkown stop condition: " + key, DB::ErrorCodes::LOGICAL_ERROR);
++initialized_count;
}
......@@ -521,7 +521,7 @@ public:
{
if (input_files.size() < 1)
{
throw DB::Exception("No tests were specified", 0);
throw DB::Exception("No tests were specified", DB::ErrorCodes::BAD_ARGUMENTS);
}
std::string name;
......@@ -694,7 +694,7 @@ private:
size_t ram_size_needed = config->getUInt64("preconditions.ram_size");
size_t actual_ram = getMemoryAmount();
if (!actual_ram)
throw DB::Exception("ram_size precondition not available on this platform", ErrorCodes::NOT_IMPLEMENTED);
throw DB::Exception("ram_size precondition not available on this platform", DB::ErrorCodes::NOT_IMPLEMENTED);
if (ram_size_needed > actual_ram)
{
......@@ -868,12 +868,12 @@ private:
if (!test_config->has("query") && !test_config->has("query_file"))
{
throw DB::Exception("Missing query fields in test's config: " + test_name);
throw DB::Exception("Missing query fields in test's config: " + test_name, DB::ErrorCodes::BAD_ARGUMENTS);
}
if (test_config->has("query") && test_config->has("query_file"))
{
throw DB::Exception("Found both query and query_file fields. Choose only one");
throw DB::Exception("Found both query and query_file fields. Choose only one", DB::ErrorCodes::BAD_ARGUMENTS);
}
if (test_config->has("query"))
......@@ -885,7 +885,7 @@ private:
{
const String filename = test_config->getString("query_file");
if (filename.empty())
throw DB::Exception("Empty file name");
throw DB::Exception("Empty file name", DB::ErrorCodes::BAD_ARGUMENTS);
bool tsv = fs::path(filename).extension().string() == ".tsv";
......@@ -909,7 +909,7 @@ private:
if (queries.empty())
{
throw DB::Exception("Did not find any query to execute: " + test_name);
throw DB::Exception("Did not find any query to execute: " + test_name, DB::ErrorCodes::BAD_ARGUMENTS);
}
if (test_config->has("substitutions"))
......@@ -929,7 +929,7 @@ private:
if (!test_config->has("type"))
{
throw DB::Exception("Missing type property in config: " + test_name);
throw DB::Exception("Missing type property in config: " + test_name, DB::ErrorCodes::BAD_ARGUMENTS);
}
String config_exec_type = test_config->getString("type");
......@@ -938,7 +938,7 @@ private:
else if (config_exec_type == "once")
exec_type = ExecutionType::Once;
else
throw DB::Exception("Unknown type " + config_exec_type + " in :" + test_name);
throw DB::Exception("Unknown type " + config_exec_type + " in :" + test_name, DB::ErrorCodes::BAD_ARGUMENTS);
times_to_run = test_config->getUInt("times_to_run", 1);
......@@ -951,7 +951,7 @@ private:
}
if (stop_conditions_template.empty())
throw DB::Exception("No termination conditions were found in config");
throw DB::Exception("No termination conditions were found in config", DB::ErrorCodes::BAD_ARGUMENTS);
for (size_t i = 0; i < times_to_run * queries.size(); ++i)
stop_conditions_by_run.push_back(stop_conditions_template);
......@@ -978,7 +978,7 @@ private:
else
{
if (lite_output)
throw DB::Exception("Specify main_metric for lite output");
throw DB::Exception("Specify main_metric for lite output", DB::ErrorCodes::BAD_ARGUMENTS);
}
if (metrics.size() > 0)
......@@ -1023,22 +1023,14 @@ private:
if (exec_type == ExecutionType::Loop)
{
for (const String & metric : metrics)
{
if (std::find(non_loop_metrics.begin(), non_loop_metrics.end(), metric) != non_loop_metrics.end())
{
throw DB::Exception("Wrong type of metric for loop execution type (" + metric + ")");
}
}
throw DB::Exception("Wrong type of metric for loop execution type (" + metric + ")", DB::ErrorCodes::BAD_ARGUMENTS);
}
else
{
for (const String & metric : metrics)
{
if (std::find(loop_metrics.begin(), loop_metrics.end(), metric) != loop_metrics.end())
{
throw DB::Exception("Wrong type of metric for non-loop execution type (" + metric + ")");
}
}
throw DB::Exception("Wrong type of metric for non-loop execution type (" + metric + ")", DB::ErrorCodes::BAD_ARGUMENTS);
}
}
......@@ -1439,7 +1431,7 @@ try
if (input_files.empty())
{
std::cerr << std::endl;
throw DB::Exception("Did not find any xml files");
throw DB::Exception("Did not find any xml files", DB::ErrorCodes::BAD_ARGUMENTS);
}
else
std::cerr << " found " << input_files.size() << " files." << std::endl;
......@@ -1454,7 +1446,7 @@ try
fs::path file(filename);
if (!fs::exists(file))
throw DB::Exception("File '" + filename + "' does not exist");
throw DB::Exception("File '" + filename + "' does not exist", DB::ErrorCodes::FILE_DOESNT_EXIST);
if (fs::is_directory(file))
{
......@@ -1463,7 +1455,7 @@ try
else
{
if (file.extension().string() != ".xml")
throw DB::Exception("File '" + filename + "' does not have .xml extension");
throw DB::Exception("File '" + filename + "' does not have .xml extension", DB::ErrorCodes::BAD_ARGUMENTS);
collected_files.push_back(filename);
}
}
......
......@@ -19,9 +19,6 @@ namespace DB
namespace ErrorCodes
{
extern const int ABORTED;
extern const int POCO_EXCEPTION;
extern const int STD_EXCEPTION;
extern const int UNKNOWN_EXCEPTION;
extern const int TOO_MANY_SIMULTANEOUS_QUERIES;
}
......
......@@ -66,6 +66,7 @@ namespace ErrorCodes
extern const int SUPPORT_IS_DISABLED;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int EXCESSIVE_ELEMENT_IN_CONFIG;
extern const int INVALID_CONFIG_PARAMETER;
extern const int SYSTEM_ERROR;
}
......@@ -74,7 +75,7 @@ static std::string getCanonicalPath(std::string && path)
{
Poco::trimInPlace(path);
if (path.empty())
throw Exception("path configuration parameter is empty");
throw Exception("path configuration parameter is empty", ErrorCodes::INVALID_CONFIG_PARAMETER);
if (path.back() != '/')
path += '/';
return std::move(path);
......
......@@ -16,10 +16,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
struct ComparePairFirst final
......@@ -191,7 +193,7 @@ public:
const auto time_arg = arguments.front().get();
if (!WhichDataType(time_arg).isDateTime() && !WhichDataType(time_arg).isUInt32())
throw Exception{"Illegal type " + time_arg->getName() + " of first argument of aggregate function " + getName()
+ ", must be DateTime or UInt32"};
+ ", must be DateTime or UInt32", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
for (const auto i : ext::range(1, arguments.size()))
{
......
......@@ -400,6 +400,7 @@ namespace ErrorCodes
extern const int CANNOT_GETTIMEOFDAY = 423;
extern const int CANNOT_LINK = 424;
extern const int SYSTEM_ERROR = 425;
extern const int NULL_POINTER_DEREFERENCE = 426;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;
......
......@@ -23,10 +23,12 @@ class Exception : public Poco::Exception
{
public:
Exception() {} /// For deferred initialization.
Exception(const std::string & msg, int code = 0) : Poco::Exception(msg, code) {}
Exception(const std::string & msg, const Exception & nested_exception, int code = 0)
Exception(const std::string & msg, int code) : Poco::Exception(msg, code) {}
Exception(const std::string & msg, const Exception & nested_exception, int code)
: Poco::Exception(msg, nested_exception, code), trace(nested_exception.trace) {}
explicit Exception(const Poco::Exception & exc) : Poco::Exception(exc.displayText(), ErrorCodes::POCO_EXCEPTION) {}
enum CreateFromPocoTag { CreateFromPoco };
Exception(CreateFromPocoTag, const Poco::Exception & exc) : Poco::Exception(exc.displayText(), ErrorCodes::POCO_EXCEPTION) {}
Exception * clone() const override { return new Exception(*this); }
void rethrow() const override { throw *this; }
......
......@@ -3,6 +3,15 @@
#include <Common/HashTable/HashMap.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
}
}
/** Replacement of the hash table for a small number (<10) of keys.
* Implemented as an array with linear search.
* The array is located inside the object.
......@@ -13,7 +22,6 @@
* you should check if the table is not full,
* and do a `fallback` in this case (for example, use a real hash table).
*/
template
<
typename Key,
......@@ -86,7 +94,7 @@ public:
DB::readVarUInt(size, in);
if (size > capacity)
throw DB::Exception("Illegal size");
throw DB::Exception("Illegal size", DB::ErrorCodes::INCORRECT_DATA);
is_initialized = true;
}
......@@ -306,7 +314,7 @@ public:
DB::readVarUInt(new_size, rb);
if (new_size > capacity)
throw DB::Exception("Illegal size");
throw DB::Exception("Illegal size", DB::ErrorCodes::INCORRECT_DATA);
for (size_t i = 0; i < new_size; ++i)
buf[i].read(rb);
......@@ -324,7 +332,7 @@ public:
DB::readText(new_size, rb);
if (new_size > capacity)
throw DB::Exception("Illegal size");
throw DB::Exception("Illegal size", DB::ErrorCodes::INCORRECT_DATA);
for (size_t i = 0; i < new_size; ++i)
{
......
......@@ -9,7 +9,7 @@ namespace DB
class NetException : public Exception
{
public:
NetException(const std::string & msg, int code = 0) : Exception(msg, code) {}
NetException(const std::string & msg, int code) : Exception(msg, code) {}
NetException * clone() const override { return new NetException(*this); }
void rethrow() const override { throw *this; }
......
......@@ -305,7 +305,7 @@ void PoolWithFailoverBase<TNestedPool>::reportError(const Entry & entry)
return;
}
}
throw DB::Exception("Can't find pool to report error.");
throw DB::Exception("Can't find pool to report error", DB::ErrorCodes::LOGICAL_ERROR);
}
template <typename TNestedPool>
......
......@@ -59,7 +59,7 @@ StatusFile::StatusFile(const std::string & path_)
if (-1 == flock_ret)
{
if (errno == EWOULDBLOCK)
throw Exception("Cannot lock file " + path + ". Another server instance in same directory is already running.");
throw Exception("Cannot lock file " + path + ". Another server instance in same directory is already running.", ErrorCodes::CANNOT_OPEN_FILE);
else
throwFromErrno("Cannot lock file " + path, ErrorCodes::CANNOT_OPEN_FILE);
}
......
#include "ZooKeeperHolder.h"
namespace DB
{
namespace ErrorCodes
{
extern const int NULL_POINTER_DEREFERENCE;
}
}
using namespace zkutil;
ZooKeeperHolder::UnstorableZookeeperHandler ZooKeeperHolder::getZooKeeper()
......@@ -47,7 +57,7 @@ ZooKeeperHolder::UnstorableZookeeperHandler::UnstorableZookeeperHandler(ZooKeepe
ZooKeeper * ZooKeeperHolder::UnstorableZookeeperHandler::operator->()
{
if (zk_ptr == nullptr)
throw DB::Exception(nullptr_exception_message);
throw DB::Exception(nullptr_exception_message, DB::ErrorCodes::NULL_POINTER_DEREFERENCE);
return zk_ptr.get();
}
......@@ -55,20 +65,20 @@ ZooKeeper * ZooKeeperHolder::UnstorableZookeeperHandler::operator->()
const ZooKeeper * ZooKeeperHolder::UnstorableZookeeperHandler::operator->() const
{
if (zk_ptr == nullptr)
throw DB::Exception(nullptr_exception_message);
throw DB::Exception(nullptr_exception_message, DB::ErrorCodes::NULL_POINTER_DEREFERENCE);
return zk_ptr.get();
}
ZooKeeper & ZooKeeperHolder::UnstorableZookeeperHandler::operator*()
{
if (zk_ptr == nullptr)
throw DB::Exception(nullptr_exception_message);
throw DB::Exception(nullptr_exception_message, DB::ErrorCodes::NULL_POINTER_DEREFERENCE);
return *zk_ptr;
}
const ZooKeeper & ZooKeeperHolder::UnstorableZookeeperHandler::operator*() const
{
if (zk_ptr == nullptr)
throw DB::Exception(nullptr_exception_message);
throw DB::Exception(nullptr_exception_message, DB::ErrorCodes::NULL_POINTER_DEREFERENCE);
return *zk_ptr;
}
add_executable (exception exception.cpp)
target_link_libraries (exception PRIVATE clickhouse_common_io)
add_executable (string_pool string_pool.cpp)
target_link_libraries (string_pool PRIVATE clickhouse_common_io)
target_include_directories (string_pool SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR})
......
#include <iostream>
#include <Poco/Net/NetException.h>
#include <Common/Exception.h>
int main(int, char **)
{
try
{
//throw Poco::Net::ConnectionRefusedException();
throw DB::Exception(Poco::Net::ConnectionRefusedException());
}
catch (const DB::Exception & e)
{
std::cerr << e.displayText() << std::endl;
}
catch (const Poco::Exception & e)
{
std::cerr << e.displayText() << std::endl;
}
return 0;
}
......@@ -13,6 +13,15 @@
using namespace DB;
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
}
int main(int argc, char ** argv)
{
srand(123456);
......@@ -75,7 +84,7 @@ int main(int argc, char ** argv)
}
if (res_block.rows() != n * m)
throw Exception("Result block size mismatch");
throw Exception("Result block size mismatch", ErrorCodes::LOGICAL_ERROR);
const auto & columns = res_block.getColumns();
......@@ -86,7 +95,7 @@ int main(int argc, char ** argv)
if (res < 0)
break;
else if (res > 0)
throw Exception("Result stream not sorted");
throw Exception("Result stream not sorted", ErrorCodes::LOGICAL_ERROR);
}
stopwatch.stop();
......
......@@ -392,7 +392,7 @@ void DatabaseOrdinary::renameTable(
catch (const Poco::Exception & e)
{
/// Better diagnostics.
throw Exception{e};
throw Exception{Exception::CreateFromPoco, e};
}
ASTPtr ast = getQueryFromMetadata(detail::getTableMetadataPath(metadata_path, table_name));
......
......@@ -10,6 +10,15 @@
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
}
}
RegionsHierarchy::RegionsHierarchy(IRegionsHierarchyDataSourcePtr data_source_)
: data_source(data_source_)
{
......@@ -49,7 +58,8 @@ void RegionsHierarchy::reload()
if (region_entry.id > max_region_id)
{
if (region_entry.id > max_size)
throw DB::Exception("Region id is too large: " + DB::toString(region_entry.id) + ", should be not more than " + DB::toString(max_size));
throw DB::Exception("Region id is too large: " + DB::toString(region_entry.id) + ", should be not more than " + DB::toString(max_size),
DB::ErrorCodes::INCORRECT_DATA);
max_region_id = region_entry.id;
......
......@@ -8,6 +8,14 @@
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
}
}
RegionsNames::RegionsNames(IRegionsNamesDataProviderPtr data_provider)
{
......@@ -78,7 +86,8 @@ void RegionsNames::reload()
max_region_id = name_entry.id;
if (name_entry.id > max_size)
throw DB::Exception("Region id is too large: " + DB::toString(name_entry.id) + ", should be not more than " + DB::toString(max_size));
throw DB::Exception("Region id is too large: " + DB::toString(name_entry.id) + ", should be not more than " + DB::toString(max_size),
DB::ErrorCodes::INCORRECT_DATA);
}
while (name_entry.id >= new_names_refs.size())
......
......@@ -18,6 +18,13 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_TYPE_OF_FIELD;
extern const int BAD_ARGUMENTS;
extern const int THERE_IS_NO_COLUMN;
}
static String getSchemaPath(const String & schema_dir, const String & schema_file)
{
return schema_dir + escapeForFileName(schema_file) + ".capnp";
......@@ -42,7 +49,7 @@ Field convertNodeToField(capnp::DynamicValue::Reader value)
switch (value.getType())
{
case capnp::DynamicValue::UNKNOWN:
throw Exception("Unknown field type");
throw Exception("Unknown field type", ErrorCodes::BAD_TYPE_OF_FIELD);
case capnp::DynamicValue::VOID:
return Field();
case capnp::DynamicValue::BOOL:
......@@ -87,9 +94,9 @@ Field convertNodeToField(capnp::DynamicValue::Reader value)
return field;
}
case capnp::DynamicValue::CAPABILITY:
throw Exception("CAPABILITY type not supported");
throw Exception("CAPABILITY type not supported", ErrorCodes::BAD_TYPE_OF_FIELD);
case capnp::DynamicValue::ANY_POINTER:
throw Exception("ANY_POINTER type not supported");
throw Exception("ANY_POINTER type not supported", ErrorCodes::BAD_TYPE_OF_FIELD);
}
return Field();
}
......@@ -99,7 +106,7 @@ capnp::StructSchema::Field getFieldOrThrow(capnp::StructSchema node, const std::
KJ_IF_MAYBE(child, node.findFieldByName(field))
return *child;
else
throw Exception("Field " + field + " doesn't exist in schema " + node.getShortDisplayName().cStr());
throw Exception("Field " + field + " doesn't exist in schema " + node.getShortDisplayName().cStr(), ErrorCodes::THERE_IS_NO_COLUMN);
}
void CapnProtoRowInputStream::createActions(const NestedFieldList & sortedFields, capnp::StructSchema reader)
......@@ -135,7 +142,7 @@ void CapnProtoRowInputStream::createActions(const NestedFieldList & sortedFields
break; // Collect list
}
else
throw Exception("Field " + field.tokens[level] + "is neither Struct nor List");
throw Exception("Field " + field.tokens[level] + "is neither Struct nor List", ErrorCodes::BAD_TYPE_OF_FIELD);
}
// Read field from the structure
......@@ -290,7 +297,8 @@ void registerInputFormatCapnProto(FormatFactory & factory)
auto schema_and_root = context.getSettingsRef().format_schema.toString();
boost::split(tokens, schema_and_root, boost::is_any_of(":"));
if (tokens.size() != 2)
throw Exception("Format CapnProto requires 'format_schema' setting to have a schema_file:root_object format, e.g. 'schema.capnp:Message'");
throw Exception("Format CapnProto requires 'format_schema' setting to have a schema_file:root_object format, e.g. 'schema.capnp:Message'",
ErrorCodes::BAD_ARGUMENTS);
const String & schema_dir = context.getFormatSchemaPath();
......
......@@ -38,6 +38,15 @@
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_LARGE_STRING_SIZE;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
}
/** Comparison functions: ==, !=, <, >, <=, >=.
* The comparison functions always return 0 or 1 (UInt8).
......@@ -833,7 +842,7 @@ private:
ReadBufferFromMemory in(string_value.data, string_value.size);
readDateText(date, in);
if (!in.eof())
throw Exception("String is too long for Date: " + string_value.toString());
throw Exception("String is too long for Date: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE);
ColumnPtr parsed_const_date_holder = DataTypeDate().createColumnConst(input_rows_count, date);
const ColumnConst * parsed_const_date = static_cast<const ColumnConst *>(parsed_const_date_holder.get());
......@@ -847,7 +856,7 @@ private:
ReadBufferFromMemory in(string_value.data, string_value.size);
readDateTimeText(date_time, in);
if (!in.eof())
throw Exception("String is too long for DateTime: " + string_value.toString());
throw Exception("String is too long for DateTime: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE);
ColumnPtr parsed_const_date_time_holder = DataTypeDateTime().createColumnConst(input_rows_count, UInt64(date_time));
const ColumnConst * parsed_const_date_time = static_cast<const ColumnConst *>(parsed_const_date_time_holder.get());
......@@ -861,7 +870,7 @@ private:
ReadBufferFromMemory in(string_value.data, string_value.size);
readText(uuid, in);
if (!in.eof())
throw Exception("String is too long for UUID: " + string_value.toString());
throw Exception("String is too long for UUID: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE);
ColumnPtr parsed_const_uuid_holder = DataTypeUUID().createColumnConst(input_rows_count, uuid);
const ColumnConst * parsed_const_uuid = static_cast<const ColumnConst *>(parsed_const_uuid_holder.get());
......
......@@ -20,6 +20,8 @@ namespace DB
namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN;
extern const int LOGICAL_ERROR;
}
......@@ -200,14 +202,15 @@ private:
if (arguments.size() < 6 || arguments.size() % 4 != 2)
{
throw Exception(
"Incorrect number of arguments of function " + getName() + ". Must be 2 for your point plus 4 * N for ellipses (x_i, y_i, a_i, b_i).");
"Incorrect number of arguments of function " + getName() + ". Must be 2 for your point plus 4 * N for ellipses (x_i, y_i, a_i, b_i).",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
/// For array on stack, see below.
if (arguments.size() > 10000)
{
throw Exception(
"Number of arguments of function " + getName() + " is too large.");
"Number of arguments of function " + getName() + " is too large.", ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION);
}
for (const auto arg_idx : ext::range(0, arguments.size()))
......
......@@ -145,7 +145,7 @@ public:
String sep_str = col->getValue<String>();
if (sep_str.size() != 1)
throw Exception("Illegal separator for function " + getName() + ". Must be exactly one byte.");
throw Exception("Illegal separator for function " + getName() + ". Must be exactly one byte.", ErrorCodes::BAD_ARGUMENTS);
sep = sep_str[0];
}
......
......@@ -26,6 +26,7 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_COLUMN;
}
......@@ -976,7 +977,7 @@ public:
const ColumnPtr column_replacement = block.getByPosition(arguments[2]).column;
if (!column_needle->isColumnConst() || !column_replacement->isColumnConst())
throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants.");
throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants.", ErrorCodes::ILLEGAL_COLUMN);
const IColumn * c1 = block.getByPosition(arguments[1]).column.get();
const IColumn * c2 = block.getByPosition(arguments[2]).column.get();
......
#pragma once
#include <Functions/GatherUtils/ArraySinkVisitor.h>
#include <Common/Exception.h>
namespace DB::GatherUtils
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
namespace GatherUtils
{
struct IArraySink
......@@ -11,7 +20,7 @@ struct IArraySink
virtual void accept(ArraySinkVisitor &)
{
throw Exception("Accept not implemented for " + demangle(typeid(*this).name()));
throw Exception("Accept not implemented for " + demangle(typeid(*this).name()), ErrorCodes::NOT_IMPLEMENTED);
}
};
......@@ -19,3 +28,5 @@ template <typename Derived>
class ArraySinkImpl : public Visitable<Derived, IArraySink, ArraySinkVisitor> {};
}
}
......@@ -3,7 +3,15 @@
#include <Columns/ColumnArray.h>
#include <Functions/GatherUtils/ArraySourceVisitor.h>
namespace DB::GatherUtils
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
namespace GatherUtils
{
struct IArraySource
......@@ -18,7 +26,7 @@ struct IArraySource
virtual void accept(ArraySourceVisitor &)
{
throw Exception("Accept not implemented for " + demangle(typeid(*this).name()));
throw Exception("Accept not implemented for " + demangle(typeid(*this).name()), ErrorCodes::NOT_IMPLEMENTED);
}
};
......@@ -26,3 +34,5 @@ template <typename Derived>
class ArraySourceImpl : public Visitable<Derived, IArraySource, ArraySourceVisitor> {};
}
}
#pragma once
#include <Functions/GatherUtils/ValueSourceVisitor.h>
#include <Common/Exception.h>
namespace DB::GatherUtils
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
namespace GatherUtils
{
struct IValueSource
......@@ -11,7 +20,7 @@ struct IValueSource
virtual void accept(ValueSourceVisitor &)
{
throw Exception("Accept not implemented for " + demangle(typeid(*this).name()));
throw Exception("Accept not implemented for " + demangle(typeid(*this).name()), ErrorCodes::NOT_IMPLEMENTED);
}
virtual bool isConst() const { return false; }
......@@ -21,3 +30,5 @@ template <typename Derived>
class ValueSourceImpl : public Visitable<Derived, IValueSource, ValueSourceVisitor> {};
}
}
......@@ -142,7 +142,7 @@ struct ConstSource : public Base
else
throw Exception(
"accept(ArraySourceVisitor &) is not implemented for " + demangle(typeid(ConstSource<Base>).name())
+ " because " + demangle(typeid(Base).name()) + " is not derived from IArraySource ");
+ " because " + demangle(typeid(Base).name()) + " is not derived from IArraySource", ErrorCodes::NOT_IMPLEMENTED);
}
virtual void accept(ValueSourceVisitor & visitor) // override
......@@ -152,7 +152,7 @@ struct ConstSource : public Base
else
throw Exception(
"accept(ValueSourceVisitor &) is not implemented for " + demangle(typeid(ConstSource<Base>).name())
+ " because " + demangle(typeid(Base).name()) + " is not derived from IValueSource ");
+ " because " + demangle(typeid(Base).name()) + " is not derived from IValueSource", ErrorCodes::NOT_IMPLEMENTED);
}
void next()
......
......@@ -7,6 +7,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
struct ArrayCumSumImpl
{
static bool needBoolean() { return false; }
......@@ -108,7 +113,7 @@ struct ArrayCumSumImpl
executeType<Float64,Float64>(mapped, array, res))
return res;
else
throw Exception("Unexpected column for arrayCumSum: " + mapped->getName());
throw Exception("Unexpected column for arrayCumSum: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN);
}
};
......
......@@ -7,6 +7,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** arrayCumSumNonNegative() - returns an array with cumulative sums of the original. (If value < 0 -> 0).
*/
struct ArrayCumSumNonNegativeImpl
......@@ -88,7 +93,7 @@ struct ArrayCumSumNonNegativeImpl
executeType<Float64,Float64>(mapped, array, res))
return res;
else
throw Exception("Unexpected column for arrayCumSumNonNegativeImpl: " + mapped->getName());
throw Exception("Unexpected column for arrayCumSumNonNegativeImpl: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN);
}
};
......
......@@ -7,6 +7,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
/** arrayDifference() - returns an array with the difference between all pairs of neighboring elements.
*/
struct ArrayDifferenceImpl
......@@ -85,7 +90,7 @@ struct ArrayDifferenceImpl
executeType<Float64,Float64>(mapped, array, res))
return res;
else
throw Exception("Unexpected column for arrayDifference: " + mapped->getName());
throw Exception("Unexpected column for arrayDifference: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN);
}
};
......
......@@ -7,6 +7,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
struct ArraySumImpl
{
static bool needBoolean() { return false; }
......@@ -93,7 +98,7 @@ struct ArraySumImpl
executeType<Float64,Float64>(mapped, offsets, res))
return res;
else
throw Exception("Unexpected column for arraySum: " + mapped->getName());
throw Exception("Unexpected column for arraySum: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN);
}
};
......
......@@ -127,13 +127,13 @@ public:
const ColumnPtr column_length = block.getByPosition(arguments[2]).column;
if (!column_start->isColumnConst() || !column_length->isColumnConst())
throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants.");
throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants.", ErrorCodes::ILLEGAL_COLUMN);
Field start_field = (*block.getByPosition(arguments[1]).column)[0];
Field length_field = (*block.getByPosition(arguments[2]).column)[0];
if (start_field.getType() != Field::Types::UInt64 || length_field.getType() != Field::Types::UInt64)
throw Exception("2nd and 3rd arguments of function " + getName() + " must be non-negative and must have UInt type.");
throw Exception("2nd and 3rd arguments of function " + getName() + " must be non-negative and must have UInt type.", ErrorCodes::ILLEGAL_COLUMN);
UInt64 start = start_field.get<UInt64>();
UInt64 length = length_field.get<UInt64>();
......
......@@ -64,7 +64,7 @@ void writeException(const Exception & e, WriteBuffer & buf, bool with_stack_trac
writeBinary(has_nested, buf);
if (has_nested)
writeException(Exception(*e.nested()), buf, with_stack_trace);
writeException(Exception(Exception::CreateFromPoco, *e.nested()), buf, with_stack_trace);
}
}
......@@ -55,7 +55,7 @@ int main(int, char **)
{
std::stringstream s;
s << "Failed!, read: " << x << ", expected: " << i;
throw DB::Exception(s.str());
throw DB::Exception(s.str(), 0);
}
}
stopwatch.stop();
......
......@@ -6,6 +6,15 @@
#include <IO/ReadHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
}
int main(int, char **)
try
{
......@@ -31,18 +40,18 @@ try
if (limit_in.count() != 1)
{
s << "Failed!, incorrect count(): " << limit_in.count();
throw Exception(s.str());
throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR);
}
if (in.count() != limit_in.count())
{
s << "Failed!, incorrect underlying buffer's count(): " << in.count();
throw Exception(s.str());
throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR);
}
if (src != dst)
{
s << "Failed!, incorrect destination value, read: " << dst << ", expected: " << src;
throw Exception(s.str());
throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR);
}
}
{
......@@ -65,20 +74,20 @@ try
if (in.count() != 2)
{
s << "Failed!, Incorrect underlying buffer's count: " << in.count() << ", expected: " << 2;
throw Exception(s.str());
throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR);
}
if (limit_in.count() != 1)
{
s << "Failed!, Incorrect count: " << limit_in.count() << ", expected: " << 1;
throw Exception(s.str());
throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR);
}
}
if (dst != "b")
{
s << "Failed!, Incorrect destination value: " << dst << ", expected 'b'";
throw Exception(dst);
throw Exception(dst, ErrorCodes::LOGICAL_ERROR);
}
char y;
......@@ -86,14 +95,14 @@ try
if (y != 'c')
{
s << "Failed!, Read incorrect value from underlying buffer: " << y << ", expected 'c'";
throw Exception(s.str());
throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR);
}
while (!in.eof())
in.ignore();
if (in.count() != 3)
{
s << "Failed!, Incorrect final count from underlying buffer: " << in.count() << ", expected: 3";
throw Exception(s.str());
throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR);
}
}
......@@ -110,14 +119,14 @@ try
if (limit_in.count() != 1)
{
s << "Failed!, Incorrect count: " << limit_in.count() << ", expected: " << 1;
throw Exception(s.str());
throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR);
}
}
if (in.count() != 1)
{
s << "Failed!, Incorrect final count from underlying buffer: " << in.count() << ", expected: 1";
throw Exception(s.str());
throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR);
}
}
......
......@@ -51,7 +51,7 @@ try
inflating_buf.ignore();
if (x != i)
throw DB::Exception("Failed!, read: " + std::to_string(x) + ", expected: " + std::to_string(i));
throw DB::Exception("Failed!, read: " + std::to_string(x) + ", expected: " + std::to_string(i), 0);
}
stopwatch.stop();
std::cout << "Reading done. Elapsed: " << stopwatch.elapsedSeconds() << " s."
......
......@@ -31,6 +31,7 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int TYPE_MISMATCH;
extern const int TOO_LARGE_STRING_SIZE;
}
......@@ -123,7 +124,7 @@ DayNum stringToDate(const String & s)
readDateText(date, in);
if (!in.eof())
throw Exception("String is too long for Date: " + s);
throw Exception("String is too long for Date: " + s, ErrorCodes::TOO_LARGE_STRING_SIZE);
return date;
}
......@@ -135,7 +136,7 @@ UInt64 stringToDateTime(const String & s)
readDateTimeText(date_time, in);
if (!in.eof())
throw Exception("String is too long for DateTime: " + s);
throw Exception("String is too long for DateTime: " + s, ErrorCodes::TOO_LARGE_STRING_SIZE);
return UInt64(date_time);
}
......
......@@ -20,6 +20,14 @@
using namespace DB;
namespace DB
{
namespace ErrorCodes
{
extern const int SYNTAX_ERROR;
}
}
struct TestEntry
{
String query;
......@@ -67,7 +75,7 @@ private:
auto text = query.data();
if (ASTPtr ast = tryParseQuery(parser, text, text + query.size(), message, false, "", false, 0))
return ast;
throw Exception(message);
throw Exception(message, ErrorCodes::SYNTAX_ERROR);
}
};
......
......@@ -5,6 +5,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
ColumnDefaultKind columnDefaultKindFromString(const std::string & str)
{
......@@ -15,7 +20,7 @@ ColumnDefaultKind columnDefaultKindFromString(const std::string & str)
};
const auto it = map.find(str);
return it != std::end(map) ? it->second : throw Exception{"Unknown column default specifier: " + str};
return it != std::end(map) ? it->second : throw Exception{"Unknown column default specifier: " + str, ErrorCodes::LOGICAL_ERROR};
}
......@@ -28,7 +33,7 @@ std::string toString(const ColumnDefaultKind kind)
};
const auto it = map.find(kind);
return it != std::end(map) ? it->second : throw Exception{"Invalid ColumnDefaultKind"};
return it != std::end(map) ? it->second : throw Exception{"Invalid ColumnDefaultKind", ErrorCodes::LOGICAL_ERROR};
}
......
......@@ -19,6 +19,7 @@ namespace DB
namespace ErrorCodes
{
extern const int TABLE_IS_DROPPED;
extern const int NOT_IMPLEMENTED;
}
class Context;
......@@ -295,7 +296,7 @@ public:
virtual bool mayBenefitFromIndexForIn(const ASTPtr & /* left_in_operand */) const { return false; }
/// Checks validity of the data
virtual bool checkData() const { throw DB::Exception("Check query is not supported for " + getName() + " storage"); }
virtual bool checkData() const { throw Exception("Check query is not supported for " + getName() + " storage", ErrorCodes::NOT_IMPLEMENTED); }
/// Checks that table could be dropped right now
/// Otherwise - throws an exception with detailed information.
......
......@@ -25,6 +25,7 @@ namespace ErrorCodes
extern const int ABORTED;
extern const int BAD_SIZE_OF_FILE_IN_DATA_PART;
extern const int CANNOT_WRITE_TO_OSTREAM;
extern const int CHECKSUM_DOESNT_MATCH;
extern const int UNKNOWN_TABLE;
}
......@@ -240,7 +241,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
readPODBinary(expected_hash, in);
if (expected_hash != hashing_out.getHash())
throw Exception("Checksum mismatch for file " + absolute_part_path + file_name + " transferred from " + replica_path);
throw Exception("Checksum mismatch for file " + absolute_part_path + file_name + " transferred from " + replica_path,
ErrorCodes::CHECKSUM_DOESNT_MATCH);
if (file_name != "checksums.txt" &&
file_name != "columns.txt")
......
......@@ -193,7 +193,7 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam
for (const ExpressionAction & action : expr.getActions())
{
if (action.type == ExpressionAction::ARRAY_JOIN)
throw Exception(key_name + " key cannot contain array joins");
throw Exception(key_name + " key cannot contain array joins", ErrorCodes::ILLEGAL_COLUMN);
if (action.type == ExpressionAction::APPLY_FUNCTION)
{
......@@ -346,7 +346,7 @@ void MergeTreeData::MergingParams::check(const NamesAndTypesList & columns) cons
}
}
if (miss_column)
throw Exception("Sign column " + sign_column + " does not exist in table declaration.");
throw Exception("Sign column " + sign_column + " does not exist in table declaration.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
};
/// that if the version_column column is needed, it exists and is of unsigned integer type.
......@@ -375,7 +375,7 @@ void MergeTreeData::MergingParams::check(const NamesAndTypesList & columns) cons
}
}
if (miss_column)
throw Exception("Version column " + version_column + " does not exist in table declaration.");
throw Exception("Version column " + version_column + " does not exist in table declaration.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
};
if (mode == MergingParams::Collapsing)
......@@ -392,7 +392,7 @@ void MergeTreeData::MergingParams::check(const NamesAndTypesList & columns) cons
};
if (columns.end() == std::find_if(columns.begin(), columns.end(), check_column_to_sum_exists))
throw Exception(
"Column " + column_to_sum + " listed in columns to sum does not exist in table declaration.");
"Column " + column_to_sum + " listed in columns to sum does not exist in table declaration.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
}
}
......
......@@ -19,6 +19,11 @@ namespace ProfileEvents
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
namespace
{
......@@ -149,7 +154,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa
DayNum max_month = date_lut.toFirstDayNumOfMonth(DayNum(max_date));
if (min_month != max_month)
throw Exception("Logical error: part spans more than one month.");
throw Exception("Logical error: part spans more than one month.", ErrorCodes::LOGICAL_ERROR);
part_name = new_part_info.getPartNameV0(min_date, max_date);
}
......
......@@ -27,6 +27,8 @@ namespace ErrorCodes
extern const int UNKNOWN_STATUS_OF_INSERT;
extern const int INSERT_WAS_DEDUPLICATED;
extern const int KEEPER_EXCEPTION;
extern const int TIMEOUT_EXCEEDED;
extern const int NO_ACTIVE_REPLICAS;
}
......@@ -389,14 +391,14 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
break;
if (!event->tryWait(quorum_timeout_ms))
throw Exception("Timeout while waiting for quorum");
throw Exception("Timeout while waiting for quorum", ErrorCodes::TIMEOUT_EXCEEDED);
}
/// And what if it is possible that the current replica at this time has ceased to be active and the quorum is marked as failed and deleted?
String value;
if (!zookeeper->tryGet(storage.replica_path + "/is_active", value, nullptr)
|| value != quorum_info.is_active_node_value)
throw Exception("Replica become inactive while waiting for quorum");
throw Exception("Replica become inactive while waiting for quorum", ErrorCodes::NO_ACTIVE_REPLICAS);
}
catch (...)
{
......
......@@ -14,6 +14,11 @@ namespace ProfileEvents
namespace DB
{
namespace ErrorCodes
{
extern const int TABLE_DIFFERS_TOO_MUCH;
}
static const auto PART_CHECK_ERROR_SLEEP_MS = 5 * 1000;
......@@ -213,7 +218,7 @@ void ReplicatedMergeTreePartCheckThread::checkPart(const String & part_name)
auto zk_columns = NamesAndTypesList::parse(
zookeeper->get(storage.replica_path + "/parts/" + part_name + "/columns"));
if (part->columns != zk_columns)
throw Exception("Columns of local part " + part_name + " are different from ZooKeeper");
throw Exception("Columns of local part " + part_name + " are different from ZooKeeper", ErrorCodes::TABLE_DIFFERS_TOO_MUCH);
checkDataPart(
storage.data.getFullPath() + part_name,
......
......@@ -21,6 +21,7 @@ namespace DB
namespace ErrorCodes
{
extern const int CORRUPTED_DATA;
extern const int LOGICAL_ERROR;
extern const int INCORRECT_MARK;
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
}
......@@ -285,7 +286,7 @@ MergeTreeData::DataPart::Checksums checkDataPart(
String file_name = IDataType::getFileNameForStream(name_type.name, substream_path);
auto stream_it = streams.find(file_name);
if (stream_it == streams.end())
throw Exception("Logical error: cannot find stream " + file_name);
throw Exception("Logical error: cannot find stream " + file_name, ErrorCodes::LOGICAL_ERROR);
return &stream_it->second.uncompressed_hashing_buf;
};
......@@ -319,7 +320,7 @@ MergeTreeData::DataPart::Checksums checkDataPart(
String file_name = IDataType::getFileNameForStream(name_type.name, substream_path);
auto stream_it = streams.find(file_name);
if (stream_it == streams.end())
throw Exception("Logical error: cannot find stream " + file_name);
throw Exception("Logical error: cannot find stream " + file_name, ErrorCodes::LOGICAL_ERROR);
stream_it->second.assertEnd();
stream_it->second.saveChecksums(checksums_data);
......
......@@ -19,6 +19,7 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int THERE_IS_NO_COLUMN;
}
......@@ -80,7 +81,7 @@ void StorageDictionary::checkNamesAndTypesCompatibleWithDictionary(const Diction
message += " in dictionary " + dictionary_name + ". ";
message += "There are only columns ";
message += generateNamesAndTypesDescription(dictionary_names_and_types.begin(), dictionary_names_and_types.end());
throw Exception(message);
throw Exception(message, ErrorCodes::THERE_IS_NO_COLUMN);
}
}
}
......
......@@ -1014,7 +1014,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry)
}
else
{
throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)));
throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)), ErrorCodes::LOGICAL_ERROR);
}
if (do_fetch)
......@@ -1862,7 +1862,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
if (!prev.found_new_part_info.isDisjoint(curr.found_new_part_info))
{
throw Exception("Intersected final parts detected: " + prev.found_new_part_name
+ " and " + curr.found_new_part_name + ". It should be investigated.");
+ " and " + curr.found_new_part_name + ". It should be investigated.", ErrorCodes::INCORRECT_DATA);
}
}
}
......
......@@ -15,6 +15,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
NamesAndTypesList StorageSystemZooKeeper::getNamesAndTypes()
{
......@@ -103,7 +108,7 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, const Contex
{
String path = extractPath(query_info.query);
if (path.empty())
throw Exception("SELECT from system.zookeeper table must contain condition like path = 'path' in WHERE clause.");
throw Exception("SELECT from system.zookeeper table must contain condition like path = 'path' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS);
zkutil::ZooKeeperPtr zookeeper = context.getZooKeeper();
......
......@@ -1189,7 +1189,7 @@ void BaseDaemon::handleSignal(int signal_id)
onInterruptSignals(signal_id);
}
else
throw DB::Exception(std::string("Unsupported signal: ") + strsignal(signal_id));
throw DB::Exception(std::string("Unsupported signal: ") + strsignal(signal_id), 0);
}
void BaseDaemon::onInterruptSignals(int signal_id)
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册