Fix comments, reduce extra copying

This commit is contained in:
avogar 2021-12-23 16:46:16 +03:00
parent 5e5d6609ed
commit ce68c54b03
5 changed files with 259 additions and 76 deletions

View File

@ -6,43 +6,19 @@
#include <IO/ReadBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/PeekableReadBuffer.h>
#include <unordered_set>
#include <boost/algorithm/string.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING;
extern const int ILLEGAL_COLUMN;
extern const int CANNOT_PARSE_BOOL;
}
static const std::unordered_set<std::string_view> true_values =
{
"enable",
"enabled",
"true",
"yes",
"on",
"t",
"y",
"1",
};
static const std::unordered_set<std::string_view> false_values =
{
"disable",
"disabled",
"false",
"no",
"off",
"f",
"n",
"0",
};
static const ColumnUInt8 * checkAndGetSerializeColumnType(const IColumn & column)
{
const auto * col = checkAndGetColumn<ColumnUInt8>(&column);
@ -79,10 +55,9 @@ void SerializationBool::serializeTextEscaped(const IColumn & column, size_t row_
void SerializationBool::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
if (istr.eof())
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
String input;
readEscapedString(input, istr);
deserializeFromString(column, input, settings);
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL);
deserializeWithCustom(column, istr, settings, [](ReadBuffer & buf){ return buf.eof() || *buf.position() == '\t' || *buf.position() == '\n'; });
}
void SerializationBool::serializeTextJSON(const IColumn &column, size_t row_num, WriteBuffer &ostr, const FormatSettings &settings) const
@ -104,11 +79,11 @@ void SerializationBool::deserializeTextJSON(IColumn &column, ReadBuffer &istr, c
readBoolText(value, istr);
else
throw Exception("Invalid boolean value, should be true/false, 1/0.",
ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
ErrorCodes::CANNOT_PARSE_BOOL);
col->insert(value);
}
else
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL);
}
void SerializationBool::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -119,10 +94,9 @@ void SerializationBool::serializeTextCSV(const IColumn & column, size_t row_num,
void SerializationBool::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
if (istr.eof())
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
String input;
readCSVString(input, istr, settings.csv);
deserializeFromString(column, input, settings);
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL);
deserializeWithCustom(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof() || *buf.position() == settings.csv.delimiter || *buf.position() == '\n'; });
}
void SerializationBool::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -133,10 +107,9 @@ void SerializationBool::serializeTextRaw(const IColumn & column, size_t row_num,
void SerializationBool::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
if (istr.eof())
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
String input;
readString(input, istr);
deserializeFromString(column, input, settings);
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL);
deserializeWithCustom(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof() || *buf.position() == '\t' || *buf.position() == '\n'; });
}
void SerializationBool::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -146,27 +119,22 @@ void SerializationBool::serializeTextQuoted(const IColumn & column, size_t row_n
void SerializationBool::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
if (!istr.eof())
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
if (istr.eof())
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL);
auto * col = checkAndGetDeserializeColumnType(column);
bool value = false;
if (*istr.position() == 't' || *istr.position() == 'f' || *istr.position() == 'T' || *istr.position() == 'F')
readBoolTextWord(value, istr, true);
else if (*istr.position() == '1' || *istr.position() == '0')
readBoolText(value, istr);
else
throw Exception("Invalid boolean value, should be true/false, TRUE/FALSE, 1/0.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
col->insert(value);
if (!deserializeImpl(col, istr))
throw Exception(
ErrorCodes::CANNOT_PARSE_BOOL,
"Invalid boolean value, should be one of True/False/T/F/Y/N/Yes/No/On/Off/Enable/Disable/Enabled/Disabled/1/0");
}
void SerializationBool::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
String input;
readStringUntilEOF(input, istr);
deserializeFromString(column, input, settings);
assert(istr.eof());
if (istr.eof())
throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL);
deserializeWithCustom(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof(); });
}
void SerializationBool::serializeCustom(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
@ -198,35 +166,134 @@ void SerializationBool::serializeSimple(const IColumn & column, size_t row_num,
ostr.write(str_false, sizeof(str_false) - 1);
}
void SerializationBool::deserializeFromString(IColumn & column, String & input, const FormatSettings & settings) const
void SerializationBool::deserializeWithCustom(
IColumn & column, ReadBuffer & istr, const FormatSettings & settings, std::function<bool(ReadBuffer &)> check_end_of_value) const
{
ColumnUInt8 * col = checkAndGetDeserializeColumnType(column);
if (settings.bool_true_representation == input)
PeekableReadBuffer buf(istr);
buf.setCheckpoint();
if (checkString(settings.bool_true_representation, buf) && check_end_of_value(buf))
{
col->insert(true);
return;
}
else if (settings.bool_false_representation == input)
buf.rollbackToCheckpoint();
if (checkString(settings.bool_false_representation, buf) && check_end_of_value(buf))
{
col->insert(false);
buf.dropCheckpoint();
if (buf.hasUnreadData())
throw Exception(
ErrorCodes::CANNOT_PARSE_BOOL,
"Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if "
"bool_true_representation or bool_false_representation contains some delimiters of input format");
return;
}
buf.rollbackToCheckpoint();
if (deserializeImpl(col, buf) && check_end_of_value(buf))
{
buf.dropCheckpoint();
if (buf.hasUnreadData())
throw Exception(
ErrorCodes::CANNOT_PARSE_BOOL,
"Cannot continue parsing after parsed bool value because it will result in the loss of some data. It may happen if "
"bool_true_representation or bool_false_representation contains some delimiters of input format");
return;
}
throw Exception(
ErrorCodes::CANNOT_PARSE_BOOL,
"Invalid boolean value, should be '{}' or '{}' controlled by setting bool_true_representation and "
"bool_false_representation or one of "
"True/False/T/F/Y/N/Yes/No/On/Off/Enable/Disable/Enabled/Disabled/1/0",
settings.bool_true_representation, settings.bool_false_representation);
}
bool SerializationBool::deserializeImpl(ColumnUInt8 * column, ReadBuffer & istr) const
{
if (checkCharCaseInsensitive('1', istr))
{
column->insert(true);
}
else if (checkCharCaseInsensitive('0', istr))
{
column->insert(false);
}
/// 'True' and 'T'
else if (checkCharCaseInsensitive('t', istr))
{
/// Check if it's just short form `T` or full form `True`
if (checkCharCaseInsensitive('r', istr))
{
if (!checkStringCaseInsensitive("ue", istr))
return false;
}
column->insert(true);
}
/// 'False' and 'F'
else if (checkCharCaseInsensitive('f', istr))
{
/// Check if it's just short form `F` or full form `False`
if (checkCharCaseInsensitive('a', istr))
{
if (!checkStringCaseInsensitive("lse", istr))
return false;
}
column->insert(false);
}
/// 'Yes' and 'Y'
else if (checkCharCaseInsensitive('y', istr))
{
/// Check if it's just short form `Y` or full form `Yes`
if (checkCharCaseInsensitive('e', istr))
{
if (!checkCharCaseInsensitive('s', istr))
return false;
}
column->insert(true);
}
/// 'No' and 'N'
else if (checkCharCaseInsensitive('n', istr))
{
/// Check if it's just short form `N` or full form `No`
checkCharCaseInsensitive('o', istr);
column->insert(false);
}
/// 'On' and 'Off'
else if (checkCharCaseInsensitive('o', istr))
{
if (checkCharCaseInsensitive('n', istr))
column->insert(true);
else if (checkStringCaseInsensitive("ff", istr))
{
column->insert(false);
}
else
return false;
}
/// 'Enable' and 'Enabled'
else if (checkStringCaseInsensitive("enable", istr))
{
/// Check if it's 'enable' or 'enabled'
checkCharCaseInsensitive('d', istr);
column->insert(true);
}
/// 'Disable' and 'Disabled'
else if (checkStringCaseInsensitive("disable", istr))
{
/// Check if it's 'disable' or 'disabled'
checkCharCaseInsensitive('d', istr);
column->insert(false);
}
else
{
String input_lower = boost::algorithm::to_lower_copy(input);
if (true_values.contains(input_lower))
{
col->insert(true);
}
else if (false_values.contains(input_lower))
{
col->insert(false);
}
else
throw Exception(
"Invalid boolean value '" + input + "', should be " + settings.bool_true_representation + " or " + settings.bool_false_representation
+ " controlled by setting bool_true_representation and bool_false_representation or one of "
"True/False/T/F/Y/N/Yes/No/On/Off",
ErrorCodes::ILLEGAL_COLUMN);
return false;
}
return true;
}
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <DataTypes/Serializations/SerializationWrapper.h>
#include <Columns/ColumnsNumber.h>
#include <unordered_set>
namespace DB
@ -39,7 +40,8 @@ public:
protected:
void serializeCustom(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const;
void serializeSimple(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const;
void deserializeFromString(IColumn & column, String & input, const FormatSettings & settings) const;
void deserializeWithCustom(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, std::function<bool(ReadBuffer & buf)> check_end_of_value) const;
bool deserializeImpl(ColumnUInt8 * column, ReadBuffer & istr) const;
};
}

View File

@ -182,6 +182,7 @@ struct ConvertImpl
vec_null_map_to = &col_null_map_to->getData();
}
bool result_is_bool = isBool(result_type);
for (size_t i = 0; i < input_rows_count; ++i)
{
if constexpr (std::is_same_v<FromDataType, DataTypeUUID> != std::is_same_v<ToDataType, DataTypeUUID>)
@ -271,7 +272,7 @@ struct ConvertImpl
if constexpr (std::is_same_v<ToDataType, DataTypeUInt8>)
{
if (result_type->getName() == "Bool")
if (result_is_bool)
vec_to[i] = static_cast<bool>(vec_to[i]);
}
}

