reformat ParsingException

This commit is contained in:
Alexander Tokmakov 2023-01-24 23:21:29 +01:00
parent d0805cd0fa
commit d1baa7300c
21 changed files with 89 additions and 83 deletions

View File

@ -88,7 +88,7 @@ public:
{
/// A more understandable error message.
if (e.code() == DB::ErrorCodes::CANNOT_READ_ALL_DATA || e.code() == DB::ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
throw DB::ParsingException("File " + path + " is empty. You must fill it manually with appropriate value.", e.code());
throw DB::ParsingException(e.code(), "File {} is empty. You must fill it manually with appropriate value.", path);
else
throw;
}

View File

@ -178,11 +178,11 @@ private:
/// more convenient calculation of problem line number.
class ParsingException : public Exception
{
ParsingException(const std::string & msg, int code);
public:
ParsingException();
ParsingException(const std::string & msg, int code);
ParsingException(int code, const std::string & message);
ParsingException(int code, std::string && message) : Exception(message, code) {}
ParsingException(int code, std::string && message) : Exception(std::move(message), code) {}
// Format message with fmt::format, like the logging functions.
template <typename... Args>

View File

@ -373,8 +373,8 @@ void SerializationArray::deserializeBinaryBulkWithMultipleStreams(
/// Check consistency between offsets and elements subcolumns.
/// But if elements column is empty - it's ok for columns of Nested types that was added by ALTER.
if (!nested_column->empty() && nested_column->size() != last_offset)
throw ParsingException("Cannot read all array values: read just " + toString(nested_column->size()) + " of " + toString(last_offset),
ErrorCodes::CANNOT_READ_ALL_DATA);
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read all array values: read just {} of {}",
toString(nested_column->size()), toString(last_offset));
column = std::move(mutable_column);
}

View File

@ -360,19 +360,20 @@ ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & col
/// or if someone uses tab or LF in TSV null_representation.
/// In the first case we cannot continue reading anyway. The second case seems to be unlikely.
if (null_representation.find('\t') != std::string::npos || null_representation.find('\n') != std::string::npos)
throw DB::ParsingException("TSV custom null representation containing '\\t' or '\\n' may not work correctly "
"for large input.", ErrorCodes::CANNOT_READ_ALL_DATA);
throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "TSV custom null representation "
"containing '\\t' or '\\n' may not work correctly for large input.");
WriteBufferFromOwnString parsed_value;
if constexpr (escaped)
nested_serialization->serializeTextEscaped(nested_column, nested_column.size() - 1, parsed_value, settings);
else
nested_serialization->serializeTextRaw(nested_column, nested_column.size() - 1, parsed_value, settings);
throw DB::ParsingException("Error while parsing \"" + std::string(pos, buf.buffer().end()) + std::string(istr.position(), std::min(size_t(10), istr.available())) + "\" as Nullable"
+ " at position " + std::to_string(istr.count()) + ": got \"" + std::string(pos, buf.position() - pos)
+ "\", which was deserialized as \""
+ parsed_value.str() + "\". It seems that input data is ill-formatted.",
ErrorCodes::CANNOT_READ_ALL_DATA);
throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while parsing \"{}{}\" as Nullable"
" at position {}: got \"{}\", which was deserialized as \"{}\". "
"It seems that input data is ill-formatted.",
std::string(pos, buf.buffer().end()),
std::string(istr.position(), std::min(size_t(10), istr.available())),
istr.count(), std::string(pos, buf.position() - pos), parsed_value.str());
};
return safeDeserialize<ReturnType>(column, *nested_serialization, check_for_null, deserialize_nested);
@ -584,16 +585,17 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB
/// In the first case we cannot continue reading anyway. The second case seems to be unlikely.
if (null_representation.find(settings.csv.delimiter) != std::string::npos || null_representation.find('\r') != std::string::npos
|| null_representation.find('\n') != std::string::npos)
throw DB::ParsingException("CSV custom null representation containing format_csv_delimiter, '\\r' or '\\n' may not work correctly "
"for large input.", ErrorCodes::CANNOT_READ_ALL_DATA);
throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "CSV custom null representation containing "
"format_csv_delimiter, '\\r' or '\\n' may not work correctly for large input.");
WriteBufferFromOwnString parsed_value;
nested_serialization->serializeTextCSV(nested_column, nested_column.size() - 1, parsed_value, settings);
throw DB::ParsingException("Error while parsing \"" + std::string(pos, buf.buffer().end()) + std::string(istr.position(), std::min(size_t(10), istr.available())) + "\" as Nullable"
+ " at position " + std::to_string(istr.count()) + ": got \"" + std::string(pos, buf.position() - pos)
+ "\", which was deserialized as \""
+ parsed_value.str() + "\". It seems that input data is ill-formatted.",
ErrorCodes::CANNOT_READ_ALL_DATA);
throw DB::ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while parsing \"{}{}\" as Nullable"
" at position {}: got \"{}\", which was deserialized as \"{}\". "
"It seems that input data is ill-formatted.",
std::string(pos, buf.buffer().end()),
std::string(istr.position(), std::min(size_t(10), istr.available())),
istr.count(), std::string(pos, buf.position() - pos), parsed_value.str());
};
return safeDeserialize<ReturnType>(column, *nested_serialization, check_for_null, deserialize_nested);

