Miscellaneous

This commit is contained in:
Alexey Milovidov 2024-02-19 02:58:51 +01:00
parent 6eff309185
commit 6fd563df52
55 changed files with 91 additions and 90 deletions

View File

@ -219,7 +219,7 @@ public:
: IAggregateFunctionDataHelper<AggregateFunctionCountData, AggregateFunctionCountNotNullUnary>({argument}, params, createResultType())
{
if (!argument->isNullable())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: not Nullable data type passed to AggregateFunctionCountNotNullUnary");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not Nullable data type passed to AggregateFunctionCountNotNullUnary");
}
String getName() const override { return "count"; }

View File

@ -100,7 +100,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
{
AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix("Null");
if (!combinator)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: cannot find aggregate function combinator "
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find aggregate function combinator "
"to apply a function to Nullable arguments.");
DataTypes nested_types = combinator->transformArguments(types_without_low_cardinality);
@ -123,7 +123,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
auto with_original_arguments = getImpl(name, action, types_without_low_cardinality, parameters, out_properties, false);
if (!with_original_arguments)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: AggregateFunctionFactory returned nullptr");
throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionFactory returned nullptr");
return with_original_arguments;
}

View File

@ -249,7 +249,7 @@ public:
: Base(std::move(nested_function_), arguments, params), number_of_arguments(arguments.size())
{
if (number_of_arguments == 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: single argument is passed to AggregateFunctionIfNullVariadic");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Single argument is passed to AggregateFunctionIfNullVariadic");
if (number_of_arguments > MAX_ARGS)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,

View File

@ -429,7 +429,7 @@ public:
, number_of_arguments(arguments.size())
{
if (number_of_arguments == 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: single argument is passed to AggregateFunctionNullVariadic");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Single argument is passed to AggregateFunctionNullVariadic");
if (number_of_arguments > MAX_ARGS)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,

View File

@ -320,7 +320,7 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac
ReplicaState & state = getReplicaForReading();
current_connection = state.connection;
if (current_connection == nullptr)
throw Exception(ErrorCodes::NO_AVAILABLE_REPLICA, "Logical error: no available replica");
throw Exception(ErrorCodes::NO_AVAILABLE_REPLICA, "No available replica");
Packet packet;
try

View File

@ -810,7 +810,7 @@ ColumnPtr ColumnArray::filterTuple(const Filter & filt, ssize_t result_size_hint
size_t tuple_size = tuple.tupleSize();
if (tuple_size == 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty tuple");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty tuple");
Columns temporary_arrays(tuple_size);
for (size_t i = 0; i < tuple_size; ++i)
@ -1263,7 +1263,7 @@ ColumnPtr ColumnArray::replicateTuple(const Offsets & replicate_offsets) const
size_t tuple_size = tuple.tupleSize();
if (tuple_size == 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty tuple");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty tuple");
Columns temporary_arrays(tuple_size);
for (size_t i = 0; i < tuple_size; ++i)

View File

@ -1,7 +1,5 @@
#include <Common/Arena.h>
#include <Common/SipHash.h>
#include <Common/NaNUtils.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <Common/WeakHash.h>
#include <Columns/ColumnDecimal.h>
@ -28,7 +26,6 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_COLUMN;
extern const int SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT;
extern const int NOT_IMPLEMENTED;
}
@ -829,8 +826,7 @@ void ColumnNullable::applyNullMap(const ColumnNullable & other)
void ColumnNullable::checkConsistency() const
{
if (null_map->size() != getNestedColumn().size())
throw Exception(ErrorCodes::SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT,
"Logical error: Sizes of nested column and null map of Nullable column are not equal");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Sizes of nested column and null map of Nullable column are not equal");
}
ColumnPtr ColumnNullable::createWithOffsets(const IColumn::Offsets & offsets, const ColumnConst & column_with_default_value, size_t total_rows, size_t shift) const

View File

@ -21,7 +21,7 @@ static bool sameConstants(const IColumn & a, const IColumn & b)
ColumnWithTypeAndName getLeastSuperColumn(const std::vector<const ColumnWithTypeAndName *> & columns)
{
if (columns.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no src columns for supercolumn");
throw Exception(ErrorCodes::LOGICAL_ERROR, "No src columns for supercolumn");
ColumnWithTypeAndName result = *columns[0];

View File

@ -91,7 +91,7 @@ void SensitiveDataMasker::setInstance(std::unique_ptr<SensitiveDataMasker>&& sen
{
if (!sensitive_data_masker_)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: the 'sensitive_data_masker' is not set");
throw Exception(ErrorCodes::LOGICAL_ERROR, "The 'sensitive_data_masker' is not set");
if (sensitive_data_masker_->rulesCount() > 0)
{

View File

@ -209,7 +209,7 @@ public:
{
if (!is_reference_128)
throw DB::Exception(
DB::ErrorCodes::LOGICAL_ERROR, "Logical error: can't call get128Reference when is_reference_128 is not set");
DB::ErrorCodes::LOGICAL_ERROR, "Can't call get128Reference when is_reference_128 is not set");
finalize();
const auto lo = v0 ^ v1 ^ v2 ^ v3;
v1 ^= 0xdd;

View File

@ -136,7 +136,7 @@ __attribute__((__weak__)) void checkStackSize()
/// We assume that stack grows towards lower addresses. And that it starts to grow from the end of a chunk of memory of max_stack_size.
if (int_frame_address > int_stack_address + max_stack_size)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: frame address is greater than stack begin address");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Frame address is greater than stack begin address");
size_t stack_size = int_stack_address + max_stack_size - int_frame_address;
size_t max_stack_size_allowed = static_cast<size_t>(max_stack_size * STACK_SIZE_FREE_RATIO);

View File

@ -1583,7 +1583,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc
{
auto path_prefix = request.path;
if (path_prefix.empty())
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: path cannot be empty");
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Path cannot be empty");
const auto & children = node_it->value.getChildren();
response.names.reserve(children.size());

View File

@ -239,7 +239,7 @@ static DataTypePtr create(const ASTPtr & arguments)
argument_types.push_back(DataTypeFactory::instance().get(arguments->children[i]));
if (function_name.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty name of aggregate function passed");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty name of aggregate function passed");
AggregateFunctionProperties properties;
AggregateFunctionPtr function = AggregateFunctionFactory::instance().get(function_name, action, argument_types, params_row, properties);

View File

@ -141,7 +141,7 @@ static std::pair<DataTypePtr, DataTypeCustomDescPtr> create(const ASTPtr & argum
argument_types.push_back(DataTypeFactory::instance().get(arguments->children[i]));
if (function_name.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty name of aggregate function passed");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty name of aggregate function passed");
AggregateFunctionProperties properties;
/// NullsAction is not part of the type definition, instead it will have transformed the function into a different one

View File

@ -35,7 +35,7 @@ struct EmptyImpl
/// Only make sense if is_fixed_to_constant.
static void vectorFixedToConstant(const ColumnString::Chars & /*data*/, size_t /*n*/, UInt8 & /*res*/)
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: 'vectorFixedToConstant method' is called");
throw Exception(ErrorCodes::LOGICAL_ERROR, "'vectorFixedToConstant method' is called");
}
static void vectorFixedToVector(const ColumnString::Chars & data, size_t n, PaddedPODArray<UInt8> & res)

View File

@ -811,7 +811,7 @@ private:
c0_const_size = c0_const_fixed_string->getN();
}
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Logical error: ColumnConst contains not String nor FixedString column");
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnConst contains not String nor FixedString column");
}
if (c1_const)
@ -830,7 +830,7 @@ private:
c1_const_size = c1_const_fixed_string->getN();
}
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Logical error: ColumnConst contains not String nor FixedString column");
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnConst contains not String nor FixedString column");
}
using StringImpl = StringComparisonImpl<Op<int, int>>;
@ -1114,7 +1114,7 @@ private:
/// This is a paranoid check to protect from a broken query analysis.
if (c0->isNullable() != c1->isNullable())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Logical error: columns are assumed to be of identical types, but they are different in Nullable");
"Columns are assumed to be of identical types, but they are different in Nullable");
if (c0_const && c1_const)
{

View File

@ -530,7 +530,7 @@ DataTypePtr FunctionAnyArityLogical<Impl, Name>::getReturnTypeImpl(const DataTyp
{
has_nullable_arguments = arg_type->isNullable();
if (has_nullable_arguments && !Impl::specialImplementationForNulls())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: Unexpected type of argument for function \"{}\": "
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected type of argument for function \"{}\": "
" argument {} is of type {}", getName(), i + 1, arg_type->getName());
}

View File

@ -177,7 +177,7 @@ public:
}
else if (mode == "logical error")
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: trap");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trap");
}
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown trap mode");

