Merge branch 'master' of https://github.com/amosbird/ClickHouse into amosbird-master

This commit is contained in:
Alexey Milovidov 2018-07-04 23:54:54 +03:00
commit 7bdafa6105
13 changed files with 84 additions and 19 deletions

View File

@ -241,7 +241,7 @@ void DataTypeAggregateFunction::serializeTextCSV(const IColumn & column, size_t
void DataTypeAggregateFunction::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
String s;
readCSV(s, istr, settings.csv.delimiter);
readCSV(s, istr, settings.csv);
deserializeFromString(function, column, s);
}

View File

@ -415,7 +415,7 @@ void DataTypeArray::serializeTextCSV(const IColumn & column, size_t row_num, Wri
void DataTypeArray::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
String s;
readCSV(s, istr, settings.csv.delimiter);
readCSV(s, istr, settings.csv);
ReadBufferFromString rb(s);
deserializeText(column, rb, settings);
}

View File

@ -194,7 +194,7 @@ template <typename Type>
void DataTypeEnum<Type>::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
std::string name;
readCSVString(name, istr, settings.csv.delimiter);
readCSVString(name, istr, settings.csv);
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(name)));
}

View File

@ -197,7 +197,7 @@ void DataTypeFixedString::serializeTextCSV(const IColumn & column, size_t row_nu
void DataTypeFixedString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
read(*this, column, [&istr, delimiter = settings.csv.delimiter](ColumnFixedString::Chars_t & data) { readCSVStringInto(data, istr, delimiter); });
read(*this, column, [&istr, &csv = settings.csv](ColumnFixedString::Chars_t & data) { readCSVStringInto(data, istr, csv); });
}

View File

@ -288,7 +288,7 @@ void DataTypeString::serializeTextCSV(const IColumn & column, size_t row_num, Wr
void DataTypeString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
read(column, [&](ColumnString::Chars_t & data) { readCSVStringInto(data, istr, settings.csv.delimiter); });
read(column, [&](ColumnString::Chars_t & data) { readCSVStringInto(data, istr, settings.csv); });
}

View File

@ -83,16 +83,16 @@ static inline void skipWhitespacesAndTabs(ReadBuffer & buf)
}
static void skipRow(ReadBuffer & istr, const char delimiter, size_t num_columns)
static void skipRow(ReadBuffer & istr, const FormatSettings::CSV & csv, size_t num_columns)
{
String tmp;
for (size_t i = 0; i < num_columns; ++i)
{
skipWhitespacesAndTabs(istr);
readCSVString(tmp, istr, delimiter);
readCSVString(tmp, istr, csv);
skipWhitespacesAndTabs(istr);
skipDelimiter(istr, delimiter, i + 1 == num_columns);
skipDelimiter(istr, csv.delimiter, i + 1 == num_columns);
}
}
@ -107,7 +107,7 @@ void CSVRowInputStream::readPrefix()
String tmp;
if (with_names)
skipRow(istr, format_settings.csv.delimiter, num_columns);
skipRow(istr, format_settings.csv, num_columns);
}

View File

@ -37,6 +37,8 @@ BlockInputStreamPtr FormatFactory::getInput(const String & name, ReadBuffer & bu
FormatSettings format_settings;
format_settings.csv.delimiter = settings.format_csv_delimiter;
format_settings.csv.single_quote = settings.format_csv_single_quote;
format_settings.csv.double_quote = settings.format_csv_double_quote;
format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions;
format_settings.skip_unknown_fields = settings.input_format_skip_unknown_fields;
format_settings.date_time_input_format = settings.date_time_input_format;
@ -59,6 +61,8 @@ BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer &
format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers;
format_settings.json.quote_denormals = settings.output_format_json_quote_denormals;
format_settings.csv.delimiter = settings.format_csv_delimiter;
format_settings.csv.single_quote = settings.format_csv_single_quote;
format_settings.csv.double_quote = settings.format_csv_double_quote;
format_settings.pretty.max_rows = settings.output_format_pretty_max_rows;
format_settings.pretty.color = settings.output_format_pretty_color;
format_settings.write_statistics = settings.output_format_write_statistics;

View File

@ -24,6 +24,8 @@ struct FormatSettings
struct CSV
{
char delimiter = ',';
bool single_quote = true;
bool double_quote = true;
};
CSV csv;

View File

@ -2,6 +2,7 @@
#include <Common/hex.h>
#include <Common/PODArray.h>
#include <Common/StringUtils/StringUtils.h>
#include <Formats/FormatSettings.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/readFloatText.h>
@ -500,18 +501,19 @@ void readBackQuotedStringWithSQLStyle(String & s, ReadBuffer & buf)
template <typename Vector>
void readCSVStringInto(Vector & s, ReadBuffer & buf, const char delimiter)
void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV & csv)
{
if (buf.eof())
throwReadAfterEOF();
char maybe_quote = *buf.position();
const char delimiter = csv.delimiter;
const char maybe_quote = *buf.position();
/// Emptiness and not even in quotation marks.
if (maybe_quote == delimiter)
return;
if (maybe_quote == '\'' || maybe_quote == '"')
if ((csv.single_quote && maybe_quote == '\'') || (csv.double_quote && maybe_quote == '"'))
{
++buf.position();
@ -575,13 +577,13 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const char delimiter)
}
}
void readCSVString(String & s, ReadBuffer & buf, const char delimiter)
void readCSVString(String & s, ReadBuffer & buf, const FormatSettings::CSV & csv)
{
s.clear();
readCSVStringInto(s, buf, delimiter);
readCSVStringInto(s, buf, csv);
}
template void readCSVStringInto<PaddedPODArray<UInt8>>(PaddedPODArray<UInt8> & s, ReadBuffer & buf, const char delimiter);
template void readCSVStringInto<PaddedPODArray<UInt8>>(PaddedPODArray<UInt8> & s, ReadBuffer & buf, const FormatSettings::CSV & csv);
template <typename Vector, typename ReturnType>