View File

@ -44,11 +44,10 @@ namespace JSONUtils
{
const auto current_object_size = memory.size() + static_cast<size_t>(pos - in.position());
if (min_bytes != 0 && current_object_size > 10 * min_bytes)
throw ParsingException(
"Size of JSON object is extremely large. Expected not greater than " + std::to_string(min_bytes)
+ " bytes, but current is " + std::to_string(current_object_size)
+ " bytes per row. Increase the value setting 'min_chunk_bytes_for_parallel_parsing' or check your data manually, most likely JSON is malformed",
ErrorCodes::INCORRECT_DATA);
throw ParsingException(ErrorCodes::INCORRECT_DATA,
"Size of JSON object is extremely large. Expected not greater than {} bytes, but current is {} bytes per row. "
"Increase the value setting 'min_chunk_bytes_for_parallel_parsing' or check your data manually, "
"most likely JSON is malformed", min_bytes, current_object_size);
if (quotes)
{

View File

@ -103,7 +103,7 @@ Block NativeReader::read()
if (istr.eof())
{
if (use_index)
throw ParsingException("Input doesn't contain all data for index.", ErrorCodes::CANNOT_READ_ALL_DATA);
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Input doesn't contain all data for index.");
return res;
}

View File

@ -95,14 +95,14 @@ void parseUUIDWithoutSeparator(const UInt8 * src36, std::reverse_iterator<UInt8
void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf)
{
WriteBufferFromOwnString out;
out << "Cannot parse input: expected " << quote << s;
out << quote << s;
if (buf.eof())
out << " at end of stream.";
else
out << " before: " << quote << String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position()));
throw ParsingException(out.str(), ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED);
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "Cannot parse input: expected {}", out.str());
}
@ -603,8 +603,7 @@ static ReturnType readAnyQuotedStringInto(Vector & s, ReadBuffer & buf)
}
if constexpr (throw_exception)
throw ParsingException("Cannot parse quoted string: expected closing quote",
ErrorCodes::CANNOT_PARSE_QUOTED_STRING);
throw ParsingException(ErrorCodes::CANNOT_PARSE_QUOTED_STRING, "Cannot parse quoted string: expected closing quote");
else
return ReturnType(false);
}
@ -939,10 +938,10 @@ ReturnType readJSONStringInto(Vector & s, ReadBuffer & buf)
{
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
auto error = [](const char * message [[maybe_unused]], int code [[maybe_unused]])
auto error = []<typename T>(T && message [[maybe_unused]], int code [[maybe_unused]])
{
if constexpr (throw_exception)
throw ParsingException(message, code);
throw ParsingException(code, message);
return ReturnType(false);
};
@ -990,10 +989,10 @@ ReturnType readJSONObjectPossiblyInvalid(Vector & s, ReadBuffer & buf)
{
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
auto error = [](const char * message [[maybe_unused]], int code [[maybe_unused]])
auto error = []<typename T>(T && message [[maybe_unused]], int code [[maybe_unused]])
{
if constexpr (throw_exception)
throw ParsingException(message, code);
throw ParsingException(code, message);
return ReturnType(false);
};
@ -1161,7 +1160,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D
s_pos[size] = 0;
if constexpr (throw_exception)
throw ParsingException(std::string("Cannot parse DateTime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME);
throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s);
else
return false;
}
@ -1184,7 +1183,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D
s_pos[size] = 0;
if constexpr (throw_exception)
throw ParsingException(std::string("Cannot parse time component of DateTime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME);
throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", s);
else
return false;
}
@ -1211,7 +1210,7 @@ ReturnType readDateTimeTextFallback(time_t & datetime, ReadBuffer & buf, const D
else
{
if constexpr (throw_exception)
throw ParsingException("Cannot parse datetime", ErrorCodes::CANNOT_PARSE_DATETIME);
throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse datetime");
else
return false;
}

View File

@ -287,7 +287,7 @@ inline void readBoolTextWord(bool & x, ReadBuffer & buf, bool support_upper_case
[[fallthrough]];
}
default:
throw ParsingException("Unexpected Bool value", ErrorCodes::CANNOT_PARSE_BOOL);
throw ParsingException(ErrorCodes::CANNOT_PARSE_BOOL, "Unexpected Bool value");
}
}
@ -331,9 +331,8 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
if (has_sign)
{
if constexpr (throw_exception)
throw ParsingException(
"Cannot parse number with multiple sign (+/-) characters",
ErrorCodes::CANNOT_PARSE_NUMBER);
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER,
"Cannot parse number with multiple sign (+/-) characters");
else
return ReturnType(false);
}
@ -349,9 +348,8 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
if (has_sign)
{
if constexpr (throw_exception)
throw ParsingException(
"Cannot parse number with multiple sign (+/-) characters",
ErrorCodes::CANNOT_PARSE_NUMBER);
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER,
"Cannot parse number with multiple sign (+/-) characters");
else
return ReturnType(false);
}
@ -361,7 +359,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf)
else
{
if constexpr (throw_exception)
throw ParsingException("Unsigned type must not contain '-' symbol", ErrorCodes::CANNOT_PARSE_NUMBER);
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Unsigned type must not contain '-' symbol");
else
return ReturnType(false);
}
@ -423,8 +421,8 @@ end:
if (has_sign && !has_number)
{
if constexpr (throw_exception)
throw ParsingException(
"Cannot parse number with a sign character but without any numeric character", ErrorCodes::CANNOT_PARSE_NUMBER);
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER,
"Cannot parse number with a sign character but without any numeric character");
else
return ReturnType(false);
}
@ -808,7 +806,7 @@ inline ReturnType readUUIDTextImpl(UUID & uuid, ReadBuffer & buf)
if constexpr (throw_exception)
{
throw ParsingException(std::string("Cannot parse uuid ") + s, ErrorCodes::CANNOT_PARSE_UUID);
throw ParsingException(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", s);
}
else
{
@ -829,7 +827,7 @@ inline ReturnType readUUIDTextImpl(UUID & uuid, ReadBuffer & buf)
if constexpr (throw_exception)
{
throw ParsingException(std::string("Cannot parse uuid ") + s, ErrorCodes::CANNOT_PARSE_UUID);
throw ParsingException(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", s);
}
else
{
@ -855,7 +853,7 @@ inline ReturnType readIPv4TextImpl(IPv4 & ip, ReadBuffer & buf)
return ReturnType(true);
if constexpr (std::is_same_v<ReturnType, void>)
throw ParsingException(std::string("Cannot parse IPv4 ").append(buf.position(), buf.available()), ErrorCodes::CANNOT_PARSE_IPV4);
throw ParsingException(ErrorCodes::CANNOT_PARSE_IPV4, "Cannot parse IPv4 {}", std::string_view(buf.position(), buf.available()));
else
return ReturnType(false);
}
@ -877,7 +875,7 @@ inline ReturnType readIPv6TextImpl(IPv6 & ip, ReadBuffer & buf)
return ReturnType(true);
if constexpr (std::is_same_v<ReturnType, void>)
throw ParsingException(std::string("Cannot parse IPv6 ").append(buf.position(), buf.available()), ErrorCodes::CANNOT_PARSE_IPV6);
throw ParsingException(ErrorCodes::CANNOT_PARSE_IPV6, "Cannot parse IPv6 {}", std::string_view(buf.position(), buf.available()));
else
return ReturnType(false);
}
@ -1061,7 +1059,7 @@ inline void readDateTimeText(LocalDateTime & datetime, ReadBuffer & buf)
if (10 != size)
{
s[size] = 0;
throw ParsingException(std::string("Cannot parse DateTime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME);
throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse DateTime {}", s);
}
datetime.year((s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'));
@ -1077,7 +1075,7 @@ inline void readDateTimeText(LocalDateTime & datetime, ReadBuffer & buf)
if (8 != size)
{
s[size] = 0;
throw ParsingException(std::string("Cannot parse time component of DateTime ") + s, ErrorCodes::CANNOT_PARSE_DATETIME);
throw ParsingException(ErrorCodes::CANNOT_PARSE_DATETIME, "Cannot parse time component of DateTime {}", s);
}
datetime.hour((s[0] - '0') * 10 + (s[1] - '0'));
@ -1300,7 +1298,7 @@ void readQuoted(std::vector<T> & x, ReadBuffer & buf)
if (*buf.position() == ',')
++buf.position();
else
throw ParsingException("Cannot read array from text", ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT);
throw ParsingException(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, "Cannot read array from text");
}
first = false;
@ -1323,7 +1321,7 @@ void readDoubleQuoted(std::vector<T> & x, ReadBuffer & buf)
if (*buf.position() == ',')
++buf.position();
else
throw ParsingException("Cannot read array from text", ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT);
throw ParsingException(ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT, "Cannot read array from text");
}
first = false;

View File

@ -90,10 +90,10 @@ ReturnType parseDateTimeBestEffortImpl(
const DateLUTImpl & utc_time_zone,
DateTimeSubsecondPart * fractional)
{
auto on_error = [](const std::string & message [[maybe_unused]], int code [[maybe_unused]])
auto on_error = []<typename T>(T && message [[maybe_unused]], int code [[maybe_unused]])
{
if constexpr (std::is_same_v<ReturnType, void>)
throw ParsingException(message, code);
throw ParsingException(code, message);
else
return false;
};

View File

@ -121,7 +121,7 @@ inline bool readDigits(ReadBuffer & buf, T & x, uint32_t & digits, int32_t & exp
if (!tryReadIntText(addition_exp, buf))
{
if constexpr (_throw_on_error)
throw ParsingException("Cannot parse exponent while reading decimal", ErrorCodes::CANNOT_PARSE_NUMBER);
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot parse exponent while reading decimal");
else
return false;
}
@ -134,7 +134,7 @@ inline bool readDigits(ReadBuffer & buf, T & x, uint32_t & digits, int32_t & exp
if (digits_only)
{
if constexpr (_throw_on_error)
throw ParsingException("Unexpected symbol while reading decimal", ErrorCodes::CANNOT_PARSE_NUMBER);
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Unexpected symbol while reading decimal");
return false;
}
stop = true;

View File

@ -160,7 +160,7 @@ ReturnType readFloatTextPreciseImpl(T & x, ReadBuffer & buf)
if (unlikely(res.ec != std::errc()))
{
if constexpr (throw_exception)
throw ParsingException("Cannot read floating point value", ErrorCodes::CANNOT_PARSE_NUMBER);
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value");
else
return ReturnType(false);
}
@ -243,7 +243,7 @@ ReturnType readFloatTextPreciseImpl(T & x, ReadBuffer & buf)
if (unlikely(res.ec != std::errc()))
{
if constexpr (throw_exception)
throw ParsingException("Cannot read floating point value", ErrorCodes::CANNOT_PARSE_NUMBER);
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value");
else
return ReturnType(false);
}
@ -331,7 +331,7 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
if (in.eof())
{
if constexpr (throw_exception)
throw ParsingException("Cannot read floating point value", ErrorCodes::CANNOT_PARSE_NUMBER);
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value");
else
return false;
}
@ -389,7 +389,7 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
if (in.eof())
{
if constexpr (throw_exception)
throw ParsingException("Cannot read floating point value: nothing after exponent", ErrorCodes::CANNOT_PARSE_NUMBER);
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: nothing after exponent");
else
return false;
}
@ -427,7 +427,7 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
if (in.eof())
{
if constexpr (throw_exception)
throw ParsingException("Cannot read floating point value: no digits read", ErrorCodes::CANNOT_PARSE_NUMBER);
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: no digits read");
else
return false;
}
@ -438,14 +438,14 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
if (in.eof())
{
if constexpr (throw_exception)
throw ParsingException("Cannot read floating point value: nothing after plus sign", ErrorCodes::CANNOT_PARSE_NUMBER);
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: nothing after plus sign");
else
return false;
}
else if (negative)
{
if constexpr (throw_exception)
throw ParsingException("Cannot read floating point value: plus after minus sign", ErrorCodes::CANNOT_PARSE_NUMBER);
throw ParsingException(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: plus after minus sign");
else
return false;
}

View File

@ -193,7 +193,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node
{
decoder.decodeString(tmp);
if (tmp.length() != 36)
throw ParsingException(std::string("Cannot parse uuid ") + tmp, ErrorCodes::CANNOT_PARSE_UUID);
throw ParsingException(ErrorCodes::CANNOT_PARSE_UUID, "Cannot parse uuid {}", tmp);
UUID uuid;
parseUUID(reinterpret_cast<const UInt8 *>(tmp.data()), std::reverse_iterator<UInt8 *>(reinterpret_cast<UInt8 *>(&uuid) + 16));

View File

@ -143,7 +143,7 @@ inline bool JSONEachRowRowInputFormat::advanceToNextKey(size_t key_index)
skipWhitespaceIfAny(*in);
if (in->eof())
throw ParsingException("Unexpected end of stream while parsing JSONEachRow format", ErrorCodes::CANNOT_READ_ALL_DATA);
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected end of stream while parsing JSONEachRow format");
else if (*in->position() == '}')
{
++in->position();

View File

@ -62,12 +62,13 @@ Chunk ParquetBlockInputFormat::generate()
arrow::Status get_batch_reader_status = file_reader->GetRecordBatchReader(row_group_indices, column_indices, &rbr);
if (!get_batch_reader_status.ok())
throw ParsingException{"Error while reading Parquet data: " + get_batch_reader_status.ToString(), ErrorCodes::CANNOT_READ_ALL_DATA};
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}",
get_batch_reader_status.ToString());
arrow::Status read_status = rbr->ReadAll(&table);
if (!read_status.ok())
throw ParsingException{"Error while reading Parquet data: " + read_status.ToString(), ErrorCodes::CANNOT_READ_ALL_DATA};
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}", read_status.ToString());
++row_group_current;

View File

@ -92,7 +92,7 @@ static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp)
}
}
throw ParsingException("Unexpected end of stream while reading key name from TSKV format", ErrorCodes::CANNOT_READ_ALL_DATA);
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected end of stream while reading key name from TSKV format");
}
@ -161,7 +161,7 @@ bool TSKVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ex
if (in->eof())
{
throw ParsingException("Unexpected end of stream after field in TSKV format: " + name_ref.toString(), ErrorCodes::CANNOT_READ_ALL_DATA);
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected end of stream after field in TSKV format: {}", name_ref.toString());
}
else if (*in->position() == '\t')
{

View File

@ -21,9 +21,9 @@ namespace ErrorCodes
[[noreturn]] static void throwUnexpectedEof(size_t row_num)
{
throw ParsingException("Unexpected EOF while parsing row " + std::to_string(row_num) + ". "
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Unexpected EOF while parsing row {}. "
"Maybe last row has wrong format or input doesn't contain specified suffix before EOF.",
ErrorCodes::CANNOT_READ_ALL_DATA);
std::to_string(row_num));
}
static void updateFormatSettingsIfNeeded(FormatSettings::EscapingRule escaping_rule, FormatSettings & settings, const ParsedTemplateFormatString & row_format, char default_csv_delimiter, size_t file_column)

View File

@ -99,7 +99,7 @@ public:
/// Read row with raw values.
virtual std::vector<String> readRowForHeaderDetection()
{
throw Exception("Method readRowAndGetFieldsAndDataTypes is not implemented for format reader", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method readRowAndGetFieldsAndDataTypes is not implemented for format reader");
}
/// Skip single field, it's used to skip unknown columns.
@ -127,7 +127,7 @@ public:
virtual FormatSettings::EscapingRule getEscapingRule() const
{
throw Exception("Format reader doesn't have an escaping rule", ErrorCodes::NOT_IMPLEMENTED);
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Format reader doesn't have an escaping rule");
}
protected:

View File

@ -28,38 +28,38 @@ void ProxyV1Handler::run()
// read "PROXY"
if (!readWord(5, word, eol) || word != "PROXY" || eol)
throw ParsingException("PROXY protocol violation", ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED);
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
// read "TCP4" or "TCP6" or "UNKNOWN"
if (!readWord(7, word, eol))
throw ParsingException("PROXY protocol violation", ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED);
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
if (word != "TCP4" && word != "TCP6" && word != "UNKNOWN")
throw ParsingException("PROXY protocol violation", ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED);
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
if (word == "UNKNOWN" && eol)
return;
if (eol)
throw ParsingException("PROXY protocol violation", ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED);
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
// read address
if (!readWord(39, word, eol) || eol)
throw ParsingException("PROXY protocol violation", ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED);
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
stack_data.forwarded_for = std::move(word);
// read address
if (!readWord(39, word, eol) || eol)
throw ParsingException("PROXY protocol violation", ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED);
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
// read port
if (!readWord(5, word, eol) || eol)
throw ParsingException("PROXY protocol violation", ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED);
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
// read port and "\r\n"
if (!readWord(5, word, eol) || !eol)
throw ParsingException("PROXY protocol violation", ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED);
throw ParsingException(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "PROXY protocol violation");
if (!stack_data.forwarded_for.empty())
LOG_TRACE(log, "Forwarded client address from PROXY header: {}", stack_data.forwarded_for);

View File

@ -47,5 +47,8 @@ select 110, (select count() from logs where level = 'Warning' group by message_f
select 120, count() < 3 from (select count() / (select count() from logs) as freq, message_format_string from logs group by message_format_string having freq > 0.10);
select 130, count() < 10 from (select count() / (select count() from logs) as freq, message_format_string from logs group by message_format_string having freq > 0.05);
-- Each message matches its pattern (returns 0 rows)
select 140, message_format_string, any(message) from logs where message not like (replaceRegexpAll(message_format_string, '{[:.0-9dfx]*}', '%') as s)
and message not like ('Code: %Exception: '||s||'%') group by message_format_string;
drop table logs;

View File

@ -22,3 +22,5 @@ USE test;
CREATE TABLE test (id Int32) ENGINE=MergeTree() ORDER BY id;
DROP DATABASE test;
"""
rm -r $dir

View File

@ -43,3 +43,5 @@ expect ":) "
send -- "exit\r"
expect eof
spawn bash -c "rm queries_02352"