View File

@ -624,7 +624,7 @@ Aggregator::Aggregator(const Block & header_, const Params & params_)
{
size_t alignment_of_next_state = params.aggregates[i + 1].function->alignOfData();
if ((alignment_of_next_state & (alignment_of_next_state - 1)) != 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: alignOfData is not 2^N");
throw Exception(ErrorCodes::LOGICAL_ERROR, "`alignOfData` is not 2^N");
/// Extend total_size to next alignment requirement
/// Add padding by rounding up 'total_size_of_aggregate_states' to be a multiplier of alignment_of_next_state.
@ -857,7 +857,7 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod()
return AggregatedDataVariants::Type::low_cardinality_keys128;
if (size_of_field == 32)
return AggregatedDataVariants::Type::low_cardinality_keys256;
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: low cardinality numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
throw Exception(ErrorCodes::LOGICAL_ERROR, "LowCardinality numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
}
if (size_of_field == 1)
@ -872,7 +872,7 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod()
return AggregatedDataVariants::Type::keys128;
if (size_of_field == 32)
return AggregatedDataVariants::Type::keys256;
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
}
if (params.keys_size == 1 && isFixedString(types_removed_nullable[0]))

View File

@ -62,7 +62,7 @@ private:
{
auto [array_join_expression_list, _] = node.arrayJoinExpressionList();
if (!array_join_expression_list)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no ARRAY JOIN");
throw Exception(ErrorCodes::LOGICAL_ERROR, "No ARRAY JOIN");
std::vector<ASTPtr *> out;
out.reserve(array_join_expression_list->children.size());

View File

@ -23,7 +23,7 @@ namespace ErrorCodes
void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const
{
if (server_protocol_revision < DBMS_MIN_REVISION_WITH_CLIENT_INFO)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: method ClientInfo::write is called for unsupported server revision");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Method ClientInfo::write is called for unsupported server revision");
writeBinary(static_cast<UInt8>(query_kind), out);
if (empty())
@ -103,7 +103,7 @@ void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const
void ClientInfo::read(ReadBuffer & in, UInt64 client_protocol_revision)
{
if (client_protocol_revision < DBMS_MIN_REVISION_WITH_CLIENT_INFO)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: method ClientInfo::read is called for unsupported client revision");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Method ClientInfo::read is called for unsupported client revision");
UInt8 read_query_kind = 0;
readBinary(read_query_kind, in);

View File

@ -173,7 +173,7 @@ std::vector<JoinedElement> getTables(const ASTSelectQuery & select)
{
const auto * table_element = child->as<ASTTablesInSelectQueryElement>();
if (!table_element)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: TablesInSelectQueryElement expected");
throw Exception(ErrorCodes::LOGICAL_ERROR, "TablesInSelectQueryElement expected");
JoinedElement & t = joined_tables.emplace_back(*table_element);
t.rewriteCommaToCross();
@ -224,7 +224,7 @@ void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & da
{
if (joined_tables.size() != data.tables_with_columns.size())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Logical error: inconsistent number of tables: {} != {}",
"Inconsistent number of tables: {} != {}",
joined_tables.size(), data.tables_with_columns.size());
for (size_t i = 0; i < joined_tables.size(); ++i)

View File

@ -71,7 +71,7 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableExpression &
alias = table_expression.subquery->tryGetAlias();
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no known elements in ASTTableExpression");
throw Exception(ErrorCodes::LOGICAL_ERROR, "No known elements in ASTTableExpression");
}
bool DatabaseAndTableWithAlias::satisfies(const DatabaseAndTableWithAlias & db_table, bool table_may_be_an_alias) const

View File

@ -368,7 +368,7 @@ HashJoin::Type HashJoin::chooseMethod(JoinKind kind, const ColumnRawPtrs & key_c
return Type::keys128;
if (size_of_field == 32)
return Type::keys256;
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
}
/// If the keys fit in N bits, we will use a hash table for N-bit-packed keys

View File

@ -103,12 +103,12 @@ private:
/// Already processed.
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unexpected function name {}", concrete->name);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function name {}", concrete->name);
}
else if (table_join)
table_join->locality = JoinLocality::Global;
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unexpected AST node");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected AST node");
}
else if (distributed_product_mode == DistributedProductMode::DENY)
{

View File

@ -56,7 +56,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(
size_t num_children = ast->list_of_selects->children.size();
if (!num_children)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no children in ASTSelectWithUnionQuery");
throw Exception(ErrorCodes::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,

View File

@ -168,7 +168,7 @@ private:
has_asterisks = true;
if (!qualified_asterisk->qualifier)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: qualified asterisk must have a qualifier");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Qualified asterisk must have a qualifier");
auto & identifier = qualified_asterisk->qualifier->as<ASTIdentifier &>();
@ -183,7 +183,7 @@ private:
transformer->as<ASTColumnsReplaceTransformer>())
IASTColumnsTransformer::transform(transformer, columns);
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: qualified asterisk must only have children of IASTColumnsTransformer type");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Qualified asterisk must only have children of IASTColumnsTransformer type");
}
}
}