View File

@ -20,6 +20,8 @@
#include <Common/Arena.h>
#include <Common/UInt128.h>
#include <Formats/FormatSettings.h>
#include <IO/ReadBuffer.h>
#include <IO/ReadBufferFromMemory.h>
#include <IO/VarInt.h>
@ -398,7 +400,8 @@ void readStringUntilEOF(String & s, ReadBuffer & buf);
/** Read string in CSV format.
* Parsing rules:
* - string could be placed in quotes; quotes could be single: ' or double: ";
* - string could be placed in quotes; quotes could be single: ' if FormatSettings::CSV::single_quote is true
* or double: " if FormatSettings::CSV::double_quote 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),
@ -407,7 +410,7 @@ void readStringUntilEOF(String & s, ReadBuffer & buf);
* - 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;
*/
void readCSVString(String & s, ReadBuffer & buf, const char delimiter);
void readCSVString(String & s, ReadBuffer & buf, const FormatSettings::CSV & csv);
/// Read and append result to array of characters.
@ -430,7 +433,7 @@ template <typename Vector>
void readStringUntilEOFInto(Vector & s, ReadBuffer & buf);
template <typename Vector>
void readCSVStringInto(Vector & s, ReadBuffer & buf, const char delimiter);
void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV & csv);
/// ReturnType is either bool or void. If bool, the function will return false instead of throwing an exception.
template <typename Vector, typename ReturnType = void>
@ -688,7 +691,7 @@ template <typename T>
inline std::enable_if_t<std::is_arithmetic_v<T>, void>
readCSV(T & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
inline void readCSV(String & x, ReadBuffer & buf, const char delimiter = ',') { readCSVString(x, buf, delimiter); }
inline void readCSV(String & x, ReadBuffer & buf, const FormatSettings::CSV & csv) { readCSVString(x, buf, csv); }
inline void readCSV(LocalDate & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
inline void readCSV(LocalDateTime & x, ReadBuffer & buf) { readCSVSimple(x, buf); }
inline void readCSV(UUID & x, ReadBuffer & buf) { readCSVSimple(x, buf); }

View File

@ -262,6 +262,8 @@ struct Settings
M(SettingUInt64, max_network_bandwidth_for_user, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running user queries. Zero means unlimited.")\
M(SettingUInt64, max_network_bandwidth_for_all_users, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.") \
M(SettingChar, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.") \
M(SettingBool, format_csv_single_quote, 1, "If it is set to true, single quote is treated as string field quote character.") \
M(SettingBool, format_csv_double_quote, 1, "If it is set to true, double quote is treated as string field quote character.") \
M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \
\
M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \

View File

@ -0,0 +1,8 @@
\'single quote\' not end 123 2016-01-01
\'em good 456 2016-01-02
\'single quote\' not end 123 2016-01-01
\'em good 456 2016-01-02
"double quote" not end 123 2016-01-01
"em good 456 2016-01-02
"double quote" not end 123 2016-01-01
"em good 456 2016-01-02

View File

@ -0,0 +1,44 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (s String, n UInt64, d Date) ENGINE = Memory";
echo "'single quote' not end, 123, 2016-01-01
'em good, 456, 2016-01-02" | $CLICKHOUSE_CLIENT --format_csv_single_quote=0 --query="INSERT INTO test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv ORDER BY d";
$CLICKHOUSE_CLIENT --query="DROP TABLE test.csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (s String, n UInt64, d Date) ENGINE = Memory";
echo "'single quote' not end, 123, 2016-01-01
'em good, 456, 2016-01-02" | $CLICKHOUSE_CLIENT --multiquery --query="SET format_csv_single_quote=0; INSERT INTO test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv ORDER BY d";
$CLICKHOUSE_CLIENT --query="DROP TABLE test.csv";
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (s String, n UInt64, d Date) ENGINE = Memory";
echo '"double quote" not end, 123, 2016-01-01
"em good, 456, 2016-01-02' | $CLICKHOUSE_CLIENT --format_csv_double_quote=0 --query="INSERT INTO test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv ORDER BY d";
$CLICKHOUSE_CLIENT --query="DROP TABLE test.csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (s String, n UInt64, d Date) ENGINE = Memory";
echo '"double quote" not end, 123, 2016-01-01
"em good, 456, 2016-01-02' | $CLICKHOUSE_CLIENT --multiquery --query="SET format_csv_double_quote=0; INSERT INTO test.csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv ORDER BY d";
$CLICKHOUSE_CLIENT --query="DROP TABLE test.csv";