View File

@ -26,6 +26,110 @@ false
true
false
true
false
true
false
true
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
false
true
true
false
true
false
true
false
true

View File

@ -1,4 +1,5 @@
#!/usr/bin/env bash
# Tags: no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
@ -8,9 +9,17 @@ USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonex
FILE_NAME=test_02152.data
DATA_FILE=$USER_FILES_PATH/$FILE_NAME
echo -e "Custom true\nCustom false\nYes\nNo\nyes\nno\ny\nY\nN\nTrue\nFalse\ntrue\nfalse\nt\nf\nT\nF\nOn\nOff\non\noff" > $DATA_FILE
echo -e "Custom true\nCustom false\nYes\nNo\nyes\nno\ny\nY\nN\nTrue\nFalse\ntrue\nfalse\nt\nf\nT\nF\nOn\nOff\non\noff\nenable\ndisable\nenabled\ndisabled" > $DATA_FILE
$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'TSV', 'bool Bool') settings bool_true_representation='Custom true', bool_false_representation='Custom false'"
$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'TSV', 'bool Bool') settings bool_true_representation='Custom true', bool_false_representation='Custom false', input_format_parallel_parsing=0, max_read_buffer_size=2"
$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'CSV', 'bool Bool') settings bool_true_representation='Custom true', bool_false_representation='Custom false'"
$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'CSV', 'bool Bool') settings bool_true_representation='Custom true', bool_false_representation='Custom false', input_format_parallel_parsing=0, max_read_buffer_size=2"
echo -e "Yes\nNo\nyes\nno\ny\nY\nN\nTrue\nFalse\ntrue\nfalse\nt\nf\nT\nF\nOn\nOff\non\noff\nenable\ndisable\nenabled\ndisabled" > $DATA_FILE
$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'CustomSeparated', 'bool Bool') settings format_custom_escaping_rule='Quoted'"
$CLICKHOUSE_CLIENT -q "SELECT * FROM file('$FILE_NAME', 'CustomSeparated', 'bool Bool') settings format_custom_escaping_rule='Quoted', input_format_parallel_parsing=0, max_read_buffer_size=2"
rm $DATA_FILE