View File

@ -239,7 +239,7 @@ public:
/// SortCursorImpl can work with permutation, but MergeJoinCursor can't.
if (impl.permutation)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: MergeJoinCursor doesn't support permutation");
throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeJoinCursor doesn't support permutation");
}
size_t position() const { return impl.getRow(); }

View File

@ -295,7 +295,7 @@ ProcessListEntry::~ProcessListEntry()
auto user_process_list_it = parent.user_to_queries.find(user);
if (user_process_list_it == parent.user_to_queries.end())
{
LOG_ERROR(getLogger("ProcessList"), "Logical error: cannot find user in ProcessList");
LOG_ERROR(getLogger("ProcessList"), "Cannot find user in ProcessList");
std::terminate();
}
@ -323,7 +323,7 @@ ProcessListEntry::~ProcessListEntry()
if (!found)
{
LOG_ERROR(getLogger("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser");
LOG_ERROR(getLogger("ProcessList"), "Cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser");
std::terminate();
}

View File

@ -275,7 +275,7 @@ void Set::appendSetElements(SetKeyColumns & holder)
void Set::checkIsCreated() const
{
if (!is_created.load())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: Trying to use set before it has been built.");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to use set before it has been built.");
}
ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) const
@ -283,7 +283,7 @@ ColumnPtr Set::execute(const ColumnsWithTypeAndName & columns, bool negative) co
size_t num_key_columns = columns.size();
if (0 == num_key_columns)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no columns passed to Set::execute method.");
throw Exception(ErrorCodes::LOGICAL_ERROR, "No columns passed to Set::execute method.");
auto res = ColumnUInt8::create();
ColumnUInt8::Container & vec_res = res->getData();

