Merge pull request #4973 from yandex/noreturn

Added [[noreturn]] attributes for functions that throw unconditionally
This commit is contained in:
alexey-milovidov 2019-04-11 01:10:55 +03:00 committed by GitHub
commit 7b4fc33ef9
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 12 additions and 34 deletions

View File

@ -271,7 +271,7 @@ private:
void initBlockInput();
void initBlockLogsInput();
void throwUnexpectedPacket(UInt64 packet_type, const char * expected) const;
[[noreturn]] void throwUnexpectedPacket(UInt64 packet_type, const char * expected) const;
};
}

View File

@ -14,7 +14,7 @@ namespace DB
class IDataTypeDummy : public DataTypeWithSimpleSerialization
{
private:
void throwNoSerialization() const
[[noreturn]] void throwNoSerialization() const
{
throw Exception("Serialization is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}

View File

@ -77,7 +77,7 @@ namespace ProtobufColumnMatcher
namespace details
{
void throwNoCommonColumns();
[[noreturn]] void throwNoCommonColumns();
class ColumnNameMatcher
{

View File

@ -385,73 +385,61 @@ public:
bool readStringInto(PaddedPODArray<UInt8> &) override
{
cannotConvertType("String");
return false;
}
bool readInt8(Int8 &) override
{
cannotConvertType("Int8");
return false;
}
bool readUInt8(UInt8 &) override
{
cannotConvertType("UInt8");
return false;
}
bool readInt16(Int16 &) override
{
cannotConvertType("Int16");
return false;
}
bool readUInt16(UInt16 &) override
{
cannotConvertType("UInt16");
return false;
}
bool readInt32(Int32 &) override
{
cannotConvertType("Int32");
return false;
}
bool readUInt32(UInt32 &) override
{
cannotConvertType("UInt32");
return false;
}
bool readInt64(Int64 &) override
{
cannotConvertType("Int64");
return false;
}
bool readUInt64(UInt64 &) override
{
cannotConvertType("UInt64");
return false;
}
bool readUInt128(UInt128 &) override
{
cannotConvertType("UInt128");
return false;
}
bool readFloat32(Float32 &) override
{
cannotConvertType("Float32");
return false;
}
bool readFloat64(Float64 &) override
{
cannotConvertType("Float64");
return false;
}
void prepareEnumMapping8(const std::vector<std::pair<std::string, Int8>> &) override {}
@ -460,59 +448,50 @@ public:
bool readEnum8(Int8 &) override
{
cannotConvertType("Enum");
return false;
}
bool readEnum16(Int16 &) override
{
cannotConvertType("Enum");
return false;
}
bool readUUID(UUID &) override
{
cannotConvertType("UUID");
return false;
}
bool readDate(DayNum &) override
{
cannotConvertType("Date");
return false;
}
bool readDateTime(time_t &) override
{
cannotConvertType("DateTime");
return false;
}
bool readDecimal32(Decimal32 &, UInt32, UInt32) override
{
cannotConvertType("Decimal32");
return false;
}
bool readDecimal64(Decimal64 &, UInt32, UInt32) override
{
cannotConvertType("Decimal64");
return false;
}
bool readDecimal128(Decimal128 &, UInt32, UInt32) override
{
cannotConvertType("Decimal128");
return false;
}
bool readAggregateFunction(const AggregateFunctionPtr &, AggregateDataPtr, Arena &) override
{
cannotConvertType("AggregateFunction");
return false;
}
protected:
void cannotConvertType(const String & type_name)
[[noreturn]] void cannotConvertType(const String & type_name)
{
throw Exception(
String("Could not convert type '") + field->type_name() + "' from protobuf field '" + field->name() + "' to data type '"
@ -520,7 +499,7 @@ protected:
ErrorCodes::PROTOBUF_BAD_CAST);
}
void cannotConvertValue(const String & value, const String & type_name)
[[noreturn]] void cannotConvertValue(const String & value, const String & type_name)
{
throw Exception(
"Could not convert value '" + value + "' from protobuf field '" + field->name() + "' to data type '" + type_name + "'",
@ -557,7 +536,6 @@ protected:
catch (...)
{
cannotConvertValue(StringRef(str.data(), str.size()).toString(), TypeName<To>::get());
__builtin_unreachable();
}
}

View File

@ -334,14 +334,14 @@ public:
virtual void writeAggregateFunction(const AggregateFunctionPtr &, ConstAggregateDataPtr) override { cannotConvertType("AggregateFunction"); }
protected:
void cannotConvertType(const String & type_name)
[[noreturn]] void cannotConvertType(const String & type_name)
{
throw Exception(
"Could not convert data type '" + type_name + "' to protobuf type '" + field->type_name() + "' (field: " + field->name() + ")",
ErrorCodes::PROTOBUF_BAD_CAST);
}
void cannotConvertValue(const String & value)
[[noreturn]] void cannotConvertValue(const String & value)
{
throw Exception(
"Could not convert value '" + value + "' to protobuf type '" + field->type_name() + "' (field: " + field->name() + ")",

View File

@ -423,7 +423,7 @@ inline bool tryParseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, Read
/** Throw exception with verbose message when string value is not parsed completely.
*/
void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, Block & block, size_t result);
[[noreturn]] void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, Block & block, size_t result);
enum class ConvertFromStringExceptionMode

View File

@ -153,7 +153,7 @@ template <typename A, typename B>
struct NumIfImpl<A, B, NumberTraits::Error>
{
private:
static void throw_error()
[[noreturn]] static void throw_error()
{
throw Exception("Internal logic error: invalid types of arguments 2 and 3 of if", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}

View File

@ -179,7 +179,7 @@ private:
*/
virtual bool nextImpl() { return false; }
void throwReadAfterEOF()
[[noreturn]] void throwReadAfterEOF()
{
throw Exception("Attempt to read after eof", ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF);
}

View File

@ -164,7 +164,7 @@ void readVectorBinary(std::vector<T> & v, ReadBuffer & buf, size_t MAX_VECTOR_SI
void assertString(const char * s, ReadBuffer & buf);
void assertEOF(ReadBuffer & buf);
void throwAtAssertionFailed(const char * s, ReadBuffer & buf);
[[noreturn]] void throwAtAssertionFailed(const char * s, ReadBuffer & buf);
inline void assertChar(char symbol, ReadBuffer & buf)
{

View File

@ -113,7 +113,7 @@ readVarUInt(T & x, ReadBuffer & istr)
}
inline void throwReadAfterEOF()
[[noreturn]] inline void throwReadAfterEOF()
{
throw Exception("Attempt to read after eof", ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF);
}