mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-04 13:32:13 +00:00
Improve reading CSV field in CustomSeparated/Template format
This commit is contained in:
parent
10cc6fa415
commit
fcfdd73d17
@ -117,6 +117,7 @@ struct FormatSettings
|
||||
char tuple_delimiter = ',';
|
||||
bool use_best_effort_in_schema_inference = true;
|
||||
UInt64 skip_first_lines = 0;
|
||||
String custom_delimiter;
|
||||
} csv;
|
||||
|
||||
struct HiveText
|
||||
|
@ -642,9 +642,10 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV &
|
||||
|
||||
const char delimiter = settings.delimiter;
|
||||
const char maybe_quote = *buf.position();
|
||||
const String & custom_delimiter = settings.custom_delimiter;
|
||||
|
||||
/// Emptiness and not even in quotation marks.
|
||||
if (maybe_quote == delimiter)
|
||||
if (custom_delimiter.empty() && maybe_quote == delimiter)
|
||||
return;
|
||||
|
||||
if ((settings.allow_single_quotes && maybe_quote == '\'') || (settings.allow_double_quotes && maybe_quote == '"'))
|
||||
@ -682,6 +683,42 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV &
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If custom_delimiter is specified, we should read until first occurrences of
|
||||
/// custom_delimiter in buffer.
|
||||
if (!custom_delimiter.empty())
|
||||
{
|
||||
PeekableReadBuffer * peekable_buf = dynamic_cast<PeekableReadBuffer *>(&buf);
|
||||
if (!peekable_buf)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Reading CSV string with custom delimiter is allowed only when using PeekableReadBuffer");
|
||||
|
||||
while (true)
|
||||
{
|
||||
if (peekable_buf->eof())
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected EOF while reading CSV string, expected custom delimiter \"{}\"", custom_delimiter);
|
||||
|
||||
char * next_pos = reinterpret_cast<char *>(memchr(peekable_buf->position(), custom_delimiter[0], peekable_buf->available()));
|
||||
if (!next_pos)
|
||||
next_pos = peekable_buf->buffer().end();
|
||||
|
||||
appendToStringOrVector(s, *peekable_buf, next_pos);
|
||||
peekable_buf->position() = next_pos;
|
||||
|
||||
if (!buf.hasPendingData())
|
||||
continue;
|
||||
|
||||
{
|
||||
PeekableReadBufferCheckpoint checkpoint{*peekable_buf, true};
|
||||
if (checkString(custom_delimiter, *peekable_buf))
|
||||
return;
|
||||
}
|
||||
|
||||
s.push_back(*peekable_buf->position());
|
||||
++peekable_buf->position();
|
||||
}
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
/// Unquoted case. Look for delimiter or \r or \n.
|
||||
while (!buf.eof())
|
||||
{
|
||||
@ -776,6 +813,68 @@ void readCSVField(String & s, ReadBuffer & buf, const FormatSettings::CSV & sett
|
||||
s.push_back(quote);
|
||||
}
|
||||
|
||||
void readCSVWithTwoPossibleDelimitersImpl(String & s, PeekableReadBuffer & buf, const String & first_delimiter, const String & second_delimiter)
|
||||
{
|
||||
/// Read all data until first_delimiter or second_delimiter
|
||||
while (true)
|
||||
{
|
||||
if (buf.eof())
|
||||
throw Exception(ErrorCodes::INCORRECT_DATA, R"(Unexpected EOF while reading CSV string, expected on of delimiters "{}" or "{}")", first_delimiter, second_delimiter);
|
||||
|
||||
char * next_pos = buf.position();
|
||||
while (next_pos != buf.buffer().end() && *next_pos != first_delimiter[0] && *next_pos != second_delimiter[0])
|
||||
++next_pos;
|
||||
|
||||
appendToStringOrVector(s, buf, next_pos);
|
||||
buf.position() = next_pos;
|
||||
if (!buf.hasPendingData())
|
||||
continue;
|
||||
|
||||
if (*buf.position() == first_delimiter[0])
|
||||
{
|
||||
PeekableReadBufferCheckpoint checkpoint(buf, true);
|
||||
if (checkString(first_delimiter, buf))
|
||||
return;
|
||||
}
|
||||
|
||||
if (*buf.position() == second_delimiter[0])
|
||||
{
|
||||
PeekableReadBufferCheckpoint checkpoint(buf, true);
|
||||
if (checkString(second_delimiter, buf))
|
||||
return;
|
||||
}
|
||||
|
||||
s.push_back(*buf.position());
|
||||
++buf.position();
|
||||
}
|
||||
}
|
||||
|
||||
String readCSVStringWithTwoPossibleDelimiters(PeekableReadBuffer & buf, const FormatSettings::CSV & settings, const String & first_delimiter, const String & second_delimiter)
|
||||
{
|
||||
String res;
|
||||
|
||||
/// If value is quoted, use regular CSV reading since we need to read only data inside quotes.
|
||||
if (!buf.eof() && ((settings.allow_single_quotes && *buf.position() == '\'') || (settings.allow_double_quotes && *buf.position() == '"')))
|
||||
readCSVStringInto(res, buf, settings);
|
||||
else
|
||||
readCSVWithTwoPossibleDelimitersImpl(res, buf, first_delimiter, second_delimiter);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
String readCSVFieldWithTwoPossibleDelimiters(PeekableReadBuffer & buf, const FormatSettings::CSV & settings, const String & first_delimiter, const String & second_delimiter)
|
||||
{
|
||||
String res;
|
||||
|
||||
/// If value is quoted, use regular CSV reading since we need to read only data inside quotes.
|
||||
if (!buf.eof() && ((settings.allow_single_quotes && *buf.position() == '\'') || (settings.allow_double_quotes && *buf.position() == '"')))
|
||||
readCSVField(res, buf, settings);
|
||||
else
|
||||
readCSVWithTwoPossibleDelimitersImpl(res, buf, first_delimiter, second_delimiter);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
template void readCSVStringInto<PaddedPODArray<UInt8>>(PaddedPODArray<UInt8> & s, ReadBuffer & buf, const FormatSettings::CSV & settings);
|
||||
template void readCSVStringInto<NullOutput>(NullOutput & s, ReadBuffer & buf, const FormatSettings::CSV & settings);
|
||||
|
||||
|
@ -558,9 +558,10 @@ void readStringUntilWhitespace(String & s, ReadBuffer & buf);
|
||||
* - string could be placed in quotes; quotes could be single: ' if FormatSettings::CSV::allow_single_quotes is true
|
||||
* or double: " if FormatSettings::CSV::allow_double_quotes is true;
|
||||
* - or string could be unquoted - this is determined by first character;
|
||||
* - if string is unquoted, then it is read until next delimiter,
|
||||
* either until end of line (CR or LF),
|
||||
* or until end of stream;
|
||||
* - if string is unquoted, then:
|
||||
* - If settings.custom_delimiter is not specified, it is read until next settings.delimiter, either until end of line (CR or LF) or until end of stream;
|
||||
* - If settings.custom_delimiter is specified it reads until first occurrences of settings.custom_delimiter in buffer.
|
||||
* This works only if provided buffer is PeekableReadBuffer.
|
||||
* but spaces and tabs at begin and end of unquoted string are consumed but ignored (note that this behaviour differs from RFC).
|
||||
* - if string is in quotes, then it will be read until closing quote,
|
||||
* but sequences of two consecutive quotes are parsed as single quote inside string;
|
||||
@ -570,6 +571,13 @@ void readCSVString(String & s, ReadBuffer & buf, const FormatSettings::CSV & set
|
||||
/// Differ from readCSVString in that it doesn't remove quotes around field if any.
|
||||
void readCSVField(String & s, ReadBuffer & buf, const FormatSettings::CSV & settings);
|
||||
|
||||
/// Read string in CSV format until the first occurrence of first_delimiter or second_delimiter.
|
||||
/// Similar to readCSVString if string is in quotes, we read only data in quotes.
|
||||
String readCSVStringWithTwoPossibleDelimiters(PeekableReadBuffer & buf, const FormatSettings::CSV & settings, const String & first_delimiter, const String & second_delimiter);
|
||||
|
||||
/// Same as above but includes quotes in the result if any.
|
||||
String readCSVFieldWithTwoPossibleDelimiters(PeekableReadBuffer & buf, const FormatSettings::CSV & settings, const String & first_delimiter, const String & second_delimiter);
|
||||
|
||||
/// Read and append result to array of characters.
|
||||
template <typename Vector>
|
||||
void readStringInto(Vector & s, ReadBuffer & buf);
|
||||
|
@ -12,16 +12,6 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
static FormatSettings updateFormatSettings(const FormatSettings & settings)
|
||||
{
|
||||
if (settings.custom.escaping_rule != FormatSettings::EscapingRule::CSV || settings.custom.field_delimiter.empty())
|
||||
return settings;
|
||||
|
||||
auto updated = settings;
|
||||
updated.csv.delimiter = settings.custom.field_delimiter.front();
|
||||
return updated;
|
||||
}
|
||||
|
||||
CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat(
|
||||
const Block & header_,
|
||||
ReadBuffer & in_buf_,
|
||||
@ -31,7 +21,7 @@ CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat(
|
||||
bool ignore_spaces_,
|
||||
const FormatSettings & format_settings_)
|
||||
: CustomSeparatedRowInputFormat(
|
||||
header_, std::make_unique<PeekableReadBuffer>(in_buf_), params_, with_names_, with_types_, ignore_spaces_, updateFormatSettings(format_settings_))
|
||||
header_, std::make_unique<PeekableReadBuffer>(in_buf_), params_, with_names_, with_types_, ignore_spaces_, format_settings_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -171,15 +161,31 @@ bool CustomSeparatedFormatReader::checkEndOfRow()
|
||||
}
|
||||
|
||||
template <bool is_header>
|
||||
String CustomSeparatedFormatReader::readFieldIntoString(bool is_first)
|
||||
String CustomSeparatedFormatReader::readFieldIntoString(bool is_first, bool is_last, bool is_unknown)
|
||||
{
|
||||
if (!is_first)
|
||||
skipFieldDelimiter();
|
||||
skipSpaces();
|
||||
updateFormatSettings(is_last);
|
||||
if constexpr (is_header)
|
||||
{
|
||||
/// If the number of columns is unknown and we use CSV escaping rule,
|
||||
/// we don't know what delimiter to expect after the value,
|
||||
/// so we should read until we meet field_delimiter or row_after_delimiter.
|
||||
if (is_unknown && format_settings.custom.escaping_rule == FormatSettings::EscapingRule::CSV)
|
||||
return readCSVStringWithTwoPossibleDelimiters(
|
||||
*buf, format_settings.csv, format_settings.custom.field_delimiter, format_settings.custom.row_after_delimiter);
|
||||
|
||||
return readStringByEscapingRule(*buf, format_settings.custom.escaping_rule, format_settings);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (is_unknown && format_settings.custom.escaping_rule == FormatSettings::EscapingRule::CSV)
|
||||
return readCSVFieldWithTwoPossibleDelimiters(
|
||||
*buf, format_settings.csv, format_settings.custom.field_delimiter, format_settings.custom.row_after_delimiter);
|
||||
|
||||
return readFieldByEscapingRule(*buf, format_settings.custom.escaping_rule, format_settings);
|
||||
}
|
||||
}
|
||||
|
||||
template <bool is_header>
|
||||
@ -192,14 +198,14 @@ std::vector<String> CustomSeparatedFormatReader::readRowImpl()
|
||||
{
|
||||
do
|
||||
{
|
||||
values.push_back(readFieldIntoString<is_header>(values.empty()));
|
||||
values.push_back(readFieldIntoString<is_header>(values.empty(), false, true));
|
||||
} while (!checkEndOfRow());
|
||||
columns = values.size();
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i != columns; ++i)
|
||||
values.push_back(readFieldIntoString<is_header>(i == 0));
|
||||
values.push_back(readFieldIntoString<is_header>(i == 0, i + 1 == columns, false));
|
||||
}
|
||||
|
||||
skipRowEndDelimiter();
|
||||
@ -223,9 +229,33 @@ void CustomSeparatedFormatReader::skipHeaderRow()
|
||||
skipRowEndDelimiter();
|
||||
}
|
||||
|
||||
bool CustomSeparatedFormatReader::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool, const String &)
|
||||
void CustomSeparatedFormatReader::updateFormatSettings(bool is_last_column)
|
||||
{
|
||||
if (format_settings.custom.escaping_rule != FormatSettings::EscapingRule::CSV)
|
||||
return;
|
||||
|
||||
format_settings.csv.custom_delimiter.clear();
|
||||
|
||||
if (is_last_column)
|
||||
{
|
||||
if (format_settings.custom.row_after_delimiter.size() == 1)
|
||||
format_settings.csv.delimiter = format_settings.custom.row_after_delimiter.front();
|
||||
else
|
||||
format_settings.csv.custom_delimiter = format_settings.custom.row_after_delimiter;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (format_settings.custom.field_delimiter.size() == 1)
|
||||
format_settings.csv.delimiter = format_settings.custom.field_delimiter.front();
|
||||
else
|
||||
format_settings.csv.custom_delimiter = format_settings.custom.field_delimiter;
|
||||
}
|
||||
}
|
||||
|
||||
bool CustomSeparatedFormatReader::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String &)
|
||||
{
|
||||
skipSpaces();
|
||||
updateFormatSettings(is_last_file_column);
|
||||
return deserializeFieldByEscapingRule(type, serialization, column, *buf, format_settings.custom.escaping_rule, format_settings);
|
||||
}
|
||||
|
||||
@ -237,6 +267,8 @@ bool CustomSeparatedFormatReader::checkForSuffixImpl(bool check_eof)
|
||||
if (!check_eof)
|
||||
return false;
|
||||
|
||||
/// Allow optional \n before eof.
|
||||
checkChar('\n', *buf);
|
||||
return buf->eof();
|
||||
}
|
||||
|
||||
@ -246,6 +278,8 @@ bool CustomSeparatedFormatReader::checkForSuffixImpl(bool check_eof)
|
||||
if (!check_eof)
|
||||
return true;
|
||||
|
||||
/// Allow optional \n before eof.
|
||||
checkChar('\n', *buf);
|
||||
if (buf->eof())
|
||||
return true;
|
||||
}
|
||||
@ -312,7 +346,7 @@ CustomSeparatedSchemaReader::CustomSeparatedSchemaReader(
|
||||
&reader,
|
||||
getDefaultDataTypeForEscapingRule(format_setting_.custom.escaping_rule))
|
||||
, buf(in_)
|
||||
, reader(buf, ignore_spaces_, updateFormatSettings(format_setting_))
|
||||
, reader(buf, ignore_spaces_, format_setting_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -83,7 +83,9 @@ private:
|
||||
std::vector<String> readRowImpl();
|
||||
|
||||
template <bool read_string>
|
||||
String readFieldIntoString(bool is_first);
|
||||
String readFieldIntoString(bool is_first, bool is_last, bool is_unknown);
|
||||
|
||||
void updateFormatSettings(bool is_last_column);
|
||||
|
||||
PeekableReadBuffer * buf;
|
||||
bool ignore_spaces;
|
||||
|
@ -130,9 +130,16 @@ bool TemplateRowInputFormat::deserializeField(const DataTypePtr & type,
|
||||
{
|
||||
EscapingRule escaping_rule = row_format.escaping_rules[file_column];
|
||||
if (escaping_rule == EscapingRule::CSV)
|
||||
/// Will read unquoted string until settings.csv.delimiter
|
||||
settings.csv.delimiter = row_format.delimiters[file_column + 1].empty() ? default_csv_delimiter :
|
||||
row_format.delimiters[file_column + 1].front();
|
||||
{
|
||||
settings.csv.custom_delimiter.clear();
|
||||
if (row_format.delimiters[file_column + 1].empty())
|
||||
settings.csv.delimiter = default_csv_delimiter;
|
||||
else if (row_format.delimiters[file_column + 1].size() == 1)
|
||||
settings.csv.delimiter = row_format.delimiters[file_column + 1].front();
|
||||
else
|
||||
settings.csv.custom_delimiter = row_format.delimiters[file_column + 1];
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
return deserializeFieldByEscapingRule(type, serialization, column, *buf, escaping_rule, settings);
|
||||
@ -466,6 +473,7 @@ TemplateSchemaReader::TemplateSchemaReader(
|
||||
, format(format_)
|
||||
, row_format(row_format_)
|
||||
, format_reader(buf, ignore_spaces_, format, row_format, row_between_delimiter, format_settings)
|
||||
, default_csv_delimiter(format_settings_.csv.delimiter)
|
||||
{
|
||||
setColumnNames(row_format.column_names);
|
||||
}
|
||||
@ -490,7 +498,15 @@ DataTypes TemplateSchemaReader::readRowAndGetDataTypes()
|
||||
{
|
||||
format_reader.skipDelimiter(i);
|
||||
if (row_format.escaping_rules[i] == FormatSettings::EscapingRule::CSV)
|
||||
format_settings.csv.delimiter = row_format.delimiters[i + 1].empty() ? format_settings.csv.delimiter : row_format.delimiters[i + 1].front();
|
||||
{
|
||||
format_settings.csv.custom_delimiter.clear();
|
||||
if (row_format.delimiters[i + 1].empty())
|
||||
format_settings.csv.delimiter = default_csv_delimiter;
|
||||
else if (row_format.delimiters[i + 1].size() == 1)
|
||||
format_settings.csv.delimiter = row_format.delimiters[i + 1].front();
|
||||
else
|
||||
format_settings.csv.custom_delimiter = row_format.delimiters[i + 1];
|
||||
}
|
||||
|
||||
field = readFieldByEscapingRule(buf, row_format.escaping_rules[i], format_settings);
|
||||
data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, row_format.escaping_rules[i]));
|
||||
|
@ -128,6 +128,7 @@ private:
|
||||
const ParsedTemplateFormatString row_format;
|
||||
TemplateFormatReader format_reader;
|
||||
bool first_row = true;
|
||||
const char default_csv_delimiter;
|
||||
};
|
||||
|
||||
bool parseDelimiterWithDiagnosticInfo(WriteBuffer & out, ReadBuffer & buf, const String & delimiter, const String & description, bool skip_spaces);
|
||||
|
@ -111,7 +111,7 @@ public:
|
||||
|
||||
protected:
|
||||
ReadBuffer * in;
|
||||
const FormatSettings format_settings;
|
||||
FormatSettings format_settings;
|
||||
};
|
||||
|
||||
/// Base class for schema inference for formats with -WithNames and -WithNamesAndTypes suffixes.
|
||||
|
@ -0,0 +1,7 @@
|
||||
1 2
|
||||
3 4
|
||||
1 2
|
||||
3 4
|
||||
ab|c de&f
|
||||
ab|c de*f gh&k
|
||||
|av *ad &ad
|
@ -0,0 +1,20 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
echo "1||2&&3||4&&" | $CLICKHOUSE_LOCAL --input-format=CustomSeparated --format_custom_field_delimiter='||' --format_custom_row_after_delimiter='&&' --format_custom_escaping_rule='CSV' -q "select * from table"
|
||||
|
||||
echo "1||2|||3||4|||" | $CLICKHOUSE_LOCAL --input-format=CustomSeparated --format_custom_field_delimiter='||' --format_custom_row_after_delimiter='|||' --format_custom_escaping_rule='CSV' -q "select * from table"
|
||||
|
||||
echo "ab|c||de&f&&" | $CLICKHOUSE_LOCAL --input-format=CustomSeparated --format_custom_field_delimiter='||' --format_custom_row_after_delimiter='&&' --format_custom_escaping_rule='CSV' -q "select * from table"
|
||||
|
||||
echo -e "\${column_1:CSV}||\${column_2:CSV}**\${column_3:CSV}&&" > row_format_02481
|
||||
|
||||
TEMPLATE_SETTINGS="SETTINGS format_template_rows_between_delimiter='\n', format_template_row='row_format_02481'"
|
||||
|
||||
echo -e "ab|c||de*f**gh&k&&\n|av||*ad**&ad&&" | $CLICKHOUSE_LOCAL -q "select * from table" --input-format=Template --format_template_row='row_format_02481' --format_template_rows_between_delimiter ""
|
||||
|
||||
rm row_format_02481
|
||||
|
Loading…
Reference in New Issue
Block a user