View File

@ -146,7 +146,7 @@ typename SetVariantsTemplate<Variant>::Type SetVariantsTemplate<Variant>::choose
return Type::keys128;
if (size_of_field == 32)
return Type::keys256;
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
}
/// If the keys fit in N bits, we will use a hash table for N-bit-packed keys

View File

@ -35,7 +35,7 @@ void TableStatus::read(ReadBuffer & in)
void TablesStatusRequest::write(WriteBuffer & out, UInt64 server_protocol_revision) const
{
if (server_protocol_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: method TablesStatusRequest::write is called for unsupported server revision");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Method TablesStatusRequest::write is called for unsupported server revision");
writeVarUInt(tables.size(), out);
for (const auto & table_name : tables)

View File

@ -158,7 +158,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTFunction & node, const ASTPtr &, D
void TranslateQualifiedNamesMatcher::visit(const ASTQualifiedAsterisk & node, const ASTPtr &, Data & data)
{
if (!node.qualifier)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: qualified asterisk must have a qualifier");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Qualified asterisk must have a qualifier");
/// @note it could contain table alias as table name.
DatabaseAndTableWithAlias db_and_table(node.qualifier);

View File

@ -106,7 +106,7 @@ std::optional<EvaluateConstantExpressionResult> evaluateConstantExpressionImpl(c
if (result_column->empty())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Logical error: empty result column after evaluation "
"Empty result column after evaluation "
"of constant expression for IN, VALUES, or LIMIT, or aggregate function parameter, or a table function argument");
/// Expressions like rand() or now() are not constant

View File

@ -934,7 +934,7 @@ bool ParserNumber::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
if (float_value < 0)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Logical error: token number cannot begin with minus, "
"Token number cannot begin with minus, "
"but parsed float number is less than zero.");
if (negative)

