diff --git a/src/AggregateFunctions/AggregateFunctionCount.h b/src/AggregateFunctions/AggregateFunctionCount.h index 36a8617ba91..f5d6030a777 100644 --- a/src/AggregateFunctions/AggregateFunctionCount.h +++ b/src/AggregateFunctions/AggregateFunctionCount.h @@ -219,7 +219,7 @@ public: : IAggregateFunctionDataHelper({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"; } diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index b6ba562045d..18edb7c8ce0 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -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; } diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionIf.cpp b/src/AggregateFunctions/Combinators/AggregateFunctionIf.cpp index 20a4bde6bb4..9b5ee79a533 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/Combinators/AggregateFunctionIf.cpp @@ -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, diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionNull.h b/src/AggregateFunctions/Combinators/AggregateFunctionNull.h index 6b6580bf4c4..c8574e82be5 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionNull.h +++ b/src/AggregateFunctions/Combinators/AggregateFunctionNull.h @@ -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, diff --git a/src/Client/ConnectionEstablisher.h b/src/Client/ConnectionEstablisher.h index 1fa08d435e9..a3a01e63246 100644 --- a/src/Client/ConnectionEstablisher.h +++ b/src/Client/ConnectionEstablisher.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index c7d7d0143c8..8433c8afe9f 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -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 diff --git a/src/Client/PacketReceiver.h b/src/Client/PacketReceiver.h index deedf5cccdc..6b3da659290 100644 --- a/src/Client/PacketReceiver.h +++ b/src/Client/PacketReceiver.h @@ -5,7 +5,6 @@ #include #include -#include #include #include #include diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 6f60ec0e642..b3376b35b2e 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -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) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 93638371b84..ddf5fc696fb 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -1,7 +1,5 @@ #include #include -#include -#include #include #include #include @@ -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 diff --git a/src/Columns/getLeastSuperColumn.cpp b/src/Columns/getLeastSuperColumn.cpp index 6ec5ca7a9c1..4f4a5f2b9b8 100644 --- a/src/Columns/getLeastSuperColumn.cpp +++ b/src/Columns/getLeastSuperColumn.cpp @@ -21,7 +21,7 @@ static bool sameConstants(const IColumn & a, const IColumn & b) ColumnWithTypeAndName getLeastSuperColumn(const std::vector & 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]; diff --git a/src/Common/Fiber.h b/src/Common/Fiber.h index f48ace149f4..8b88bd323ef 100644 --- a/src/Common/Fiber.h +++ b/src/Common/Fiber.h @@ -17,7 +17,7 @@ private: template friend class FiberLocal; public: - template< typename StackAlloc, typename Fn> + template Fiber(StackAlloc && salloc, Fn && fn) : impl(std::allocator_arg_t(), std::forward(salloc), RoutineImpl(std::forward(fn))) { } @@ -46,6 +46,12 @@ public: current_fiber = parent_fiber; } + static FiberPtr & getCurrentFiber() + { + thread_local static FiberPtr current_fiber; + return current_fiber; + } + private: template struct RoutineImpl @@ -74,12 +80,6 @@ private: Fn fn; }; - static FiberPtr & getCurrentFiber() - { - thread_local static FiberPtr current_fiber; - return current_fiber; - } - /// Special wrapper to store data in uniquer_ptr. struct DataWrapper { @@ -146,4 +146,3 @@ private: T main_instance; }; - diff --git a/src/Common/SensitiveDataMasker.cpp b/src/Common/SensitiveDataMasker.cpp index 70346919f65..28eae6f451d 100644 --- a/src/Common/SensitiveDataMasker.cpp +++ b/src/Common/SensitiveDataMasker.cpp @@ -91,7 +91,7 @@ void SensitiveDataMasker::setInstance(std::unique_ptr&& 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) { diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index 5f27fdaa4b6..729fb76a573 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -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; diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 7e683ae91de..436b85ff30b 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -448,9 +448,6 @@ toStringEveryLineImpl([[maybe_unused]] bool fatal, const StackTraceRefTriple & s DB::writePointerHex(frame.physical_addr, out); } - if (frame.object.has_value()) - out << " in " << *frame.object; - callback(out.str()); }; #else diff --git a/src/Common/checkStackSize.cpp b/src/Common/checkStackSize.cpp index 8c2a0aaed7f..c88554ca8fe 100644 --- a/src/Common/checkStackSize.cpp +++ b/src/Common/checkStackSize.cpp @@ -1,8 +1,8 @@ +#include +#include /// THREAD_SANITIZER #include #include -#include -#include -#include /// THREAD_SANITIZER +#include #include #include #include @@ -114,6 +114,10 @@ __attribute__((__weak__)) void checkStackSize() { using namespace DB; + /// Not implemented for coroutines. + if (Fiber::getCurrentFiber()) + return; + if (!stack_address) max_stack_size = getStackSize(&stack_address); @@ -136,7 +140,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(max_stack_size * STACK_SIZE_FREE_RATIO); diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index edd6743949d..a5e062a5aa6 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -1614,7 +1614,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()); diff --git a/src/Core/MySQL/PacketEndpoint.cpp b/src/Core/MySQL/PacketEndpoint.cpp index 97b5d3b4d11..085d7595167 100644 --- a/src/Core/MySQL/PacketEndpoint.cpp +++ b/src/Core/MySQL/PacketEndpoint.cpp @@ -40,7 +40,7 @@ bool PacketEndpoint::tryReceivePacket(IMySQLReadPacket & packet, UInt64 millisec ReadBufferFromPocoSocket * socket_in = typeid_cast(in); if (!socket_in) - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: Attempt to pull the duration in a non socket stream"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to pull the duration in a non socket stream"); if (!socket_in->poll(millisecond * 1000)) return false; diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index 7dc036cafa4..14a3c6a4248 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -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); diff --git a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp index aa3b154e49b..ee9870eb0ef 100644 --- a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp +++ b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp @@ -141,7 +141,7 @@ static std::pair 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 diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 2656835f912..20db8036942 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -779,7 +779,7 @@ static void writeFieldsToColumn( casted_int32_column->insertValue(num & 0x800000 ? num | 0xFF000000 : num); } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: it is a bug."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "MaterializedMySQL is a bug."); } } } @@ -844,7 +844,7 @@ static inline bool differenceSortingKeys(const Tuple & row_old_data, const Tuple static inline size_t onUpdateData(const Row & rows_data, Block & buffer, size_t version, const std::vector & sorting_columns_index) { if (rows_data.size() % 2 != 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: It is a bug."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "MaterializedMySQL is a bug."); size_t prev_bytes = buffer.bytes(); std::vector writeable_rows_mask(rows_data.size()); diff --git a/src/Functions/EmptyImpl.h b/src/Functions/EmptyImpl.h index 52484524e6a..d3b2dda024b 100644 --- a/src/Functions/EmptyImpl.h +++ b/src/Functions/EmptyImpl.h @@ -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 & res) diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index d04f76d051a..3be675f39b3 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -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>; @@ -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) { diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 4089a5b542b..62148fa8022 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2560,7 +2560,7 @@ public: if constexpr (std::is_same_v) res = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, false)); else if constexpr (std::is_same_v) - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: It is a bug."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "MaterializedMySQL is a bug."); else if constexpr (to_decimal) { UInt64 scale = extractToDecimalScale(arguments[1]); diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index d0795941e1f..7e7ae76d6eb 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -531,7 +531,7 @@ DataTypePtr FunctionAnyArityLogical::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()); } diff --git a/src/Functions/trap.cpp b/src/Functions/trap.cpp index 99430f039a4..6ce696fedb5 100644 --- a/src/Functions/trap.cpp +++ b/src/Functions/trap.cpp @@ -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"); diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 331cd991ea1..50fab486568 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -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])) diff --git a/src/Interpreters/ArrayJoinedColumnsVisitor.h b/src/Interpreters/ArrayJoinedColumnsVisitor.h index 3bbd6982213..f16751c4561 100644 --- a/src/Interpreters/ArrayJoinedColumnsVisitor.h +++ b/src/Interpreters/ArrayJoinedColumnsVisitor.h @@ -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 out; out.reserve(array_join_expression_list->children.size()); diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index 347ec115aba..e4778edeb9c 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -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(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); diff --git a/src/Interpreters/CrossToInnerJoinVisitor.cpp b/src/Interpreters/CrossToInnerJoinVisitor.cpp index 42af164f4ad..e3e8b80e437 100644 --- a/src/Interpreters/CrossToInnerJoinVisitor.cpp +++ b/src/Interpreters/CrossToInnerJoinVisitor.cpp @@ -173,7 +173,7 @@ std::vector getTables(const ASTSelectQuery & select) { const auto * table_element = child->as(); 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) diff --git a/src/Interpreters/DatabaseAndTableWithAlias.cpp b/src/Interpreters/DatabaseAndTableWithAlias.cpp index db020cb9166..329391b45d7 100644 --- a/src/Interpreters/DatabaseAndTableWithAlias.cpp +++ b/src/Interpreters/DatabaseAndTableWithAlias.cpp @@ -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 diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 33dc178ca00..73487a0914a 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -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 diff --git a/src/Interpreters/InJoinSubqueriesPreprocessor.cpp b/src/Interpreters/InJoinSubqueriesPreprocessor.cpp index 3858830a43b..ec4241a2740 100644 --- a/src/Interpreters/InJoinSubqueriesPreprocessor.cpp +++ b/src/Interpreters/InJoinSubqueriesPreprocessor.cpp @@ -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) { diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 16bc4b1fe2e..cc1d7dd6531 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -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, diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index bf2d1eb79cd..6251a9604e1 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -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(); @@ -183,7 +183,7 @@ private: transformer->as()) 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"); } } } diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 901c82029ee..d5fb0208d45 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -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(); } diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 0fdc9347ee9..107b435ded4 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -337,7 +337,7 @@ static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys) WhichDataType which(type); if (which.isNullable()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: MySQL primary key must be not null, it is a bug."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "MySQL's primary key must be not null, it is a bug."); if (which.isDate() || which.isDate32() || which.isDateTime() || which.isDateTime64()) { diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 5b3b87114ae..3bd7b2d4206 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -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(); } diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 84260faafd4..8f11754b3be 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -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(); diff --git a/src/Interpreters/SetVariants.cpp b/src/Interpreters/SetVariants.cpp index cd9148a01cf..0fb2e5189d4 100644 --- a/src/Interpreters/SetVariants.cpp +++ b/src/Interpreters/SetVariants.cpp @@ -146,7 +146,7 @@ typename SetVariantsTemplate::Type SetVariantsTemplate::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 diff --git a/src/Interpreters/TablesStatus.cpp b/src/Interpreters/TablesStatus.cpp index 005a4515c3a..911a028f813 100644 --- a/src/Interpreters/TablesStatus.cpp +++ b/src/Interpreters/TablesStatus.cpp @@ -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) diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index 130ce2194fd..3de7e217e53 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -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); diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index 00d36750cc1..b5c3e00e299 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -106,7 +106,7 @@ std::optional 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 diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index 8104c6eb81f..21739298036 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -152,8 +152,7 @@ Block getHeaderForProcessingStage( return result; } } - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical Error: unknown processed stage."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown processed stage."); } } - diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 62c480e0f6b..486555ae86d 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -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) diff --git a/src/Parsers/IParser.h b/src/Parsers/IParser.h index d53b58baa7c..198ec0346ff 100644 --- a/src/Parsers/IParser.h +++ b/src/Parsers/IParser.h @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB @@ -73,6 +74,21 @@ public: if (unlikely(max_depth > 0 && depth > max_depth)) throw Exception(ErrorCodes::TOO_DEEP_RECURSION, "Maximum parse depth ({}) exceeded. " "Consider rising max_parser_depth parameter.", max_depth); + + /** Sometimes the maximum parser depth can be set to a high value by the user, + * but we still want to avoid stack overflow. + * For this purpose, we can use the checkStackSize function, but it is too heavy. + * The solution is to check not too frequently. + * The frequency is arbitrary, but not too large, not too small, + * and a power of two to simplify the division. + */ +#if defined(USE_MUSL) || defined(SANITIZER) || !defined(NDEBUG) + static constexpr uint32_t check_frequency = 128; +#else + static constexpr uint32_t check_frequency = 8192; +#endif + if (depth % check_frequency == 0) + checkStackSize(); } ALWAYS_INLINE void decreaseDepth() diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 0ef19a9c14f..6fa94356cd3 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -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 { diff --git a/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp b/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp index a56c24a740a..fcf338577f8 100644 --- a/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp +++ b/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp @@ -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)) { diff --git a/src/Processors/Sources/WaitForAsyncInsertSource.h b/src/Processors/Sources/WaitForAsyncInsertSource.h index 1029c164941..78af6294202 100644 --- a/src/Processors/Sources/WaitForAsyncInsertSource.h +++ b/src/Processors/Sources/WaitForAsyncInsertSource.h @@ -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); diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index cc0b5926e66..eeb8f4a6060 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -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()) { diff --git a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp index 6c7c7447070..8a13973b970 100644 --- a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp +++ b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp @@ -37,7 +37,7 @@ InputFormatPtr getInputFormatFromASTInsertQuery( const auto * ast_insert_query = ast->as(); 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 getReadBufferFromASTInsertQuery(const ASTPtr & ast) { const auto * insert_query = ast->as(); 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) { diff --git a/src/QueryPipeline/ExecutionSpeedLimits.cpp b/src/QueryPipeline/ExecutionSpeedLimits.cpp index f8ae4c76d0f..05fd394db77 100644 --- a/src/QueryPipeline/ExecutionSpeedLimits.cpp +++ b/src/QueryPipeline/ExecutionSpeedLimits.cpp @@ -113,7 +113,7 @@ static bool handleOverflowMode(OverflowMode mode, int code, FormatStringHelper #include #include -#include #include #include #include diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index e9157266901..ddea469d9d4 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -165,7 +165,7 @@ HTTPRequestHandlerFactoryPtr createHandlerFactory(IServer & server, const Poco:: return createPrometheusMainHandlerFactory(server, config, metrics_writer, name); } - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: Unknown HTTP handler factory name."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown HTTP handler factory name."); } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 833f8ecc818..9464ef74586 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -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); diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index ce70fbe18e5..168c5f729ce 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -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/" : ""); diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 1ffb5177430..cbdeabffa97 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -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 @@ -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(*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(path.c_str() + prefix_size, path.size() - prefix_size); locks.push_back(LockInfo{path, partitions[i], number}); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index babc593ff62..fec320e9483 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -870,7 +870,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; @@ -897,7 +897,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; @@ -926,12 +926,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; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 58fddde7b54..1bf1d4a3c29 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -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()) { diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index c9c16b59f9e..ebf887f5e9e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -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); } diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index da49814b83a..f506230b5ea 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -59,7 +59,7 @@ bool maybeTrueOnBloomFilter(const IColumn * hash_column, const BloomFilterPtr & const auto * non_const_column = typeid_cast(hash_column); if (!const_column && !non_const_column) - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: hash column must be Const Column or UInt64 Column."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Hash column must be Const or UInt64."); if (const_column) { diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h index db85c804d8d..8029d6d405b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h @@ -53,7 +53,7 @@ public: if (const auto & bf_granule = typeid_cast(granule.get())) return mayBeTrueOnGranule(bf_granule); - throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: require bloom filter index granule."); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Requires bloom filter index granule."); } private: diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp index 4e339964de3..da89d52a9ff 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp @@ -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"); } } diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index b9e082c0b22..b122674466f 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -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(map); })) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unknown JOIN strictness"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness"); return chunk; } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 9fbeac5e4f3..7459760b0f5 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -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(); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ee15a26f244..e805101dfcd 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2050,7 +2050,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); @@ -2113,7 +2113,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, @@ -6800,7 +6800,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. @@ -7176,7 +7176,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); diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 181fd0ac61c..5679effbcb2 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -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(); 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(); } @@ -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 = {}; diff --git a/src/Storages/System/StorageSystemStackTrace.cpp b/src/Storages/System/StorageSystemStackTrace.cpp index c9758004a4d..74864bb50e1 100644 --- a/src/Storages/System/StorageSystemStackTrace.cpp +++ b/src/Storages/System/StorageSystemStackTrace.cpp @@ -167,7 +167,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"); } } diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 4526a38a1c3..afc458ea612 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -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; diff --git a/tests/queries/0_stateless/02985_parser_check_stack_size.reference b/tests/queries/0_stateless/02985_parser_check_stack_size.reference new file mode 100644 index 00000000000..f83e0818db2 --- /dev/null +++ b/tests/queries/0_stateless/02985_parser_check_stack_size.reference @@ -0,0 +1 @@ +TOO_DEEP diff --git a/tests/queries/0_stateless/02985_parser_check_stack_size.sh b/tests/queries/0_stateless/02985_parser_check_stack_size.sh new file mode 100755 index 00000000000..c91a0a3eacc --- /dev/null +++ b/tests/queries/0_stateless/02985_parser_check_stack_size.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query "select 'create table test (x ' || repeat('Array(', 10000) || 'UInt64' || repeat(')', 10000) || ') engine=Memory' format TSVRaw" | $CLICKHOUSE_CURL "${CLICKHOUSE_URL}&max_parser_depth=100000" --data-binary @- | grep -o -F 'TOO_DEEP' diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 6c12970c4bb..a71dac91683 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -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."