View File

@ -179,7 +179,7 @@ void JSONEachRowRowInputFormat::readJSONObject(MutableColumns & columns)
else if (column_index == NESTED_FIELD)
readNestedData(name_ref.toString(), columns);
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: illegal value of column_index");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal value of column_index");
}
else
{

View File

@ -136,7 +136,7 @@ bool RowInputFormatWithDiagnosticInfo::deserializeFieldAndPrintDiagnosticInfo(co
auto * curr_position = in->position();
if (curr_position < prev_position)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: parsing is non-deterministic.");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Parsing is non-deterministic.");
if (isNativeNumber(type) || isDate(type) || isDateTime(type) || isDateTime64(type))
{

View File

@ -33,7 +33,7 @@ protected:
{
auto status = insert_future.wait_for(std::chrono::milliseconds(timeout_ms));
if (status == std::future_status::deferred)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: got future in deferred state");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Got future in deferred state");
if (status == std::future_status::timeout)
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Wait for async insert timeout ({} ms) exceeded)", timeout_ms);

View File

@ -163,7 +163,7 @@ void CreatingSetsTransform::startSubquery()
done_with_table = !external_table;
if ((done_with_set && !set_from_cache) && done_with_table)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: nothing to do with subquery");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Nothing to do with subquery");
if (table_out.initialized())
{

View File

@ -37,7 +37,7 @@ InputFormatPtr getInputFormatFromASTInsertQuery(
const auto * ast_insert_query = ast->as<ASTInsertQuery>();
if (!ast_insert_query)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: query requires data to insert, but it is not INSERT query");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Query requires data to insert, but it is not INSERT query");
if (ast_insert_query->infile && context->getApplicationType() == Context::ApplicationType::SERVER)
throw Exception(ErrorCodes::UNKNOWN_TYPE_OF_QUERY, "Query has infile and was send directly to server");
@ -47,7 +47,7 @@ InputFormatPtr getInputFormatFromASTInsertQuery(
if (input_function)
throw Exception(ErrorCodes::INVALID_USAGE_OF_INPUT, "FORMAT must be specified for function input()");
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: INSERT query requires format to be set");
throw Exception(ErrorCodes::LOGICAL_ERROR, "INSERT query requires format to be set");
}
/// Data could be in parsed (ast_insert_query.data) and in not parsed yet (input_buffer_tail_part) part of query.
@ -105,7 +105,7 @@ std::unique_ptr<ReadBuffer> getReadBufferFromASTInsertQuery(const ASTPtr & ast)
{
const auto * insert_query = ast->as<ASTInsertQuery>();
if (!insert_query)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: query requires data to insert, but it is not INSERT query");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Query requires data to insert, but it is not INSERT query");
if (insert_query->infile)
{

View File

@ -113,7 +113,7 @@ static bool handleOverflowMode(OverflowMode mode, int code, FormatStringHelper<A
ProfileEvents::increment(ProfileEvents::OverflowBreak);
return false;
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unknown overflow mode");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown overflow mode");
}
}

View File

@ -943,7 +943,7 @@ void TCPHandler::processInsertQuery()
auto wait_status = result.future.wait_for(std::chrono::milliseconds(timeout_ms));
if (wait_status == std::future_status::deferred)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: got future in deferred state");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Got future in deferred state");
if (wait_status == std::future_status::timeout)
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Wait for async insert timeout ({} ms) exceeded)", timeout_ms);

View File

@ -903,7 +903,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk(
|| part_name.empty()
|| std::string::npos != tmp_prefix.find_first_of("/.")
|| std::string::npos != part_name.find_first_of("/."))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: tmp_prefix and part_name cannot be empty or contain '.' or '/' characters.");
throw Exception(ErrorCodes::LOGICAL_ERROR, "`tmp_prefix` and `part_name` cannot be empty or contain '.' or '/' characters.");
auto part_dir = tmp_prefix + part_name;
auto part_relative_path = data.getRelativeDataPath() + String(to_detached ? "detached/" : "");

View File

@ -17,7 +17,7 @@ EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_,
: zookeeper(zookeeper_), path_prefix(path_prefix_), path(path_), conflict_path(conflict_path_)
{
if (conflict_path.empty() && path.size() <= path_prefix.size())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: name of the main node is shorter than prefix.");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Name of the main node is shorter than prefix.");
}
template <typename T>
@ -179,7 +179,7 @@ EphemeralLocksInAllPartitions::EphemeralLocksInAllPartitions(
size_t prefix_size = block_numbers_path.size() + 1 + partitions[i].size() + 1 + path_prefix.size();
const String & path = dynamic_cast<const Coordination::CreateResponse &>(*lock_responses[i]).path_created;
if (path.size() <= prefix_size)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: name of the sequential node is shorter than prefix.");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Name of the sequential node is shorter than prefix.");
UInt64 number = parse<UInt64>(path.c_str() + prefix_size, path.size() - prefix_size);
locks.push_back(LockInfo{path, partitions[i], number});

View File

@ -869,7 +869,7 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat
if (is_optional)
return;
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: Sign column for storage {} is empty", storage);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Sign column for storage {} is empty", storage);
}
bool miss_column = true;
@ -896,7 +896,7 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat
if (is_optional)
return;
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: Version column for storage {} is empty", storage);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Version column for storage {} is empty", storage);
}
bool miss_column = true;
@ -925,12 +925,12 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat
if (is_optional)
return;
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: is_deleted ({}) column for storage {} is empty", is_deleted_column, storage);
throw Exception(ErrorCodes::LOGICAL_ERROR, "`is_deleted` ({}) column for storage {} is empty", is_deleted_column, storage);
}
else
{
if (version_column.empty() && !is_optional)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: Version column ({}) for storage {} is empty while is_deleted ({}) is not.",
throw Exception(ErrorCodes::LOGICAL_ERROR, "Version column ({}) for storage {} is empty while is_deleted ({}) is not.",
version_column, storage, is_deleted_column);
bool miss_is_deleted_column = true;

View File

@ -85,7 +85,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartsSizeForMerge(size_t max_coun
if (scheduled_tasks_count > max_count)
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Logical error: invalid argument passed to getMaxSourcePartsSize: scheduled_tasks_count = {} > max_count = {}",
"Invalid argument passed to getMaxSourcePartsSize: scheduled_tasks_count = {} > max_count = {}",
scheduled_tasks_count, max_count);
}
@ -511,7 +511,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges(
/// Do not allow to "merge" part with itself for regular merges, unless it is a TTL-merge where it is ok to remove some values with expired ttl
if (parts_to_merge.size() == 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: merge selector returned only one part to merge");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Merge selector returned only one part to merge");
if (parts_to_merge.empty())
{

View File

@ -384,13 +384,13 @@ Block MergeTreeDataWriter::mergeBlock(
/// Check that after first merge merging_algorithm is waiting for data from input 0.
if (status.required_source != 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: required source after the first merge is not 0. Chunk rows: {}, is_finished: {}, required_source: {}, algorithm: {}", status.chunk.getNumRows(), status.is_finished, status.required_source, merging_algorithm->getName());
throw Exception(ErrorCodes::LOGICAL_ERROR, "Required source after the first merge is not 0. Chunk rows: {}, is_finished: {}, required_source: {}, algorithm: {}", status.chunk.getNumRows(), status.is_finished, status.required_source, merging_algorithm->getName());
status = merging_algorithm->merge();
/// Check that merge is finished.
if (!status.is_finished)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: merge is not finished after the second merge.");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Merge is not finished after the second merge.");
/// Merged Block is sorted and we don't need to use permutation anymore
permutation = nullptr;
@ -439,7 +439,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl(
auto max_month = date_lut.toNumYYYYMM(max_date);
if (min_month != max_month)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: part spans more than one month.");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Part spans more than one month.");
part_name = new_part_info.getPartNameV0(min_date, max_date);
}

View File

@ -54,9 +54,9 @@ MarkType::MarkType(bool adaptive_, bool compressed_, MergeTreeDataPartType::Valu
: adaptive(adaptive_), compressed(compressed_), part_type(part_type_)
{
if (!adaptive && part_type != MergeTreeDataPartType::Wide)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: non-Wide data part type with non-adaptive granularity");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Non-Wide data part type with non-adaptive granularity");
if (part_type == MergeTreeDataPartType::Unknown)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unknown data part type");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown data part type");
}
bool MarkType::isMarkFileExtension(std::string_view extension)
@ -71,7 +71,7 @@ std::string MarkType::getFileExtension() const
if (!adaptive)
{
if (part_type != MergeTreeDataPartType::Wide)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: non-Wide data part type with non-adaptive granularity");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Non-Wide data part type with non-adaptive granularity");
return res;
}
@ -84,7 +84,7 @@ std::string MarkType::getFileExtension() const
case MergeTreeDataPartType::InMemory:
return "";
case MergeTreeDataPartType::Unknown:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unknown data part type");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown data part type");
}
}

View File

@ -500,7 +500,7 @@ protected:
Chunk chunk;
if (!joinDispatch(join->kind, join->strictness, join->data->maps.front(),
[&](auto kind, auto strictness, auto & map) { chunk = createChunk<kind, strictness>(map); }))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unknown JOIN strictness");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness");
return chunk;
}

View File

@ -241,7 +241,7 @@ void LogSource::readData(const NameAndTypePair & name_and_type, ColumnPtr & colu
const auto & data_file_it = storage.data_files_by_names.find(data_file_name);
if (data_file_it == storage.data_files_by_names.end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no information about file {} in StorageLog", data_file_name);
throw Exception(ErrorCodes::LOGICAL_ERROR, "No information about file {} in StorageLog", data_file_name);
const auto & data_file = *data_file_it->second;
size_t offset = stream_for_prefix ? 0 : offsets[data_file.index];
@ -448,7 +448,7 @@ ISerialization::OutputStreamGetter LogSink::createStreamGetter(const NameAndType
String data_file_name = ISerialization::getFileNameForStream(name_and_type, path);
auto it = streams.find(data_file_name);
if (it == streams.end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: stream was not created when writing data in LogSink");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Stream was not created when writing data in LogSink");
Stream & stream = it->second;
if (stream.written)
@ -473,7 +473,7 @@ void LogSink::writeData(const NameAndTypePair & name_and_type, const IColumn & c
{
const auto & data_file_it = storage.data_files_by_names.find(data_file_name);
if (data_file_it == storage.data_files_by_names.end())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no information about file {} in StorageLog", data_file_name);
throw Exception(ErrorCodes::LOGICAL_ERROR, "No information about file {} in StorageLog", data_file_name);
const auto & data_file = *data_file_it->second;
const auto & columns = metadata_snapshot->getColumns();

View File

@ -2043,7 +2043,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry, bool need_to_che
if (entry.quorum)
{
if (entry.type != LogEntry::GET_PART)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: log entry with quorum but type is not GET_PART");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Log entry with quorum but type is not GET_PART");
LOG_DEBUG(log, "No active replica has part {} which needs to be written with quorum. Will try to mark that quorum as failed.", entry.new_part_name);
@ -2106,7 +2106,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry, bool need_to_che
auto part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version);
if (part_info.min_block != part_info.max_block)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: log entry with quorum for part covering more than one block number");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Log entry with quorum for part covering more than one block number");
ops.emplace_back(zkutil::makeCreateRequest(
fs::path(zookeeper_path) / "quorum" / "failed_parts" / entry.new_part_name,
@ -6796,7 +6796,7 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry(
}
else
{
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unexpected name of log node: {}", entry.znode_name);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected name of log node: {}", entry.znode_name);
}
/** Second - find the corresponding entry in the queue of the specified replica.
@ -7172,7 +7172,7 @@ void StorageReplicatedMergeTree::fetchPartition(
}
if (best_replica.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: cannot choose best replica.");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot choose best replica.");
LOG_INFO(log, "Found {} replicas, {} of them are active. Selected {} to fetch from.", replicas.size(), active_replicas.size(), best_replica);

View File

@ -207,12 +207,12 @@ void StorageView::read(
static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query)
{
if (!select_query.tables() || select_query.tables()->children.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no table expression in view select AST");
throw Exception(ErrorCodes::LOGICAL_ERROR, "No table expression in view select AST");
auto * select_element = select_query.tables()->children[0]->as<ASTTablesInSelectQueryElement>();
if (!select_element->table_expression)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: incorrect table expression");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect table expression");
return select_element->table_expression->as<ASTTableExpression>();
}
@ -243,7 +243,7 @@ void StorageView::replaceWithSubquery(ASTSelectQuery & outer_query, ASTPtr view_
}
if (!table_expression->database_and_table_name)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: incorrect table expression");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect table expression");
}
DatabaseAndTableWithAlias db_table(table_expression->database_and_table_name);
@ -270,7 +270,7 @@ ASTPtr StorageView::restoreViewName(ASTSelectQuery & select_query, const ASTPtr
ASTTableExpression * table_expression = getFirstTableExpression(select_query);
if (!table_expression->subquery)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: incorrect table expression");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Incorrect table expression");
ASTPtr subquery = table_expression->subquery;
table_expression->subquery = {};

View File

@ -168,7 +168,7 @@ bool wait(int timeout_ms)
continue; /// Drain delayed notifications.
}
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: read wrong number of bytes from pipe");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Read wrong number of bytes from pipe");
}
}

View File

@ -145,7 +145,7 @@ bool isCompatible(ASTPtr & node)
return false;
if (!function->arguments)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: function->arguments is not set");
throw Exception(ErrorCodes::LOGICAL_ERROR, "function->arguments is not set");
String name = function->name;

View File

@ -448,3 +448,8 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -P 'Sql|Html|Xml|Cpu|Tcp|Udp|Http|Db|Json|Yaml' | grep -v -P 'RabbitMQ|Azure|Aws|aws|Avro|IO/S3' &&
echo "Abbreviations such as SQL, XML, HTTP, should be in all caps. For example, SQL is right, Sql is wrong. XMLHttpRequest is very wrong."
find $ROOT_PATH/{src,base,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -F -i 'ErrorCodes::LOGICAL_ERROR, "Logical error:' &&
echo "If an exception has LOGICAL_ERROR code, there is no need to include the text 'Logical error' in the exception message, because then the phrase 'Logical error' will be printed twice."