Refactor and improve TSV, CSV and JSONCompactEachRow formats, fix some bugs in formats

This commit is contained in:
avogar 2021-10-14 13:32:49 +03:00
parent 6c6dd7a0c8
commit 324dfd4f81
72 changed files with 1743 additions and 1285 deletions

View File

@ -582,6 +582,17 @@ DataTypes Block::getDataTypes() const
return res;
}
Names Block::getDataTypeNames() const
{
Names res;
res.reserve(columns());
for (const auto & elem : data)
res.push_back(elem.type->getName());
return res;
}
bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs)
{

View File

@ -90,6 +90,7 @@ public:
NamesAndTypesList getNamesAndTypesList() const;
Names getNames() const;
DataTypes getDataTypes() const;
Names getDataTypeNames() const;
/// Returns number of rows from first column in block, not equal to nullptr. If no columns, returns 0.
size_t rows() const;

View File

@ -557,8 +557,10 @@ class IColumn;
M(Bool, input_format_csv_arrays_as_nested_csv, false, R"(When reading Array from CSV, expect that its elements were serialized in nested CSV and then put into string. Example: "[""Hello"", ""world"", ""42"""" TV""]". Braces around array can be omitted.)", 0) \
M(Bool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).", 0) \
M(Bool, input_format_with_names_use_header, true, "For TSVWithNames and CSVWithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \
M(Bool, input_format_with_types_use_header, true, "For TSVWithNamesTypes and CSVWithNamesAndTypes input formats this controls whether format parser should check if data types from the input match data types from the header.", 0) \
M(Bool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \
M(Bool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", IMPORTANT) \
M(Bool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, JSONCompactEachRow, CSV and TSV formats).", IMPORTANT) \
M(Bool, input_format_csv_empty_as_default, true, "Treat empty fields in CSV input as default values.", 0) \
M(Bool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \
M(Bool, input_format_tsv_enum_as_number, false, "Treat inserted enum values in TSV formats as enum indices \\N", 0) \
M(Bool, input_format_null_as_default, true, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \

View File

@ -51,6 +51,7 @@ public:
bool isNullable() const override { return false; }
bool onlyNull() const override { return false; }
bool lowCardinality() const override { return true; }
bool isLowCardinalityNullable() const override { return dictionary_type->isNullable(); }
static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type);
static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys);

View File

@ -270,6 +270,9 @@ public:
virtual bool lowCardinality() const { return false; }
/// Checks if this type is LowCardinality(Nullable(...))
virtual bool isLowCardinalityNullable() const { return false; }
/// Strings, Numbers, Date, DateTime, Nullable
virtual bool canBeInsideLowCardinality() const { return false; }

View File

@ -3,6 +3,7 @@
#include <Columns/IColumn.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
#include <IO/ReadBufferFromString.h>
#include <Common/escapeForFileName.h>
#include <DataTypes/NestedUtils.h>
@ -195,4 +196,18 @@ bool ISerialization::isSpecialCompressionAllowed(const SubstreamPath & path)
return true;
}
void ISerialization::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
String field;
/// Read until \t or \n.
readString(field, istr);
ReadBufferFromString buf(field);
deserializeWholeText(column, buf, settings);
}
void ISerialization::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
serializeText(column, row_num, ostr, settings);
}
}

View File

@ -246,6 +246,14 @@ public:
serializeText(column, row_num, ostr, settings);
}
/** Text deserialization without escaping and quoting. Reads all data until first \n or \t
* into a temporary string and then call deserializeWholeText. It was implemented this way
* because this function is rarely used and because proper implementation requires a lot of
* additional code in data types serialization and ReadHelpers.
*/
virtual void deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const;
virtual void serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const;
static String getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path);
static String getFileNameForStream(const String & name_in_storage, const SubstreamPath & path);
static String getSubcolumnNameForStream(const SubstreamPath & path);

View File

@ -163,7 +163,7 @@ void SerializationFixedString::deserializeTextQuoted(IColumn & column, ReadBuffe
void SerializationFixedString::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
read(*this, column, [&istr](ColumnFixedString::Chars & data) { readStringInto(data, istr); });
read(*this, column, [&istr](ColumnFixedString::Chars & data) { readStringUntilEOFInto(data, istr); });
}

View File

@ -9,6 +9,7 @@
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/ConcatReadBuffer.h>
#include <IO/ReadBufferFromString.h>
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
@ -218,8 +219,36 @@ void SerializationNullable::deserializeTextEscaped(IColumn & column, ReadBuffer
deserializeTextEscapedImpl<void>(column, istr, settings, nested);
}
void SerializationNullable::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
deserializeTextRawImpl<void>(column, istr, settings, nested);
}
void SerializationNullable::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
const ColumnNullable & col = assert_cast<const ColumnNullable &>(column);
if (col.isNullAt(row_num))
writeString(settings.tsv.null_representation, ostr);
else
nested->serializeTextRaw(col.getNestedColumn(), row_num, ostr, settings);
}
template<typename ReturnType>
ReturnType SerializationNullable::deserializeTextRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested)
{
return deserializeTextEscapedAndRawImpl<ReturnType, false>(column, istr, settings, nested);
}
template<typename ReturnType>
ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings,
const SerializationPtr & nested)
{
return deserializeTextEscapedAndRawImpl<ReturnType, true>(column, istr, settings, nested);
}
template<typename ReturnType, bool escaped>
ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings,
const SerializationPtr & nested)
{
/// Little tricky, because we cannot discriminate null from first character.
@ -229,7 +258,13 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R
/// This is not null, surely.
return safeDeserialize<ReturnType>(column, *nested,
[] { return false; },
[&nested, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextEscaped(nested_column, istr, settings); });
[&nested, &istr, &settings] (IColumn & nested_column)
{
if constexpr (escaped)
nested->deserializeTextEscaped(nested_column, istr, settings);
else
nested->deserializeTextRaw(nested_column, istr, settings);
});
}
else
{
@ -255,7 +290,10 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R
{
/// We could step back to consume backslash again.
--istr.position();
nested->deserializeTextEscaped(nested_column, istr, settings);
if constexpr (escaped)
nested->deserializeTextEscaped(nested_column, istr, settings);
else
nested->deserializeTextRaw(nested_column, istr, settings);
}
else
{
@ -263,7 +301,10 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R
ReadBufferFromMemory prefix("\\", 1);
ConcatReadBuffer prepended_istr(prefix, istr);
nested->deserializeTextEscaped(nested_column, prepended_istr, settings);
if constexpr (escaped)
nested->deserializeTextEscaped(nested_column, prepended_istr, settings);
else
nested->deserializeTextRaw(nested_column, prepended_istr, settings);
/// Synchronise cursor position in original buffer.
@ -469,5 +510,6 @@ template bool SerializationNullable::deserializeTextEscapedImpl<bool>(IColumn &
template bool SerializationNullable::deserializeTextQuotedImpl<bool>(IColumn & column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested);
template bool SerializationNullable::deserializeTextCSVImpl<bool>(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested);
template bool SerializationNullable::deserializeTextJSONImpl<bool>(IColumn & column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested);
template bool SerializationNullable::deserializeTextRawImpl<bool>(IColumn & column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested);
}

View File

@ -68,6 +68,9 @@ public:
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
void deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
void serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
/// If ReturnType is bool, check for NULL and deserialize value into non-nullable column (and return true) or insert default value of nested type (and return false)
/// If ReturnType is void, deserialize Nullable(T)
template <typename ReturnType = bool>
@ -80,6 +83,10 @@ public:
static ReturnType deserializeTextCSVImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested);
template <typename ReturnType = bool>
static ReturnType deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested);
template <typename ReturnType = bool>
static ReturnType deserializeTextRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested);
template <typename ReturnType = bool, bool escaped>
static ReturnType deserializeTextEscapedAndRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested);
};
}

View File

@ -245,7 +245,7 @@ static inline void read(IColumn & column, Reader && reader)
void SerializationString::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
read(column, [&](ColumnString::Chars & data) { readStringInto(data, istr); });
read(column, [&](ColumnString::Chars & data) { readStringUntilEOFInto(data, istr); });
}

View File

@ -15,7 +15,6 @@
#include <Processors/Formats/Impl/ParallelFormattingOutputFormat.h>
#include <Poco/URI.h>
#include <IO/ReadHelpers.h>
namespace DB
{
@ -57,7 +56,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes;
format_settings.csv.crlf_end_of_line = settings.output_format_csv_crlf_end_of_line;
format_settings.csv.delimiter = settings.format_csv_delimiter;
format_settings.csv.empty_as_default = settings.input_format_defaults_for_omitted_fields;
format_settings.csv.empty_as_default = settings.input_format_csv_empty_as_default;
format_settings.csv.input_format_enum_as_number = settings.input_format_csv_enum_as_number;
format_settings.csv.null_representation = settings.output_format_csv_null_representation;
format_settings.csv.unquoted_null_literal_as_null = settings.input_format_csv_unquoted_null_literal_as_null;
@ -108,10 +107,12 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions;
format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions;
format_settings.with_names_use_header = settings.input_format_with_names_use_header;
format_settings.with_types_use_header = settings.input_format_with_types_use_header;
format_settings.write_statistics = settings.output_format_write_statistics;
format_settings.arrow.low_cardinality_as_dictionary = settings.output_format_arrow_low_cardinality_as_dictionary;
format_settings.arrow.import_nested = settings.input_format_arrow_import_nested;
format_settings.orc.import_nested = settings.input_format_orc_import_nested;
format_settings.defaults_for_omitted_fields = settings.input_format_defaults_for_omitted_fields;
/// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context
if (format_settings.schema.is_server)

View File

@ -25,10 +25,12 @@ struct FormatSettings
bool skip_unknown_fields = false;
bool with_names_use_header = false;
bool with_types_use_header = false;
bool write_statistics = true;
bool import_nested_json = false;
bool null_as_default = true;
bool decimal_trailing_zeros = false;
bool defaults_for_omitted_fields = true;
enum class DateTimeInputFormat
{

View File

@ -1,4 +1,7 @@
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromString.h>
#include <DataTypes/Serializations/SerializationNullable.h>
#include <base/find_symbols.h>
namespace DB
@ -9,7 +12,8 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
std::pair<bool, size_t> fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
template <const char opening_bracket, const char closing_bracket>
static std::pair<bool, size_t> fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
{
skipWhitespaceIfAny(in);
@ -49,19 +53,19 @@ std::pair<bool, size_t> fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, D
}
else
{
pos = find_first_symbols<'{', '}', '\\', '"'>(pos, in.buffer().end());
pos = find_first_symbols<opening_bracket, closing_bracket, '\\', '"'>(pos, in.buffer().end());
if (pos > in.buffer().end())
throw Exception("Position in buffer is out of bounds. There must be a bug.", ErrorCodes::LOGICAL_ERROR);
else if (pos == in.buffer().end())
continue;
else if (*pos == '{')
else if (*pos == opening_bracket)
{
++balance;
++pos;
}
else if (*pos == '}')
else if (*pos == closing_bracket)
{
--balance;
++pos;
@ -87,6 +91,16 @@ std::pair<bool, size_t> fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, D
return {loadAtPosition(in, memory, pos), number_of_rows};
}
std::pair<bool, size_t> fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
{
return fileSegmentationEngineJSONEachRowImpl<'{', '}'>(in, memory, min_chunk_size);
}
std::pair<bool, size_t> fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
{
return fileSegmentationEngineJSONEachRowImpl<'[', ']'>(in, memory, min_chunk_size);
}
bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf)
{
/// For JSONEachRow we can safely skip whitespace characters
@ -94,4 +108,37 @@ bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf)
return buf.eof() || *buf.position() == '[';
}
bool readFieldImpl(ReadBuffer & in, IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, const String & column_name, const FormatSettings & format_settings, bool yield_strings)
{
try
{
bool as_nullable = format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable();
if (yield_strings)
{
String str;
readJSONString(str, in);
ReadBufferFromString buf(str);
if (as_nullable)
return SerializationNullable::deserializeWholeTextImpl(column, buf, format_settings, serialization);
serialization->deserializeWholeText(column, buf, format_settings);
return true;
}
if (as_nullable)
return SerializationNullable::deserializeTextJSONImpl(column, in, format_settings, serialization);
serialization->deserializeTextJSON(column, in, format_settings);
return true;
}
catch (Exception & e)
{
e.addMessage("(while reading the value of key " + column_name + ")");
throw;
}
}
}

View File

@ -1,10 +1,16 @@
#pragma once
#include <Formats/FormatSettings.h>
#include <IO/ReadBuffer.h>
namespace DB
{
std::pair<bool, size_t> fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size);
std::pair<bool, size_t> fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size);
std::pair<bool, size_t> fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size);
bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf);
bool readFieldImpl(ReadBuffer & in, IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, const String & column_name, const FormatSettings & format_settings, bool yield_strings);
}

View File

@ -15,6 +15,7 @@ void registerFileSegmentationEngineCSV(FormatFactory & factory);
void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory);
void registerFileSegmentationEngineRegexp(FormatFactory & factory);
void registerFileSegmentationEngineJSONAsString(FormatFactory & factory);
void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory);
/// Formats for both input/output.
@ -88,6 +89,7 @@ void registerFormats()
registerFileSegmentationEngineJSONEachRow(factory);
registerFileSegmentationEngineRegexp(factory);
registerFileSegmentationEngineJSONAsString(factory);
registerFileSegmentationEngineJSONCompactEachRow(factory);
registerInputFormatNative(factory);
registerOutputFormatNative(factory);

View File

@ -16,16 +16,11 @@ struct ColumnMapping
using OptionalIndexes = std::vector<std::optional<size_t>>;
OptionalIndexes column_indexes_for_input_fields;
/// Tracks which columns we have read in a single read() call.
/// For columns that are never read, it is initialized to false when we
/// read the file header, and never changed afterwards.
/// For other columns, it is updated on each read() call.
std::vector<UInt8> read_columns;
/// The list of column indexes that are not presented in input data.
std::vector<UInt8> not_presented_columns;
/// Whether we have any columns that are not read from file at all,
/// and must be always initialized with defaults.
bool have_always_default_columns{false};
/// The list of column names in input data. Needed for better exception messages.
std::vector<String> names_of_columns;
};
using ColumnMappingPtr = std::shared_ptr<ColumnMapping>;

View File

@ -113,4 +113,11 @@ void IRowOutputFormat::writeTotals(const DB::Columns & columns, size_t row_num)
write(columns, row_num);
}
void registerOutputFormatWithNamesAndTypes(const String & base_format_name, RegisterOutputFormatWithNamesAndTypes register_func)
{
register_func(base_format_name, false, false);
register_func(base_format_name + "WithNames", true, false);
register_func(base_format_name + "WithNamesAndTypes", true, true);
}
}

View File

@ -87,4 +87,7 @@ private:
};
using RegisterOutputFormatWithNamesAndTypes = std::function<void(const String & format_name, bool with_names, bool with_types)>;
void registerOutputFormatWithNamesAndTypes(const String & base_format_name, RegisterOutputFormatWithNamesAndTypes register_func);
}

View File

@ -19,63 +19,24 @@ namespace ErrorCodes
}
CSVRowInputFormat::CSVRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
bool with_names_, const FormatSettings & format_settings_)
: RowInputFormatWithDiagnosticInfo(header_, in_, params_)
, with_names(with_names_)
, format_settings(format_settings_)
CSVRowInputFormat::CSVRowInputFormat(
const Block & header_,
ReadBuffer & in_,
const Params & params_,
bool with_names_,
bool with_types_,
const FormatSettings & format_settings_)
: RowInputFormatWithNamesAndTypes(header_, in_, params_, with_names_, with_types_, format_settings_)
{
const String bad_delimiters = " \t\"'.UL";
if (bad_delimiters.find(format_settings.csv.delimiter) != String::npos)
throw Exception(String("CSV format may not work correctly with delimiter '") + format_settings.csv.delimiter +
"'. Try use CustomSeparated format instead.", ErrorCodes::BAD_ARGUMENTS);
const auto & sample = getPort().getHeader();
size_t num_columns = sample.columns();
data_types.resize(num_columns);
column_indexes_by_names.reserve(num_columns);
for (size_t i = 0; i < num_columns; ++i)
{
const auto & column_info = sample.getByPosition(i);
data_types[i] = column_info.type;
column_indexes_by_names.emplace(column_info.name, i);
}
}
/// Map an input file column to a table column, based on its name.
void CSVRowInputFormat::addInputColumn(const String & column_name)
{
const auto column_it = column_indexes_by_names.find(column_name);
if (column_it == column_indexes_by_names.end())
{
if (format_settings.skip_unknown_fields)
{
column_mapping->column_indexes_for_input_fields.push_back(std::nullopt);
return;
}
throw Exception(
"Unknown field found in CSV header: '" + column_name + "' " +
"at position " + std::to_string(column_mapping->column_indexes_for_input_fields.size()) +
"\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed",
ErrorCodes::INCORRECT_DATA
);
}
const auto column_index = column_it->second;
if (column_mapping->read_columns[column_index])
throw Exception("Duplicate field found while parsing CSV header: " + column_name, ErrorCodes::INCORRECT_DATA);
column_mapping->read_columns[column_index] = true;
column_mapping->column_indexes_for_input_fields.emplace_back(column_index);
String("CSV format may not work correctly with delimiter '") + format_settings.csv.delimiter
+ "'. Try use CustomSeparated format instead.",
ErrorCodes::BAD_ARGUMENTS);
}
static void skipEndOfLine(ReadBuffer & in)
{
/// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic)
@ -99,29 +60,6 @@ static void skipEndOfLine(ReadBuffer & in)
throw Exception("Expected end of line", ErrorCodes::INCORRECT_DATA);
}
static void skipDelimiter(ReadBuffer & in, const char delimiter, bool is_last_column)
{
if (is_last_column)
{
if (in.eof())
return;
/// we support the extra delimiter at the end of the line
if (*in.position() == delimiter)
{
++in.position();
if (in.eof())
return;
}
skipEndOfLine(in);
}
else
assertChar(delimiter, in);
}
/// Skip `whitespace` symbols allowed in CSV.
static inline void skipWhitespacesAndTabs(ReadBuffer & in)
{
@ -131,255 +69,138 @@ static inline void skipWhitespacesAndTabs(ReadBuffer & in)
++in.position();
}
static void skipRow(ReadBuffer & in, const FormatSettings::CSV & settings, size_t num_columns)
void CSVRowInputFormat::skipFieldDelimiter()
{
String tmp;
for (size_t i = 0; i < num_columns; ++i)
{
skipWhitespacesAndTabs(in);
readCSVString(tmp, in, settings);
skipWhitespacesAndTabs(in);
skipDelimiter(in, settings.delimiter, i + 1 == num_columns);
}
skipWhitespacesAndTabs(*in);
assertChar(format_settings.csv.delimiter, *in);
}
void CSVRowInputFormat::setupAllColumnsByTableSchema()
String CSVRowInputFormat::readFieldIntoString()
{
const auto & header = getPort().getHeader();
column_mapping->read_columns.assign(header.columns(), true);
column_mapping->column_indexes_for_input_fields.resize(header.columns());
for (size_t i = 0; i < column_mapping->column_indexes_for_input_fields.size(); ++i)
column_mapping->column_indexes_for_input_fields[i] = i;
skipWhitespacesAndTabs(*in);
String field;
readCSVString(field, *in, format_settings.csv);
return field;
}
void CSVRowInputFormat::readPrefix()
void CSVRowInputFormat::skipField()
{
/// In this format, we assume, that if first string field contain BOM as value, it will be written in quotes,
/// so BOM at beginning of stream cannot be confused with BOM in first string value, and it is safe to skip it.
skipBOMIfExists(*in);
size_t num_columns = data_types.size();
const auto & header = getPort().getHeader();
/// This is a bit of abstraction leakage, but we have almost the same code in other places.
/// Thus, we check if this InputFormat is working with the "real" beginning of the data in case of parallel parsing.
if (with_names && getCurrentUnitNumber() == 0)
{
/// This CSV file has a header row with column names. Depending on the
/// settings, use it or skip it.
if (format_settings.with_names_use_header)
{
/// Look at the file header to see which columns we have there.
/// The missing columns are filled with defaults.
column_mapping->read_columns.assign(header.columns(), false);
do
{
String column_name;
skipWhitespacesAndTabs(*in);
readCSVString(column_name, *in, format_settings.csv);
skipWhitespacesAndTabs(*in);
addInputColumn(column_name);
}
while (checkChar(format_settings.csv.delimiter, *in));
skipDelimiter(*in, format_settings.csv.delimiter, true);
for (auto read_column : column_mapping->read_columns)
{
if (!read_column)
{
column_mapping->have_always_default_columns = true;
break;
}
}
return;
}
else
{
skipRow(*in, format_settings.csv, num_columns);
setupAllColumnsByTableSchema();
}
}
else if (!column_mapping->is_set)
setupAllColumnsByTableSchema();
readFieldIntoString();
}
bool CSVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext)
void CSVRowInputFormat::skipRowEndDelimiter()
{
skipWhitespacesAndTabs(*in);
if (in->eof())
return false;
return;
updateDiagnosticInfo();
/// we support the extra delimiter at the end of the line
if (*in->position() == format_settings.csv.delimiter)
++in->position();
/// Track whether we have to fill any columns in this row with default
/// values. If not, we return an empty column mask to the caller, so that
/// it doesn't have to check it.
bool have_default_columns = column_mapping->have_always_default_columns;
skipWhitespacesAndTabs(*in);
if (in->eof())
return;
ext.read_columns.assign(column_mapping->read_columns.size(), true);
const auto delimiter = format_settings.csv.delimiter;
for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column)
{
const auto & table_column = column_mapping->column_indexes_for_input_fields[file_column];
const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size();
if (table_column)
{
skipWhitespacesAndTabs(*in);
ext.read_columns[*table_column] = readField(*columns[*table_column], data_types[*table_column],
serializations[*table_column], is_last_file_column);
if (!ext.read_columns[*table_column])
have_default_columns = true;
skipWhitespacesAndTabs(*in);
}
else
{
/// We never read this column from the file, just skip it.
String tmp;
readCSVString(tmp, *in, format_settings.csv);
}
skipDelimiter(*in, delimiter, is_last_file_column);
}
if (have_default_columns)
{
for (size_t i = 0; i < column_mapping->read_columns.size(); i++)
{
if (!column_mapping->read_columns[i])
{
/// The column value for this row is going to be overwritten
/// with default by the caller, but the general assumption is
/// that the column size increases for each row, so we have
/// to insert something. Since we do not care about the exact
/// value, we do not have to use the default value specified by
/// the data type, and can just use IColumn::insertDefault().
columns[i]->insertDefault();
ext.read_columns[i] = false;
}
}
}
return true;
skipEndOfLine(*in);
}
bool CSVRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out)
void CSVRowInputFormat::skipRow()
{
do
{
skipField();
skipWhitespacesAndTabs(*in);
}
while (checkChar(format_settings.csv.delimiter, *in));
skipRowEndDelimiter();
}
std::vector<String> CSVRowInputFormat::readHeaderRow()
{
std::vector<String> fields;
do
{
fields.push_back(readFieldIntoString());
skipWhitespacesAndTabs(*in);
}
while (checkChar(format_settings.csv.delimiter, *in));
skipRowEndDelimiter();
return fields;
}
bool CSVRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out)
{
const char delimiter = format_settings.csv.delimiter;
for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column)
try
{
if (file_column == 0 && in->eof())
{
out << "<End of stream>\n";
return false;
}
skipWhitespacesAndTabs(*in);
if (column_mapping->column_indexes_for_input_fields[file_column].has_value())
assertChar(delimiter, *in);
}
catch (const DB::Exception &)
{
if (*in->position() == '\n' || *in->position() == '\r')
{
const auto & header = getPort().getHeader();
size_t col_idx = column_mapping->column_indexes_for_input_fields[file_column].value();
if (!deserializeFieldAndPrintDiagnosticInfo(header.getByPosition(col_idx).name, data_types[col_idx], *columns[col_idx],
out, file_column))
return false;
out << "ERROR: Line feed found where delimiter (" << delimiter << ") is expected."
" It's like your file has less columns than expected.\n"
"And if your file has the right number of columns, maybe it has unescaped quotes in values.\n";
}
else
{
static const String skipped_column_str = "<SKIPPED COLUMN>";
static const DataTypePtr skipped_column_type = std::make_shared<DataTypeNothing>();
static const MutableColumnPtr skipped_column = skipped_column_type->createColumn();
if (!deserializeFieldAndPrintDiagnosticInfo(skipped_column_str, skipped_column_type, *skipped_column, out, file_column))
return false;
}
skipWhitespacesAndTabs(*in);
/// Delimiters
if (file_column + 1 == column_mapping->column_indexes_for_input_fields.size())
{
if (in->eof())
return false;
/// we support the extra delimiter at the end of the line
if (*in->position() == delimiter)
{
++in->position();
if (in->eof())
break;
}
if (!in->eof() && *in->position() != '\n' && *in->position() != '\r')
{
out << "ERROR: There is no line feed. ";
verbosePrintString(in->position(), in->position() + 1, out);
out << " found instead.\n"
" It's like your file has more columns than expected.\n"
"And if your file has the right number of columns, maybe it has an unquoted string value with a comma.\n";
return false;
}
skipEndOfLine(*in);
}
else
{
try
{
assertChar(delimiter, *in);
}
catch (const DB::Exception &)
{
if (*in->position() == '\n' || *in->position() == '\r')
{
out << "ERROR: Line feed found where delimiter (" << delimiter << ") is expected."
" It's like your file has less columns than expected.\n"
"And if your file has the right number of columns, maybe it has unescaped quotes in values.\n";
}
else
{
out << "ERROR: There is no delimiter (" << delimiter << "). ";
verbosePrintString(in->position(), in->position() + 1, out);
out << " found instead.\n";
}
return false;
}
out << "ERROR: There is no delimiter (" << delimiter << "). ";
verbosePrintString(in->position(), in->position() + 1, out);
out << " found instead.\n";
}
return false;
}
return true;
}
bool CSVRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out)
{
skipWhitespacesAndTabs(*in);
if (in->eof())
return true;
/// we support the extra delimiter at the end of the line
if (*in->position() == format_settings.csv.delimiter)
{
++in->position();
skipWhitespacesAndTabs(*in);
if (in->eof())
return true;
}
if (!in->eof() && *in->position() != '\n' && *in->position() != '\r')
{
out << "ERROR: There is no line feed. ";
verbosePrintString(in->position(), in->position() + 1, out);
out << " found instead.\n"
" It's like your file has more columns than expected.\n"
"And if your file has the right number of columns, maybe it has an unquoted string value with a comma.\n";
return false;
}
skipEndOfLine(*in);
return true;
}
void CSVRowInputFormat::syncAfterError()
{
skipToNextLineOrEOF(*in);
}
void CSVRowInputFormat::tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column)
bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & /*column_name*/)
{
const auto & index = column_mapping->column_indexes_for_input_fields[file_column];
if (index)
{
const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size();
readField(column, type, serializations[*index], is_last_file_column);
}
else
{
String tmp;
readCSVString(tmp, *in, format_settings.csv);
}
}
skipWhitespacesAndTabs(*in);
bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column)
{
const bool at_delimiter = !in->eof() && *in->position() == format_settings.csv.delimiter;
const bool at_last_column_line_end = is_last_file_column
&& (in->eof() || *in->position() == '\n' || *in->position() == '\r');
@ -398,7 +219,7 @@ bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, co
column.insertDefault();
return false;
}
else if (format_settings.null_as_default && !type->isNullable())
else if (format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable())
{
/// If value is null but type is not nullable then use default value instead.
return SerializationNullable::deserializeTextCSVImpl(column, *in, format_settings, serialization);
@ -411,28 +232,23 @@ bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, co
}
}
void CSVRowInputFormat::resetParser()
{
RowInputFormatWithDiagnosticInfo::resetParser();
column_mapping->column_indexes_for_input_fields.clear();
column_mapping->read_columns.clear();
column_mapping->have_always_default_columns = false;
}
void registerInputFormatCSV(FormatFactory & factory)
{
for (bool with_names : {false, true})
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
{
factory.registerInputFormat(with_names ? "CSVWithNames" : "CSV", [=](
factory.registerInputFormat(format_name, [with_names, with_types](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<CSVRowInputFormat>(sample, buf, params, with_names, settings);
return std::make_shared<CSVRowInputFormat>(sample, buf, std::move(params), with_names, with_types, settings);
});
}
};
registerInputFormatWithNamesAndTypes("CSV", register_func);
}
static std::pair<bool, size_t> fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
@ -501,8 +317,7 @@ static std::pair<bool, size_t> fileSegmentationEngineCSVImpl(ReadBuffer & in, DB
void registerFileSegmentationEngineCSV(FormatFactory & factory)
{
factory.registerFileSegmentationEngine("CSV", &fileSegmentationEngineCSVImpl);
factory.registerFileSegmentationEngine("CSVWithNames", &fileSegmentationEngineCSVImpl);
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "CSV", &fileSegmentationEngineCSVImpl);
}
}

View File

@ -4,7 +4,7 @@
#include <unordered_map>
#include <Core/Block.h>
#include <Processors/Formats/RowInputFormatWithDiagnosticInfo.h>
#include <Processors/Formats/RowInputFormatWithNamesAndTypes.h>
#include <Formats/FormatSettings.h>
@ -14,41 +14,39 @@ namespace DB
/** A stream for inputting data in csv format.
* Does not conform with https://tools.ietf.org/html/rfc4180 because it skips spaces and tabs between values.
*/
class CSVRowInputFormat : public RowInputFormatWithDiagnosticInfo
class CSVRowInputFormat : public RowInputFormatWithNamesAndTypes
{
public:
/** with_names - in the first line the header with column names
* with_types - on the next line header with type names
*/
CSVRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
bool with_names_, const FormatSettings & format_settings_);
bool with_names_, bool with_types_, const FormatSettings & format_settings_);
String getName() const override { return "CSVRowInputFormat"; }
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
void readPrefix() override;
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
void resetParser() override;
private:
/// There fields are computed in constructor.
bool with_names;
const FormatSettings format_settings;
DataTypes data_types;
using IndexesMap = std::unordered_map<String, size_t>;
IndexesMap column_indexes_by_names;
void addInputColumn(const String & column_name);
void setupAllColumnsByTableSchema();
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override;
void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override;
bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override;
bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override;
bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override
{
return *pos != '\n' && *pos != '\r' && *pos != format_settings.csv.delimiter && *pos != ' ' && *pos != '\t';
}
bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column);
bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override;
void skipField(const String & /*column_name*/) override { skipField(); }
void skipField();
void skipRow() override;
void skipFieldDelimiter() override;
void skipRowEndDelimiter() override;
Names readHeaderRow() override;
String readFieldIntoString();
};
}

View File

@ -8,8 +8,8 @@ namespace DB
{
CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_)
: IRowOutputFormat(header_, out_, params_), with_names(with_names_), format_settings(format_settings_)
CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, bool with_types_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_)
: IRowOutputFormat(header_, out_, params_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_)
{
const auto & sample = getPort(PortKind::Main).getHeader();
size_t columns = sample.columns();
@ -18,25 +18,27 @@ CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header_
data_types[i] = sample.safeGetByPosition(i).type;
}
void CSVRowOutputFormat::writeLine(const std::vector<String> & values)
{
for (size_t i = 0; i < values.size(); ++i)
{
writeCSVString(values[i], out);
if (i + 1 == values.size())
writeRowEndDelimiter();
else
writeFieldDelimiter();
}
}
void CSVRowOutputFormat::doWritePrefix()
{
const auto & sample = getPort(PortKind::Main).getHeader();
size_t columns = sample.columns();
if (with_names)
{
for (size_t i = 0; i < columns; ++i)
{
writeCSVString(sample.safeGetByPosition(i).name, out);
writeLine(sample.getNames());
char delimiter = format_settings.csv.delimiter;
if (i + 1 == columns)
delimiter = '\n';
writeChar(delimiter, out);
}
}
if (with_types)
writeLine(sample.getDataTypeNames());
}
@ -72,18 +74,20 @@ void CSVRowOutputFormat::writeBeforeExtremes()
void registerOutputFormatCSV(FormatFactory & factory)
{
for (bool with_names : {false, true})
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
{
factory.registerOutputFormat(with_names ? "CSVWithNames" : "CSV", [=](
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & format_settings)
factory.registerOutputFormat(format_name, [=](
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & format_settings)
{
return std::make_shared<CSVRowOutputFormat>(buf, sample, with_names, params, format_settings);
return std::make_shared<CSVRowOutputFormat>(buf, sample, with_names, with_types, params, format_settings);
});
factory.markOutputFormatSupportsParallelFormatting(with_names ? "CSVWithNames" : "CSV");
}
factory.markOutputFormatSupportsParallelFormatting(format_name);
};
registerOutputFormatWithNamesAndTypes("CSV", register_func);
}
}

View File

@ -20,7 +20,7 @@ public:
/** with_names - output in the first line a header with column names
* with_types - output in the next line header with the names of the types
*/
CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_);
CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, bool with_types, const RowOutputFormatParams & params_, const FormatSettings & format_settings_);
String getName() const override { return "CSVRowOutputFormat"; }
@ -38,9 +38,11 @@ public:
return String("text/csv; charset=UTF-8; header=") + (with_names ? "present" : "absent");
}
protected:
private:
void writeLine(const std::vector<String> & values);
bool with_names;
bool with_types;
const FormatSettings format_settings;
DataTypes data_types;
};

View File

@ -185,7 +185,7 @@ void registerInputFormatJSONAsString(FormatFactory & factory)
void registerFileSegmentationEngineJSONAsString(FormatFactory & factory)
{
factory.registerFileSegmentationEngine("JSONAsString", &fileSegmentationEngineJSONEachRowImpl);
factory.registerFileSegmentationEngine("JSONAsString", &fileSegmentationEngineJSONEachRow);
}
void registerNonTrivialPrefixAndSuffixCheckerJSONAsString(FormatFactory & factory)

View File

@ -1,8 +1,11 @@
#include <Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromString.h>
#include <Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h>
#include <IO/Operators.h>
#include <Formats/FormatFactory.h>
#include <Formats/verbosePrintString.h>
#include <Formats/JSONEachRowUtils.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/Serializations/SerializationNullable.h>
@ -16,179 +19,36 @@ namespace ErrorCodes
}
JSONCompactEachRowRowInputFormat::JSONCompactEachRowRowInputFormat(ReadBuffer & in_,
const Block & header_,
Params params_,
const FormatSettings & format_settings_,
bool with_names_,
bool yield_strings_)
: IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), with_names(with_names_), yield_strings(yield_strings_)
JSONCompactEachRowRowInputFormat::JSONCompactEachRowRowInputFormat(
const Block & header_,
ReadBuffer & in_,
Params params_,
bool with_names_,
bool with_types_,
bool yield_strings_,
const FormatSettings & format_settings_)
: RowInputFormatWithNamesAndTypes(header_, in_, std::move(params_), with_names_, with_types_, format_settings_)
, yield_strings(yield_strings_)
{
const auto & sample = getPort().getHeader();
size_t num_columns = sample.columns();
data_types.resize(num_columns);
column_indexes_by_names.reserve(num_columns);
for (size_t i = 0; i < num_columns; ++i)
{
const auto & column_info = sample.getByPosition(i);
data_types[i] = column_info.type;
column_indexes_by_names.emplace(column_info.name, i);
}
}
void JSONCompactEachRowRowInputFormat::resetParser()
void JSONCompactEachRowRowInputFormat::skipRowStartDelimiter()
{
IRowInputFormat::resetParser();
column_indexes_for_input_fields.clear();
not_seen_columns.clear();
}
void JSONCompactEachRowRowInputFormat::readPrefix()
{
/// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it.
skipBOMIfExists(*in);
if (with_names)
{
size_t num_columns = getPort().getHeader().columns();
read_columns.assign(num_columns, false);
assertChar('[', *in);
do
{
skipWhitespaceIfAny(*in);
String column_name;
readJSONString(column_name, *in);
addInputColumn(column_name);
skipWhitespaceIfAny(*in);
}
while (checkChar(',', *in));
assertChar(']', *in);
skipEndOfLine();
/// Type checking
assertChar('[', *in);
for (size_t i = 0; i < column_indexes_for_input_fields.size(); ++i)
{
skipWhitespaceIfAny(*in);
String data_type;
readJSONString(data_type, *in);
if (column_indexes_for_input_fields[i] &&
data_types[*column_indexes_for_input_fields[i]]->getName() != data_type)
{
throw Exception(
"Type of '" + getPort().getHeader().getByPosition(*column_indexes_for_input_fields[i]).name
+ "' must be " + data_types[*column_indexes_for_input_fields[i]]->getName() +
", not " + data_type,
ErrorCodes::INCORRECT_DATA
);
}
if (i != column_indexes_for_input_fields.size() - 1)
assertChar(',', *in);
skipWhitespaceIfAny(*in);
}
assertChar(']', *in);
}
else
{
size_t num_columns = getPort().getHeader().columns();
read_columns.assign(num_columns, true);
column_indexes_for_input_fields.resize(num_columns);
for (size_t i = 0; i < num_columns; ++i)
{
column_indexes_for_input_fields[i] = i;
}
}
for (size_t i = 0; i < read_columns.size(); ++i)
{
if (!read_columns[i])
{
not_seen_columns.emplace_back(i);
}
}
}
void JSONCompactEachRowRowInputFormat::addInputColumn(const String & column_name)
{
names_of_columns.emplace_back(column_name);
const auto column_it = column_indexes_by_names.find(column_name);
if (column_it == column_indexes_by_names.end())
{
if (format_settings.skip_unknown_fields)
{
column_indexes_for_input_fields.push_back(std::nullopt);
return;
}
throw Exception(
"Unknown field found in JSONCompactEachRow header: '" + column_name + "' " +
"at position " + std::to_string(column_indexes_for_input_fields.size()) +
"\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed",
ErrorCodes::INCORRECT_DATA
);
}
const auto column_index = column_it->second;
if (read_columns[column_index])
throw Exception("Duplicate field found while parsing JSONCompactEachRow header: " + column_name, ErrorCodes::INCORRECT_DATA);
read_columns[column_index] = true;
column_indexes_for_input_fields.emplace_back(column_index);
}
bool JSONCompactEachRowRowInputFormat::readRow(DB::MutableColumns &columns, DB::RowReadExtension &ext)
{
skipEndOfLine();
if (in->eof())
return false;
size_t num_columns = columns.size();
read_columns.assign(num_columns, false);
skipWhitespaceIfAny(*in);
assertChar('[', *in);
for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column)
{
const auto & table_column = column_indexes_for_input_fields[file_column];
if (table_column)
{
readField(*table_column, columns);
}
else
{
skipJSONField(*in, StringRef(names_of_columns[file_column]));
}
}
skipWhitespaceIfAny(*in);
if (in->eof())
throw ParsingException("Unexpected end of stream while parsing JSONCompactEachRow format", ErrorCodes::CANNOT_READ_ALL_DATA);
if (file_column + 1 != column_indexes_for_input_fields.size())
{
assertChar(',', *in);
skipWhitespaceIfAny(*in);
}
}
void JSONCompactEachRowRowInputFormat::skipFieldDelimiter()
{
skipWhitespaceIfAny(*in);
assertChar(',', *in);
}
void JSONCompactEachRowRowInputFormat::skipRowEndDelimiter()
{
skipWhitespaceIfAny(*in);
assertChar(']', *in);
for (const auto & name : not_seen_columns)
columns[name]->insertDefault();
ext.read_columns = read_columns;
return true;
}
void JSONCompactEachRowRowInputFormat::skipEndOfLine()
{
skipWhitespaceIfAny(*in);
if (!in->eof() && (*in->position() == ',' || *in->position() == ';'))
++in->position();
@ -196,39 +56,55 @@ void JSONCompactEachRowRowInputFormat::skipEndOfLine()
skipWhitespaceIfAny(*in);
}
void JSONCompactEachRowRowInputFormat::readField(size_t index, MutableColumns & columns)
String JSONCompactEachRowRowInputFormat::readFieldIntoString()
{
try
skipWhitespaceIfAny(*in);
String field;
readJSONString(field, *in);
return field;
}
void JSONCompactEachRowRowInputFormat::skipField(const String & column_name)
{
skipWhitespaceIfAny(*in);
skipJSONField(*in, column_name);
}
void JSONCompactEachRowRowInputFormat::skipRow()
{
skipRowStartDelimiter();
size_t i = 0;
do
{
read_columns[index] = true;
const auto & type = data_types[index];
const auto & serialization = serializations[index];
if (yield_strings)
{
String str;
readJSONString(str, *in);
ReadBufferFromString buf(str);
if (format_settings.null_as_default && !type->isNullable())
read_columns[index] = SerializationNullable::deserializeWholeTextImpl(*columns[index], buf, format_settings, serialization);
else
serialization->deserializeWholeText(*columns[index], buf, format_settings);
}
else
{
if (format_settings.null_as_default && !type->isNullable())
read_columns[index] = SerializationNullable::deserializeTextJSONImpl(*columns[index], *in, format_settings, serialization);
else
serialization->deserializeTextJSON(*columns[index], *in, format_settings);
}
if (i >= column_mapping->names_of_columns.size())
throw Exception(ErrorCodes::INCORRECT_DATA, "The number of columns in a row differs from the number of column names");
skipField(column_mapping->names_of_columns[i++]);
skipWhitespaceIfAny(*in);
}
catch (Exception & e)
while (checkChar(',', *in));
skipRowEndDelimiter();
}
std::vector<String> JSONCompactEachRowRowInputFormat::readHeaderRow()
{
skipRowStartDelimiter();
std::vector<String> fields;
do
{
e.addMessage("(while reading the value of key " + getPort().getHeader().getByPosition(index).name + ")");
throw;
fields.push_back(readFieldIntoString());
skipWhitespaceIfAny(*in);
}
while (checkChar(',', *in));
skipRowEndDelimiter();
return fields;
}
bool JSONCompactEachRowRowInputFormat::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool /*is_last_file_column*/, const String & column_name)
{
skipWhitespaceIfAny(*in);
return readFieldImpl(*in, column, type, serialization, column_name, format_settings, yield_strings);
}
void JSONCompactEachRowRowInputFormat::syncAfterError()
@ -236,43 +112,99 @@ void JSONCompactEachRowRowInputFormat::syncAfterError()
skipToUnescapedNextLineOrEOF(*in);
}
bool JSONCompactEachRowRowInputFormat::parseRowStartWithDiagnosticInfo(WriteBuffer & out)
{
skipWhitespaceIfAny(*in);
if (!checkChar('[', *in))
{
out << "ERROR: There is no '[' before the row.\n";
return false;
}
return true;
}
bool JSONCompactEachRowRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out)
{
try
{
skipWhitespaceIfAny(*in);
assertChar(',', *in);
}
catch (const DB::Exception &)
{
if (*in->position() == ']')
{
out << "ERROR: Closing parenthesis (']') found where comma is expected."
" It's like your file has less columns than expected.\n"
"And if your file has the right number of columns, maybe it has unescaped quotes in values.\n";
}
else
{
out << "ERROR: There is no comma. ";
verbosePrintString(in->position(), in->position() + 1, out);
out << " found instead.\n";
}
return false;
}
return true;
}
bool JSONCompactEachRowRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out)
{
skipWhitespaceIfAny(*in);
if (in->eof())
{
out << "ERROR: Unexpected end of file. ']' expected at the end of row.";
return false;
}
if (!checkChar(']', *in))
{
out << "ERROR: There is no closing parenthesis (']') at the end of the row. ";
verbosePrintString(in->position(), in->position() + 1, out);
out << " found instead.\n";
return false;
}
skipWhitespaceIfAny(*in);
if (in->eof())
return true;
if ((*in->position() == ',' || *in->position() == ';'))
++in->position();
skipWhitespaceIfAny(*in);
return true;
}
void registerInputFormatJSONCompactEachRow(FormatFactory & factory)
{
factory.registerInputFormat("JSONCompactEachRow", [](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
for (bool yield_strings : {true, false})
{
return std::make_shared<JSONCompactEachRowRowInputFormat>(buf, sample, std::move(params), settings, false, false);
});
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
{
factory.registerInputFormat(format_name, [with_names, with_types, yield_strings](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<JSONCompactEachRowRowInputFormat>(sample, buf, std::move(params), with_names, with_types, yield_strings, settings);
});
};
factory.registerInputFormat("JSONCompactEachRowWithNamesAndTypes", [](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<JSONCompactEachRowRowInputFormat>(buf, sample, std::move(params), settings, true, false);
});
registerInputFormatWithNamesAndTypes(yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", register_func);
}
}
factory.registerInputFormat("JSONCompactStringsEachRow", [](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<JSONCompactEachRowRowInputFormat>(buf, sample, std::move(params), settings, false, true);
});
factory.registerInputFormat("JSONCompactStringsEachRowWithNamesAndTypes", [](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<JSONCompactEachRowRowInputFormat>(buf, sample, std::move(params), settings, true, true);
});
void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory)
{
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "JSONCompactEachRow", &fileSegmentationEngineJSONCompactEachRow);
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "JSONCompactStringsEachRow", &fileSegmentationEngineJSONCompactEachRow);
}
}

View File

@ -1,7 +1,7 @@
#pragma once
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/RowInputFormatWithNamesAndTypes.h>
#include <Formats/FormatSettings.h>
#include <Common/HashTable/HashMap.h>
@ -17,49 +17,43 @@ class ReadBuffer;
* - JSONCompactStringsEachRowWithNamesAndTypes
*
*/
class JSONCompactEachRowRowInputFormat : public IRowInputFormat
class JSONCompactEachRowRowInputFormat : public RowInputFormatWithNamesAndTypes
{
public:
JSONCompactEachRowRowInputFormat(
ReadBuffer & in_,
const Block & header_,
ReadBuffer & in_,
Params params_,
const FormatSettings & format_settings_,
bool with_names_,
bool yield_strings_);
bool with_types_,
bool yield_strings_,
const FormatSettings & format_settings_);
String getName() const override { return "JSONCompactEachRowRowInputFormat"; }
void readPrefix() override;
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
void resetParser() override;
private:
void addInputColumn(const String & column_name);
void skipEndOfLine();
void readField(size_t index, MutableColumns & columns);
bool parseRowStartWithDiagnosticInfo(WriteBuffer & out) override;
bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override;
bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override;
bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override
{
return *pos != ',' && *pos != ']' && *pos != ' ' && *pos != '\t';
}
const FormatSettings format_settings;
bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override;
using IndexesMap = std::unordered_map<String, size_t>;
IndexesMap column_indexes_by_names;
void skipField(const String & column_name) override;
void skipRow() override;
void skipRowStartDelimiter() override;
void skipFieldDelimiter() override;
void skipRowEndDelimiter() override;
using OptionalIndexes = std::vector<std::optional<size_t>>;
OptionalIndexes column_indexes_for_input_fields;
Names readHeaderRow() override;
String readFieldIntoString();
DataTypes data_types;
std::vector<UInt8> read_columns;
std::vector<size_t> not_seen_columns;
/// This is for the correct exceptions in skipping unknown fields.
std::vector<String> names_of_columns;
/// For *WithNamesAndTypes formats.
bool with_names;
/// For JSONCompactString* formats.
bool yield_strings;
};

View File

@ -13,12 +13,10 @@ JSONCompactEachRowRowOutputFormat::JSONCompactEachRowRowOutputFormat(WriteBuffer
const RowOutputFormatParams & params_,
const FormatSettings & settings_,
bool with_names_,
bool with_types_,
bool yield_strings_)
: IRowOutputFormat(header_, out_, params_), settings(settings_), with_names(with_names_), yield_strings(yield_strings_)
: IRowOutputFormat(header_, out_, params_), settings(settings_), with_names(with_names_), with_types(with_types_), yield_strings(yield_strings_)
{
const auto & sample = getPort(PortKind::Main).getHeader();
NamesAndTypesList columns(sample.getNamesAndTypesList());
fields.assign(columns.begin(), columns.end());
}
@ -57,39 +55,40 @@ void JSONCompactEachRowRowOutputFormat::writeTotals(const Columns & columns, siz
{
writeChar('\n', out);
size_t num_columns = columns.size();
writeChar('[', out);
writeRowStartDelimiter();
for (size_t i = 0; i < num_columns; ++i)
{
if (i != 0)
JSONCompactEachRowRowOutputFormat::writeFieldDelimiter();
writeFieldDelimiter();
JSONCompactEachRowRowOutputFormat::writeField(*columns[i], *serializations[i], row_num);
writeField(*columns[i], *serializations[i], row_num);
}
writeCString("]\n", out);
writeRowEndDelimiter();
}
void JSONCompactEachRowRowOutputFormat::writeLine(const std::vector<String> & values)
{
writeRowStartDelimiter();
for (size_t i = 0; i < values.size(); ++i)
{
writeChar('\"', out);
writeString(values[i], out);
writeChar('\"', out);
if (i != values.size() - 1)
writeFieldDelimiter();
}
writeRowEndDelimiter();
}
void JSONCompactEachRowRowOutputFormat::doWritePrefix()
{
const auto & header = getPort(PortKind::Main).getHeader();
if (with_names)
{
writeChar('[', out);
for (size_t i = 0; i < fields.size(); ++i)
{
writeChar('\"', out);
writeString(fields[i].name, out);
writeChar('\"', out);
if (i != fields.size() - 1)
writeCString(", ", out);
}
writeCString("]\n[", out);
for (size_t i = 0; i < fields.size(); ++i)
{
writeJSONString(fields[i].type->getName(), out, settings);
if (i != fields.size() - 1)
writeCString(", ", out);
}
writeCString("]\n", out);
}
writeLine(header.getNames());
if (with_types)
writeLine(header.getDataTypeNames());
}
void JSONCompactEachRowRowOutputFormat::consumeTotals(DB::Chunk chunk)
@ -100,45 +99,23 @@ void JSONCompactEachRowRowOutputFormat::consumeTotals(DB::Chunk chunk)
void registerOutputFormatJSONCompactEachRow(FormatFactory & factory)
{
factory.registerOutputFormat("JSONCompactEachRow", [](
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & format_settings)
for (bool yield_strings : {false, true})
{
return std::make_shared<JSONCompactEachRowRowOutputFormat>(buf, sample, params, format_settings, false, false);
});
factory.markOutputFormatSupportsParallelFormatting("JSONCompactEachRow");
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
{
factory.registerOutputFormat(format_name, [=](
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & format_settings)
{
return std::make_shared<JSONCompactEachRowRowOutputFormat>(buf, sample, params, format_settings, with_names, with_types, yield_strings);
});
factory.markOutputFormatSupportsParallelFormatting(format_name);
};
factory.registerOutputFormat("JSONCompactEachRowWithNamesAndTypes", [](
WriteBuffer &buf,
const Block &sample,
const RowOutputFormatParams & params,
const FormatSettings &format_settings)
{
return std::make_shared<JSONCompactEachRowRowOutputFormat>(buf, sample, params, format_settings, true, false);
});
factory.markOutputFormatSupportsParallelFormatting("JSONCompactEachRowWithNamesAndTypes");
factory.registerOutputFormat("JSONCompactStringsEachRow", [](
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & format_settings)
{
return std::make_shared<JSONCompactEachRowRowOutputFormat>(buf, sample, params, format_settings, false, true);
});
factory.markOutputFormatSupportsParallelFormatting("JSONCompactStringsEachRow");
factory.registerOutputFormat("JSONCompactStringsEachRowWithNamesAndTypes", [](
WriteBuffer &buf,
const Block &sample,
const RowOutputFormatParams & params,
const FormatSettings &format_settings)
{
return std::make_shared<JSONCompactEachRowRowOutputFormat>(buf, sample, params, format_settings, true, true);
});
factory.markOutputFormatSupportsParallelFormatting("JSONCompactStringsEachRowWithNamesAndTypes");
registerOutputFormatWithNamesAndTypes(yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", register_func);
}
}

View File

@ -21,15 +21,14 @@ public:
const RowOutputFormatParams & params_,
const FormatSettings & settings_,
bool with_names_,
bool with_types_,
bool yield_strings_);
String getName() const override { return "JSONCompactEachRowRowOutputFormat"; }
void doWritePrefix() override;
void writeBeforeTotals() override {}
void writeTotals(const Columns & columns, size_t row_num) override;
void writeAfterTotals() override {}
void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override;
void writeFieldDelimiter() override;
@ -42,11 +41,11 @@ protected:
void consumeExtremes(Chunk) override {}
private:
void writeLine(const std::vector<String> & values);
FormatSettings settings;
NamesAndTypes fields;
bool with_names;
bool with_types;
bool yield_strings;
};
}

View File

@ -136,37 +136,10 @@ void JSONEachRowRowInputFormat::readField(size_t index, MutableColumns & columns
if (seen_columns[index])
throw Exception("Duplicate field found while parsing JSONEachRow format: " + columnName(index), ErrorCodes::INCORRECT_DATA);
try
{
seen_columns[index] = read_columns[index] = true;
const auto & type = getPort().getHeader().getByPosition(index).type;
const auto & serialization = serializations[index];
if (yield_strings)
{
String str;
readJSONString(str, *in);
ReadBufferFromString buf(str);
if (format_settings.null_as_default && !type->isNullable())
read_columns[index] = SerializationNullable::deserializeWholeTextImpl(*columns[index], buf, format_settings, serialization);
else
serialization->deserializeWholeText(*columns[index], buf, format_settings);
}
else
{
if (format_settings.null_as_default && !type->isNullable())
read_columns[index] = SerializationNullable::deserializeTextJSONImpl(*columns[index], *in, format_settings, serialization);
else
serialization->deserializeTextJSON(*columns[index], *in, format_settings);
}
}
catch (Exception & e)
{
e.addMessage("(while reading the value of key " + columnName(index) + ")");
throw;
}
seen_columns[index] = true;
const auto & type = getPort().getHeader().getByPosition(index).type;
const auto & serialization = serializations[index];
read_columns[index] = readFieldImpl(*in, *columns[index], type, serialization, columnName(index), format_settings, yield_strings);
}
inline bool JSONEachRowRowInputFormat::advanceToNextKey(size_t key_index)
@ -282,8 +255,13 @@ bool JSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi
if (!seen_columns[i])
header.getByPosition(i).type->insertDefaultInto(*columns[i]);
/// return info about defaults set
ext.read_columns = read_columns;
/// Return info about defaults set.
/// If defaults_for_omitted_fields is set to 0, we should just leave already inserted defaults.
if (format_settings.defaults_for_omitted_fields)
ext.read_columns = read_columns;
else
ext.read_columns.assign(read_columns.size(), true);
return true;
}
@ -355,8 +333,8 @@ void registerInputFormatJSONEachRow(FormatFactory & factory)
void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory)
{
factory.registerFileSegmentationEngine("JSONEachRow", &fileSegmentationEngineJSONEachRowImpl);
factory.registerFileSegmentationEngine("JSONStringsEachRow", &fileSegmentationEngineJSONEachRowImpl);
factory.registerFileSegmentationEngine("JSONEachRow", &fileSegmentationEngineJSONEachRow);
factory.registerFileSegmentationEngine("JSONStringsEachRow", &fileSegmentationEngineJSONEachRow);
}
void registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(FormatFactory & factory)

View File

@ -60,7 +60,7 @@ RegexpRowInputFormat::ColumnFormat RegexpRowInputFormat::stringToFormat(const St
bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns)
{
const auto & type = getPort().getHeader().getByPosition(index).type;
bool parse_as_nullable = format_settings.null_as_default && !type->isNullable();
bool parse_as_nullable = format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable();
bool read = true;
ReadBuffer field_buf(const_cast<char *>(matched_fields[index].data()), matched_fields[index].size(), 0);
try
@ -94,9 +94,9 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns)
break;
case ColumnFormat::Raw:
if (parse_as_nullable)
read = SerializationNullable::deserializeWholeTextImpl(*columns[index], field_buf, format_settings, serialization);
read = SerializationNullable::deserializeTextRawImpl(*columns[index], field_buf, format_settings, serialization);
else
serialization->deserializeWholeText(*columns[index], field_buf, format_settings);
serialization->deserializeTextRaw(*columns[index], field_buf, format_settings);
break;
default:
break;

View File

@ -143,7 +143,7 @@ bool TSKVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ex
seen_columns[index] = read_columns[index] = true;
const auto & type = getPort().getHeader().getByPosition(index).type;
const auto & serialization = serializations[index];
if (format_settings.null_as_default && !type->isNullable())
if (format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable())
read_columns[index] = SerializationNullable::deserializeTextEscapedImpl(*columns[index], *in, format_settings, serialization);
else
serialization->deserializeTextEscaped(*columns[index], *in, format_settings);

View File

@ -8,7 +8,7 @@ namespace DB
{
TSKVRowOutputFormat::TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const RowOutputFormatParams & params_, const FormatSettings & format_settings_)
: TabSeparatedRowOutputFormat(out_, header, false, false, params_, format_settings_)
: TabSeparatedRowOutputFormat(out_, header, false, false, false, params_, format_settings_)
{
const auto & sample = getPort(PortKind::Main).getHeader();
NamesAndTypesList columns(sample.getNamesAndTypesList());

View File

@ -1,58 +0,0 @@
#pragma once
#include <Core/Block.h>
#include <Formats/FormatSettings.h>
#include <IO/ReadBufferFromString.h>
#include <Processors/Formats/RowInputFormatWithDiagnosticInfo.h>
namespace DB
{
/** A stream to input data in tsv format, but without escaping individual values.
* It only supports columns without '\n' or '\t'
*/
class TabSeparatedRawRowInputFormat : public TabSeparatedRowInputFormat
{
public:
/** with_names - the first line is the header with the names of the columns
* with_types - on the next line header with type names
*/
TabSeparatedRawRowInputFormat(
const Block & header_,
ReadBuffer & in_,
const Params & params_,
bool with_names_,
bool with_types_,
const FormatSettings & format_settings_)
: TabSeparatedRowInputFormat(header_, in_, params_, with_names_, with_types_, format_settings_)
{
}
String getName() const override { return "TabSeparatedRawRowInputFormat"; }
bool readField(IColumn & column, const DataTypePtr &, const SerializationPtr & serialization, bool) override
{
String tmp;
while (!in->eof())
{
char * pos = find_first_symbols<'\n', '\t'>(in->position(), in->buffer().end());
tmp.append(in->position(), pos - in->position());
in->position() = pos;
if (pos == in->buffer().end())
in->next();
else
break;
}
ReadBufferFromString cell(tmp);
serialization->deserializeWholeText(column, cell, format_settings);
return true;
}
};
}

View File

@ -1,35 +0,0 @@
#pragma once
#include <Formats/FormatSettings.h>
#include <Processors/Formats/Impl/TabSeparatedRowOutputFormat.h>
namespace DB
{
/** A stream for outputting data in tsv format, but without escaping individual values.
* (That is, the output is irreversible.)
*/
class TabSeparatedRawRowOutputFormat : public TabSeparatedRowOutputFormat
{
public:
TabSeparatedRawRowOutputFormat(
WriteBuffer & out_,
const Block & header_,
bool with_names_,
bool with_types_,
const RowOutputFormatParams & params_,
const FormatSettings & format_settings_)
: TabSeparatedRowOutputFormat(out_, header_, with_names_, with_types_, params_, format_settings_)
{
}
String getName() const override { return "TabSeparatedRawRowOutputFormat"; }
void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override
{
serialization.serializeText(column, row_num, out, format_settings);
}
};
}

View File

@ -1,9 +1,8 @@
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/ReadBufferFromString.h>
#include <IO/Operators.h>
#include <Processors/Formats/Impl/TabSeparatedRowInputFormat.h>
#include <Processors/Formats/Impl/TabSeparatedRawRowInputFormat.h>
#include <Formats/verbosePrintString.h>
#include <Formats/FormatFactory.h>
#include <DataTypes/DataTypeNothing.h>
@ -19,19 +18,6 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
static void skipTSVRow(ReadBuffer & in, const size_t num_columns)
{
NullOutput null_sink;
for (size_t i = 0; i < num_columns; ++i)
{
readEscapedStringInto(null_sink, in);
assertChar(i == num_columns - 1 ? '\n' : '\t', in);
}
}
/** Check for a common error case - usage of Windows line feed.
*/
static void checkForCarriageReturn(ReadBuffer & in)
@ -45,187 +31,74 @@ static void checkForCarriageReturn(ReadBuffer & in)
}
TabSeparatedRowInputFormat::TabSeparatedRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
bool with_names_, bool with_types_, const FormatSettings & format_settings_)
: RowInputFormatWithDiagnosticInfo(header_, in_, params_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_)
TabSeparatedRowInputFormat::TabSeparatedRowInputFormat(
const Block & header_,
ReadBuffer & in_,
const Params & params_,
bool with_names_,
bool with_types_,
bool is_raw_,
const FormatSettings & format_settings_)
: RowInputFormatWithNamesAndTypes(header_, in_, params_, with_names_, with_types_, format_settings_), is_raw(is_raw_)
{
const auto & sample = getPort().getHeader();
size_t num_columns = sample.columns();
data_types.resize(num_columns);
column_indexes_by_names.reserve(num_columns);
for (size_t i = 0; i < num_columns; ++i)
{
const auto & column_info = sample.getByPosition(i);
data_types[i] = column_info.type;
column_indexes_by_names.emplace(column_info.name, i);
}
column_mapping->column_indexes_for_input_fields.reserve(num_columns);
column_mapping->read_columns.assign(num_columns, false);
}
void TabSeparatedRowInputFormat::setupAllColumnsByTableSchema()
void TabSeparatedRowInputFormat::skipFieldDelimiter()
{
const auto & header = getPort().getHeader();
column_mapping->read_columns.assign(header.columns(), true);
column_mapping->column_indexes_for_input_fields.resize(header.columns());
for (size_t i = 0; i < column_mapping->column_indexes_for_input_fields.size(); ++i)
column_mapping->column_indexes_for_input_fields[i] = i;
assertChar('\t', *in);
}
void TabSeparatedRowInputFormat::addInputColumn(const String & column_name)
{
const auto column_it = column_indexes_by_names.find(column_name);
if (column_it == column_indexes_by_names.end())
{
if (format_settings.skip_unknown_fields)
{
column_mapping->column_indexes_for_input_fields.push_back(std::nullopt);
return;
}
throw Exception(
"Unknown field found in TSV header: '" + column_name + "' " +
"at position " + std::to_string(column_mapping->column_indexes_for_input_fields.size()) +
"\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed",
ErrorCodes::INCORRECT_DATA
);
}
const auto column_index = column_it->second;
if (column_mapping->read_columns[column_index])
throw Exception("Duplicate field found while parsing TSV header: " + column_name, ErrorCodes::INCORRECT_DATA);
column_mapping->read_columns[column_index] = true;
column_mapping->column_indexes_for_input_fields.emplace_back(column_index);
}
void TabSeparatedRowInputFormat::fillUnreadColumnsWithDefaults(MutableColumns & columns, RowReadExtension & row_read_extension)
{
/// It is safe to memorize this on the first run - the format guarantees this does not change
if (unlikely(row_num == 1))
{
columns_to_fill_with_default_values.clear();
for (size_t index = 0; index < column_mapping->read_columns.size(); ++index)
if (column_mapping->read_columns[index] == 0)
columns_to_fill_with_default_values.push_back(index);
}
for (const auto column_index : columns_to_fill_with_default_values)
{
data_types[column_index]->insertDefaultInto(*columns[column_index]);
row_read_extension.read_columns[column_index] = false;
}
}
void TabSeparatedRowInputFormat::readPrefix()
{
if (with_names || with_types || data_types.at(0)->textCanContainOnlyValidUTF8())
{
/// In this format, we assume that column name or type cannot contain BOM,
/// so, if format has header,
/// then BOM at beginning of stream cannot be confused with name or type of field, and it is safe to skip it.
skipBOMIfExists(*in);
}
/// This is a bit of abstraction leakage, but we have almost the same code in other places.
/// Thus, we check if this InputFormat is working with the "real" beginning of the data in case of parallel parsing.
if (with_names && getCurrentUnitNumber() == 0)
{
if (format_settings.with_names_use_header)
{
String column_name;
for (;;)
{
readEscapedString(column_name, *in);
if (!checkChar('\t', *in))
{
/// Check last column for \r before adding it, otherwise an error will be:
/// "Unknown field found in TSV header"
checkForCarriageReturn(*in);
addInputColumn(column_name);
break;
}
else
addInputColumn(column_name);
}
if (!in->eof())
{
assertChar('\n', *in);
}
}
else
{
setupAllColumnsByTableSchema();
skipTSVRow(*in, column_mapping->column_indexes_for_input_fields.size());
}
}
else if (!column_mapping->is_set)
setupAllColumnsByTableSchema();
if (with_types)
{
skipTSVRow(*in, column_mapping->column_indexes_for_input_fields.size());
}
}
bool TabSeparatedRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext)
void TabSeparatedRowInputFormat::skipRowEndDelimiter()
{
if (in->eof())
return false;
return;
updateDiagnosticInfo();
if (unlikely(row_num <= 1))
checkForCarriageReturn(*in);
ext.read_columns.assign(column_mapping->read_columns.size(), true);
for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column)
assertChar('\n', *in);
}
String TabSeparatedRowInputFormat::readFieldIntoString()
{
String field;
readEscapedString(field, *in);
return field;
}
void TabSeparatedRowInputFormat::skipField()
{
NullOutput null_sink;
readEscapedStringInto(null_sink, *in);
}
void TabSeparatedRowInputFormat::skipRow()
{
do
{
const auto & column_index = column_mapping->column_indexes_for_input_fields[file_column];
const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size();
if (column_index)
{
const auto & type = data_types[*column_index];
ext.read_columns[*column_index] = readField(*columns[*column_index], type, serializations[*column_index], is_last_file_column);
}
else
{
NullOutput null_sink;
readEscapedStringInto(null_sink, *in);
}
/// skip separators
if (file_column + 1 < column_mapping->column_indexes_for_input_fields.size())
{
assertChar('\t', *in);
}
else if (!in->eof())
{
if (unlikely(row_num == 1))
checkForCarriageReturn(*in);
assertChar('\n', *in);
}
skipField();
}
while (checkChar('\t', *in));
fillUnreadColumnsWithDefaults(columns, ext);
skipRowEndDelimiter();
}
return true;
std::vector<String> TabSeparatedRowInputFormat::readHeaderRow()
{
std::vector<String> fields;
do
{
fields.push_back(readFieldIntoString());
}
while (checkChar('\t', *in));
skipRowEndDelimiter();
return fields;
}
bool TabSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr & type,
const SerializationPtr & serialization, bool is_last_file_column)
const SerializationPtr & serialization, bool is_last_file_column, const String & /*column_name*/)
{
const bool at_delimiter = !is_last_file_column && !in->eof() && *in->position() == '\t';
const bool at_last_column_line_end = is_last_file_column && (in->eof() || *in->position() == '\n');
@ -235,137 +108,112 @@ bool TabSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr &
column.insertDefault();
return false;
}
else if (format_settings.null_as_default && !type->isNullable())
bool as_nullable = format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable();
if (is_raw)
{
if (as_nullable)
return SerializationNullable::deserializeTextRawImpl(column, *in, format_settings, serialization);
serialization->deserializeTextRaw(column, *in, format_settings);
return true;
}
if (as_nullable)
return SerializationNullable::deserializeTextEscapedImpl(column, *in, format_settings, serialization);
serialization->deserializeTextEscaped(column, *in, format_settings);
return true;
}
bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out)
bool TabSeparatedRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out)
{
for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column)
try
{
if (file_column == 0 && in->eof())
assertChar('\t', *in);
}
catch (const DB::Exception &)
{
if (*in->position() == '\n')
{
out << "<End of stream>\n";
return false;
out << "ERROR: Line feed found where tab is expected."
" It's like your file has less columns than expected.\n"
"And if your file has the right number of columns, "
"maybe it has an unescaped backslash in value before tab, which causes the tab to be escaped.\n";
}
if (column_mapping->column_indexes_for_input_fields[file_column].has_value())
else if (*in->position() == '\r')
{
const auto & header = getPort().getHeader();
size_t col_idx = column_mapping->column_indexes_for_input_fields[file_column].value();
if (!deserializeFieldAndPrintDiagnosticInfo(header.getByPosition(col_idx).name, data_types[col_idx], *columns[col_idx],
out, file_column))
return false;
out << "ERROR: Carriage return found where tab is expected.\n";
}
else
{
static const String skipped_column_str = "<SKIPPED COLUMN>";
static const DataTypePtr skipped_column_type = std::make_shared<DataTypeNothing>();
static const MutableColumnPtr skipped_column = skipped_column_type->createColumn();
if (!deserializeFieldAndPrintDiagnosticInfo(skipped_column_str, skipped_column_type, *skipped_column, out, file_column))
return false;
}
/// Delimiters
if (file_column + 1 == column_mapping->column_indexes_for_input_fields.size())
{
if (!in->eof())
{
try
{
assertChar('\n', *in);
}
catch (const DB::Exception &)
{
if (*in->position() == '\t')
{
out << "ERROR: Tab found where line feed is expected."
" It's like your file has more columns than expected.\n"
"And if your file has the right number of columns, maybe it has an unescaped tab in a value.\n";
}
else if (*in->position() == '\r')
{
out << "ERROR: Carriage return found where line feed is expected."
" It's like your file has DOS/Windows style line separators, that is illegal in TabSeparated format.\n";
}
else
{
out << "ERROR: There is no line feed. ";
verbosePrintString(in->position(), in->position() + 1, out);
out << " found instead.\n";
}
return false;
}
}
}
else
{
try
{
assertChar('\t', *in);
}
catch (const DB::Exception &)
{
if (*in->position() == '\n')
{
out << "ERROR: Line feed found where tab is expected."
" It's like your file has less columns than expected.\n"
"And if your file has the right number of columns, "
"maybe it has an unescaped backslash in value before tab, which causes the tab to be escaped.\n";
}
else if (*in->position() == '\r')
{
out << "ERROR: Carriage return found where tab is expected.\n";
}
else
{
out << "ERROR: There is no tab. ";
verbosePrintString(in->position(), in->position() + 1, out);
out << " found instead.\n";
}
return false;
}
out << "ERROR: There is no tab. ";
verbosePrintString(in->position(), in->position() + 1, out);
out << " found instead.\n";
}
return false;
}
return true;
}
void TabSeparatedRowInputFormat::tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column)
bool TabSeparatedRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out)
{
const auto & index = column_mapping->column_indexes_for_input_fields[file_column];
if (index)
{
bool can_be_parsed_as_null = removeLowCardinality(type)->isNullable();
if (in->eof())
return true;
// check null value for type is not nullable. don't cross buffer bound for simplicity, so maybe missing some case
if (!can_be_parsed_as_null && !in->eof())
try
{
assertChar('\n', *in);
}
catch (const DB::Exception &)
{
if (*in->position() == '\t')
{
if (*in->position() == '\\' && in->available() >= 2)
out << "ERROR: Tab found where line feed is expected."
" It's like your file has more columns than expected.\n"
"And if your file has the right number of columns, maybe it has an unescaped tab in a value.\n";
}
else if (*in->position() == '\r')
{
out << "ERROR: Carriage return found where line feed is expected."
" It's like your file has DOS/Windows style line separators, that is illegal in TabSeparated format.\n";
}
else
{
out << "ERROR: There is no line feed. ";
verbosePrintString(in->position(), in->position() + 1, out);
out << " found instead.\n";
}
return false;
}
return true;
}
void TabSeparatedRowInputFormat::checkNullValueForNonNullable(DataTypePtr type)
{
bool can_be_parsed_as_null = removeLowCardinality(type)->isNullable() || format_settings.null_as_default;
// check null value for type is not nullable. don't cross buffer bound for simplicity, so maybe missing some case
if (!can_be_parsed_as_null && !in->eof())
{
if (*in->position() == '\\' && in->available() >= 2)
{
++in->position();
if (*in->position() == 'N')
{
++in->position();
if (*in->position() == 'N')
{
++in->position();
throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected NULL value of not Nullable type {}", type->getName());
}
else
{
--in->position();
}
throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected NULL value of not Nullable type {}", type->getName());
}
else
{
--in->position();
}
}
const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size();
readField(column, type, serializations[*index], is_last_file_column);
}
else
{
NullOutput null_sink;
readEscapedStringInto(null_sink, *in);
}
}
@ -374,66 +222,28 @@ void TabSeparatedRowInputFormat::syncAfterError()
skipToUnescapedNextLineOrEOF(*in);
}
void TabSeparatedRowInputFormat::resetParser()
{
RowInputFormatWithDiagnosticInfo::resetParser();
const auto & sample = getPort().getHeader();
column_mapping->read_columns.assign(sample.columns(), false);
column_mapping->column_indexes_for_input_fields.clear();
columns_to_fill_with_default_values.clear();
}
void registerInputFormatTabSeparated(FormatFactory & factory)
{
for (const auto * name : {"TabSeparated", "TSV"})
for (bool is_raw : {false, true})
{
factory.registerInputFormat(name, [](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
{
return std::make_shared<TabSeparatedRowInputFormat>(sample, buf, params, false, false, settings);
});
}
factory.registerInputFormat(format_name, [with_names, with_types, is_raw](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRowInputFormat>(sample, buf, std::move(params), with_names, with_types, is_raw, settings);
});
};
for (const auto * name : {"TabSeparatedRaw", "TSVRaw"})
{
factory.registerInputFormat(name, [](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRawRowInputFormat>(sample, buf, params, false, false, settings);
});
}
for (const auto * name : {"TabSeparatedWithNames", "TSVWithNames"})
{
factory.registerInputFormat(name, [](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRowInputFormat>(sample, buf, params, true, false, settings);
});
}
for (const auto * name : {"TabSeparatedWithNamesAndTypes", "TSVWithNamesAndTypes"})
{
factory.registerInputFormat(name, [](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRowInputFormat>(sample, buf, params, true, true, settings);
});
registerInputFormatWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func);
registerInputFormatWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func);
}
}
template <bool is_raw>
static std::pair<bool, size_t> fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
{
bool need_more_data = true;
@ -442,13 +252,18 @@ static std::pair<bool, size_t> fileSegmentationEngineTabSeparatedImpl(ReadBuffer
while (loadAtPosition(in, memory, pos) && need_more_data)
{
pos = find_first_symbols<'\\', '\r', '\n'>(pos, in.buffer().end());
if constexpr (is_raw)
pos = find_first_symbols<'\r', '\n'>(pos, in.buffer().end());
else
pos = find_first_symbols<'\\', '\r', '\n'>(pos, in.buffer().end());
if (pos > in.buffer().end())
throw Exception("Position in buffer is out of bounds. There must be a bug.", ErrorCodes::LOGICAL_ERROR);
else if (pos == in.buffer().end())
throw Exception("Position in buffer is out of bounds. There must be a bug.", ErrorCodes::LOGICAL_ERROR);
if (pos == in.buffer().end())
continue;
else if (*pos == '\\')
if (!is_raw && *pos == '\\')
{
++pos;
if (loadAtPosition(in, memory, pos))
@ -472,11 +287,13 @@ static std::pair<bool, size_t> fileSegmentationEngineTabSeparatedImpl(ReadBuffer
void registerFileSegmentationEngineTabSeparated(FormatFactory & factory)
{
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TSV", &fileSegmentationEngineTabSeparatedImpl<false>);
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TabSeparated", &fileSegmentationEngineTabSeparatedImpl<false>);
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TSVRaw", &fileSegmentationEngineTabSeparatedImpl<true>);
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TabSeparatedRaw", &fileSegmentationEngineTabSeparatedImpl<true>);
// We can use the same segmentation engine for TSKV.
for (const auto & name : {"TabSeparated", "TSV", "TSKV", "TabSeparatedWithNames", "TSVWithNames"})
{
factory.registerFileSegmentationEngine(name, &fileSegmentationEngineTabSeparatedImpl);
}
factory.registerFileSegmentationEngine("TSKV", &fileSegmentationEngineTabSeparatedImpl<false>);
}
}

View File

@ -2,7 +2,7 @@
#include <Core/Block.h>
#include <Formats/FormatSettings.h>
#include <Processors/Formats/RowInputFormatWithDiagnosticInfo.h>
#include <Processors/Formats/RowInputFormatWithNamesAndTypes.h>
namespace DB
@ -10,46 +10,39 @@ namespace DB
/** A stream to input data in tsv format.
*/
class TabSeparatedRowInputFormat : public RowInputFormatWithDiagnosticInfo
class TabSeparatedRowInputFormat : public RowInputFormatWithNamesAndTypes
{
public:
/** with_names - the first line is the header with the names of the columns
* with_types - on the next line header with type names
*/
TabSeparatedRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
bool with_names_, bool with_types_, const FormatSettings & format_settings_);
bool with_names_, bool with_types_, bool is_raw, const FormatSettings & format_settings_);
String getName() const override { return "TabSeparatedRowInputFormat"; }
bool readRow(MutableColumns & columns, RowReadExtension &) override;
void readPrefix() override;
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
void resetParser() override;
protected:
bool with_names;
bool with_types;
const FormatSettings format_settings;
virtual bool readField(IColumn & column, const DataTypePtr & type,
const SerializationPtr & serialization, bool is_last_file_column);
private:
DataTypes data_types;
bool is_raw;
using IndexesMap = std::unordered_map<String, size_t>;
IndexesMap column_indexes_by_names;
bool readField(IColumn & column, const DataTypePtr & type,
const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override;
std::vector<size_t> columns_to_fill_with_default_values;
void skipField(const String & /*column_name*/) override { skipField(); }
void skipField();
void skipRow() override;
void skipFieldDelimiter() override;
void skipRowEndDelimiter() override;
void addInputColumn(const String & column_name);
void setupAllColumnsByTableSchema();
void fillUnreadColumnsWithDefaults(MutableColumns & columns, RowReadExtension & row_read_extension);
Names readHeaderRow() override;
String readFieldIntoString();
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override;
void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override;
void checkNullValueForNonNullable(DataTypePtr type) override;
bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override;
bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override;
bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override { return *pos != '\n' && *pos != '\t'; }
};

View File

@ -1,5 +1,4 @@
#include <Processors/Formats/Impl/TabSeparatedRowOutputFormat.h>
#include <Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h>
#include <Formats/FormatFactory.h>
#include <IO/WriteHelpers.h>
@ -11,41 +10,43 @@ TabSeparatedRowOutputFormat::TabSeparatedRowOutputFormat(
const Block & header_,
bool with_names_,
bool with_types_,
bool is_raw_,
const RowOutputFormatParams & params_,
const FormatSettings & format_settings_)
: IRowOutputFormat(header_, out_, params_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_)
: IRowOutputFormat(header_, out_, params_), with_names(with_names_), with_types(with_types_), is_raw(is_raw_), format_settings(format_settings_)
{
}
void TabSeparatedRowOutputFormat::writeLine(const std::vector<String> & values)
{
for (size_t i = 0; i < values.size(); ++i)
{
writeEscapedString(values[i], out);
if (i + 1 == values.size())
writeRowEndDelimiter();
else
writeFieldDelimiter();
}
}
void TabSeparatedRowOutputFormat::doWritePrefix()
{
const auto & header = getPort(PortKind::Main).getHeader();
size_t columns = header.columns();
if (with_names)
{
for (size_t i = 0; i < columns; ++i)
{
writeEscapedString(header.safeGetByPosition(i).name, out);
writeChar(i == columns - 1 ? '\n' : '\t', out);
}
}
writeLine(header.getNames());
if (with_types)
{
for (size_t i = 0; i < columns; ++i)
{
writeEscapedString(header.safeGetByPosition(i).type->getName(), out);
writeChar(i == columns - 1 ? '\n' : '\t', out);
}
}
writeLine(header.getDataTypeNames());
}
void TabSeparatedRowOutputFormat::writeField(const IColumn & column, const ISerialization & serialization, size_t row_num)
{
serialization.serializeTextEscaped(column, row_num, out, format_settings);
if (is_raw)
serialization.serializeTextRaw(column, row_num, out, format_settings);
else
serialization.serializeTextEscaped(column, row_num, out, format_settings);
}
@ -75,56 +76,23 @@ void TabSeparatedRowOutputFormat::writeBeforeExtremes()
void registerOutputFormatTabSeparated(FormatFactory & factory)
{
for (const auto * name : {"TabSeparated", "TSV"})
for (bool is_raw : {false, true})
{
factory.registerOutputFormat(name, [](
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & settings)
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
{
return std::make_shared<TabSeparatedRowOutputFormat>(buf, sample, false, false, params, settings);
});
factory.markOutputFormatSupportsParallelFormatting(name);
}
factory.registerOutputFormat(format_name, [=](
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRowOutputFormat>(buf, sample, with_names, with_types, is_raw, params, settings);
});
factory.markOutputFormatSupportsParallelFormatting(format_name);
};
for (const auto * name : {"TabSeparatedRaw", "TSVRaw"})
{
factory.registerOutputFormat(name, [](
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRawRowOutputFormat>(buf, sample, false, false, params, settings);
});
factory.markOutputFormatSupportsParallelFormatting(name);
}
for (const auto * name : {"TabSeparatedWithNames", "TSVWithNames"})
{
factory.registerOutputFormat(name, [](
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRowOutputFormat>(buf, sample, true, false, params, settings);
});
factory.markOutputFormatSupportsParallelFormatting(name);
}
for (const auto * name : {"TabSeparatedWithNamesAndTypes", "TSVWithNamesAndTypes"})
{
factory.registerOutputFormat(name, [](
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRowOutputFormat>(buf, sample, true, true, params, settings);
});
factory.markOutputFormatSupportsParallelFormatting(name);
registerOutputFormatWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func);
registerOutputFormatWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func);
}
}

View File

@ -23,6 +23,7 @@ public:
const Block & header_,
bool with_names_,
bool with_types_,
bool is_raw_,
const RowOutputFormatParams & params_,
const FormatSettings & format_settings_);
@ -39,10 +40,13 @@ public:
/// https://www.iana.org/assignments/media-types/text/tab-separated-values
String getContentType() const override { return "text/tab-separated-values; charset=UTF-8"; }
protected:
private:
void writeLine(const std::vector<String> & values);
bool with_names;
bool with_types;
bool is_raw;
protected:
const FormatSettings format_settings;
};

View File

@ -130,7 +130,7 @@ void TemplateBlockOutputFormat::serializeField(const IColumn & column, const ISe
serialization.serializeTextXML(column, row_num, out, settings);
break;
case ColumnFormat::Raw:
serialization.serializeText(column, row_num, out, settings);
serialization.serializeTextRaw(column, row_num, out, settings);
break;
default:
__builtin_unreachable();

View File

@ -45,8 +45,8 @@ TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, ReadBuffer
}
else
{
if (format.formats[i] == ColumnFormat::Xml || format.formats[i] == ColumnFormat::Raw)
format.throwInvalidFormat("XML and Raw deserialization is not supported", i);
if (format.formats[i] == ColumnFormat::Xml)
format.throwInvalidFormat("XML deserialization is not supported", i);
}
}
@ -54,8 +54,8 @@ TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, ReadBuffer
std::vector<UInt8> column_in_format(header_.columns(), false);
for (size_t i = 0; i < row_format.columnsCount(); ++i)
{
if (row_format.formats[i] == ColumnFormat::Xml || row_format.formats[i] == ColumnFormat::Raw)
row_format.throwInvalidFormat("XML and Raw deserialization is not supported", i);
if (row_format.formats[i] == ColumnFormat::Xml)
row_format.throwInvalidFormat("XML deserialization is not supported", i);
if (row_format.format_idx_to_column_idx[i])
{
@ -194,7 +194,7 @@ bool TemplateRowInputFormat::deserializeField(const DataTypePtr & type,
{
ColumnFormat col_format = row_format.formats[file_column];
bool read = true;
bool parse_as_nullable = settings.null_as_default && !type->isNullable();
bool parse_as_nullable = settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable();
try
{
switch (col_format)
@ -226,6 +226,12 @@ bool TemplateRowInputFormat::deserializeField(const DataTypePtr & type,
else
serialization->deserializeTextJSON(column, buf, settings);
break;
case ColumnFormat::Raw:
if (parse_as_nullable)
read = SerializationNullable::deserializeTextRawImpl(column, buf, settings, serialization);
else
serialization->deserializeTextRaw(column, buf, settings);
break;
default:
__builtin_unreachable();
}

View File

@ -168,7 +168,7 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx)
bool read = true;
const auto & type = types[column_idx];
const auto & serialization = serializations[column_idx];
if (format_settings.null_as_default && !type->isNullable())
if (format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable())
read = SerializationNullable::deserializeTextQuotedImpl(column, buf, format_settings, serialization);
else
serialization->deserializeTextQuoted(column, buf, format_settings);
@ -409,7 +409,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx
Field value = convertFieldToType(expression_value, type, value_raw.second.get());
/// Check that we are indeed allowed to insert a NULL.
if (value.isNull() && !type.isNullable())
if (value.isNull() && !type.isNullable() && !type.isLowCardinalityNullable())
{
if (format_settings.null_as_default)
{

View File

@ -0,0 +1,265 @@
#include <Processors/Formats/RowInputFormatWithNamesAndTypes.h>
#include <DataTypes/DataTypeNothing.h>
#include <IO/ReadHelpers.h>
#include <IO/Operators.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
}
RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes(
const Block & header_,
ReadBuffer & in_,
const Params & params_,
bool with_names_,
bool with_types_,
const FormatSettings & format_settings_)
: RowInputFormatWithDiagnosticInfo(header_, in_, params_), format_settings(format_settings_), with_names(with_names_), with_types(with_types_)
{
const auto & sample = getPort().getHeader();
size_t num_columns = sample.columns();
data_types.resize(num_columns);
column_indexes_by_names.reserve(num_columns);
for (size_t i = 0; i < num_columns; ++i)
{
const auto & column_info = sample.getByPosition(i);
data_types[i] = column_info.type;
column_indexes_by_names.emplace(column_info.name, i);
}
}
void RowInputFormatWithNamesAndTypes::setupAllColumnsByTableSchema()
{
const auto & header = getPort().getHeader();
column_mapping->column_indexes_for_input_fields.resize(header.columns());
column_mapping->names_of_columns = header.getNames();
for (size_t i = 0; i < column_mapping->column_indexes_for_input_fields.size(); ++i)
column_mapping->column_indexes_for_input_fields[i] = i;
}
void RowInputFormatWithNamesAndTypes::addInputColumn(const String & column_name, std::vector<bool> & read_columns)
{
column_mapping->names_of_columns.push_back(column_name);
const auto column_it = column_indexes_by_names.find(column_name);
if (column_it == column_indexes_by_names.end())
{
if (format_settings.skip_unknown_fields)
{
column_mapping->column_indexes_for_input_fields.push_back(std::nullopt);
return;
}
throw Exception(
ErrorCodes::INCORRECT_DATA,
"Unknown field found in {} header: '{}' at position {}\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed",
getName(), column_name, column_mapping->column_indexes_for_input_fields.size());
}
const auto column_index = column_it->second;
if (read_columns[column_index])
throw Exception("Duplicate field found while parsing TSV header: " + column_name, ErrorCodes::INCORRECT_DATA);
read_columns[column_index] = true;
column_mapping->column_indexes_for_input_fields.emplace_back(column_index);
}
void RowInputFormatWithNamesAndTypes::readPrefix()
{
if (with_names || with_types || data_types.at(0)->textCanContainOnlyValidUTF8())
{
/// We assume that column name or type cannot contain BOM, so, if format has header,
/// then BOM at beginning of stream cannot be confused with name or type of field, and it is safe to skip it.
skipBOMIfExists(*in);
}
/// This is a bit of abstraction leakage, but we need it in parallel parsing:
/// we check if this InputFormat is working with the "real" beginning of the data.
if (with_names && getCurrentUnitNumber() == 0)
{
if (format_settings.with_names_use_header)
{
std::vector<bool> read_columns(data_types.size(), false);
auto column_names = readHeaderRow();
for (const auto & name : column_names)
addInputColumn(name, read_columns);
for (size_t i = 0; i != read_columns.size(); ++i)
{
if (!read_columns[i])
column_mapping->not_presented_columns.push_back(i);
}
}
else
{
setupAllColumnsByTableSchema();
skipRow();
}
}
else if (!column_mapping->is_set)
setupAllColumnsByTableSchema();
if (with_types && getCurrentUnitNumber() == 0)
{
if (format_settings.with_types_use_header)
{
auto types = readHeaderRow();
if (types.size() != column_mapping->column_indexes_for_input_fields.size())
throw Exception(
ErrorCodes::INCORRECT_DATA,
"The number of data types differs from the number of column names in input data");
/// Check that types from input matches types from header.
for (size_t i = 0; i < types.size(); ++i)
{
if (column_mapping->column_indexes_for_input_fields[i] &&
data_types[*column_mapping->column_indexes_for_input_fields[i]]->getName() != types[i])
{
throw Exception(
ErrorCodes::INCORRECT_DATA,
"Type of '{}' must be {}, not {}",
getPort().getHeader().getByPosition(*column_mapping->column_indexes_for_input_fields[i]).name,
data_types[*column_mapping->column_indexes_for_input_fields[i]]->getName(), types[i]);
}
}
}
else
skipRow();
}
}
void RowInputFormatWithNamesAndTypes::insertDefaultsForNotSeenColumns(MutableColumns & columns, RowReadExtension & ext)
{
for (auto index : column_mapping->not_presented_columns)
{
columns[index]->insertDefault();
ext.read_columns[index] = false;
}
}
bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadExtension & ext)
{
if (in->eof())
return false;
updateDiagnosticInfo();
skipRowStartDelimiter();
ext.read_columns.resize(data_types.size());
for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column)
{
const auto & column_index = column_mapping->column_indexes_for_input_fields[file_column];
const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size();
if (column_index)
ext.read_columns[*column_index] = readField(
*columns[*column_index],
data_types[*column_index],
serializations[*column_index],
is_last_file_column,
column_mapping->names_of_columns[file_column]);
else
skipField(column_mapping->names_of_columns[file_column]);
if (!is_last_file_column)
skipFieldDelimiter();
}
skipRowEndDelimiter();
insertDefaultsForNotSeenColumns(columns, ext);
/// If defaults_for_omitted_fields is set to 0, we should leave already inserted defaults.
if (!format_settings.defaults_for_omitted_fields)
ext.read_columns.assign(ext.read_columns.size(), true);
return true;
}
void RowInputFormatWithNamesAndTypes::resetParser()
{
RowInputFormatWithDiagnosticInfo::resetParser();
column_mapping->column_indexes_for_input_fields.clear();
column_mapping->not_presented_columns.clear();
}
void RowInputFormatWithNamesAndTypes::tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column)
{
const auto & index = column_mapping->column_indexes_for_input_fields[file_column];
if (index)
{
checkNullValueForNonNullable(type);
const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size();
readField(column, type, serializations[*index], is_last_file_column, column_mapping->names_of_columns[file_column]);
}
else
{
skipField(column_mapping->names_of_columns[file_column]);
}
}
bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out)
{
if (in->eof())
{
out << "<End of stream>\n";
return false;
}
if (!parseRowStartWithDiagnosticInfo(out))
return false;
for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column)
{
if (column_mapping->column_indexes_for_input_fields[file_column].has_value())
{
const auto & header = getPort().getHeader();
size_t col_idx = column_mapping->column_indexes_for_input_fields[file_column].value();
if (!deserializeFieldAndPrintDiagnosticInfo(header.getByPosition(col_idx).name, data_types[col_idx], *columns[col_idx], out, file_column))
return false;
}
else
{
static const String skipped_column_str = "<SKIPPED COLUMN>";
static const DataTypePtr skipped_column_type = std::make_shared<DataTypeNothing>();
static const MutableColumnPtr skipped_column = skipped_column_type->createColumn();
if (!deserializeFieldAndPrintDiagnosticInfo(skipped_column_str, skipped_column_type, *skipped_column, out, file_column))
return false;
}
/// Delimiters
if (file_column + 1 != column_mapping->column_indexes_for_input_fields.size())
{
if (!parseFieldDelimiterWithDiagnosticInfo(out))
return false;
}
}
return parseRowEndWithDiagnosticInfo(out);
}
void registerInputFormatWithNamesAndTypes(const String & base_format_name, RegisterFormatWithNamesAndTypesFunc register_func)
{
register_func(base_format_name, false, false);
register_func(base_format_name + "WithNames", true, false);
register_func(base_format_name + "WithNamesAndTypes", true, true);
}
void registerFileSegmentationEngineForFormatWithNamesAndTypes(
FormatFactory & factory, const String & base_format_name, FormatFactory::FileSegmentationEngine segmentation_engine)
{
factory.registerFileSegmentationEngine(base_format_name, segmentation_engine);
factory.registerFileSegmentationEngine(base_format_name + "WithNames", segmentation_engine);
factory.registerFileSegmentationEngine(base_format_name + "WithNamesAndTypes", segmentation_engine);
}
}

View File

@ -0,0 +1,73 @@
#pragma once
#include <Processors/Formats/RowInputFormatWithDiagnosticInfo.h>
#include <Formats/FormatSettings.h>
#include <Formats/FormatFactory.h>
namespace DB
{
/// Base class for input formats with -WithNames and -WithNamesAndTypes suffixes.
class RowInputFormatWithNamesAndTypes : public RowInputFormatWithDiagnosticInfo
{
public:
/** with_names - in the first line the header with column names
* with_types - in the second line the header with column names
*/
RowInputFormatWithNamesAndTypes(
const Block & header_,
ReadBuffer & in_,
const Params & params_,
bool with_names_, bool with_types_, const FormatSettings & format_settings_);
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
void readPrefix() override;
void resetParser() override;
protected:
/// Return false if there was no real value and we inserted default value.
virtual bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) = 0;
virtual void skipField(const String & column_name) = 0;
virtual void skipRow() = 0;
virtual void skipRowStartDelimiter() {}
virtual void skipFieldDelimiter() {}
virtual void skipRowEndDelimiter() {}
/// Methods for parsing with diagnostic info.
virtual void checkNullValueForNonNullable(DataTypePtr /*type*/) {}
virtual bool parseRowStartWithDiagnosticInfo(WriteBuffer & /*out*/) { return true; }
virtual bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) = 0;
virtual bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) = 0;
/// Read the list of names or types.
virtual std::vector<String> readHeaderRow() = 0;
const FormatSettings format_settings;
DataTypes data_types;
private:
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override;
void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override;
void setupAllColumnsByTableSchema();
void addInputColumn(const String & column_name, std::vector<bool> & read_columns);
void insertDefaultsForNotSeenColumns(MutableColumns & columns, RowReadExtension & ext);
bool with_names;
bool with_types;
std::unordered_map<String, size_t> column_indexes_by_names;
};
using RegisterFormatWithNamesAndTypesFunc = std::function<void(
const String & format_name,
bool with_names,
bool with_types)>;
void registerInputFormatWithNamesAndTypes(const String & base_format_name, RegisterFormatWithNamesAndTypesFunc register_func);
void registerFileSegmentationEngineForFormatWithNamesAndTypes(
FormatFactory & factory, const String & base_format_name, FormatFactory::FileSegmentationEngine segmentation_engine);
}

View File

@ -1,6 +1,10 @@
"Hello, ""World""",123,"[1,2,3]",456,"['abc','def']","Newline
here"
"x","y","z","a","b"
"Hello, ""World""",123,"[1,2,3]",456,"['abc','def']","Newline
here"
"x","y","z","a","b"
"String","UInt8","Array(UInt8)","Tuple(UInt16, Array(String))","String"
"Hello, ""World""",123,"[1,2,3]",456,"['abc','def']","Newline
here"
0,"0","[]","2000-01-01","2000-01-01 00:00:00"

View File

@ -1,3 +1,4 @@
SELECT 'Hello, "World"' AS x, 123 AS y, [1, 2, 3] AS z, (456, ['abc', 'def']) AS a, 'Newline\nhere' AS b FORMAT CSVWithNames;
SELECT 'Hello, "World"' AS x, 123 AS y, [1, 2, 3] AS z, (456, ['abc', 'def']) AS a, 'Newline\nhere' AS b FORMAT CSV;
SELECT 'Hello, "World"' AS x, 123 AS y, [1, 2, 3] AS z, (456, ['abc', 'def']) AS a, 'Newline\nhere' AS b FORMAT CSVWithNames;
SELECT 'Hello, "World"' AS x, 123 AS y, [1, 2, 3] AS z, (456, ['abc', 'def']) AS a, 'Newline\nhere' AS b FORMAT CSVWithNamesAndTypes;
SELECT number, toString(number), range(number), toDate('2000-01-01') + number, toDateTime('2000-01-01 00:00:00') + number FROM system.numbers LIMIT 10 FORMAT CSV;

View File

@ -13,7 +13,7 @@ Hello "world", 789 ,2016-01-03
"Hello
world", 100, 2016-01-04,
default,,
default-eof,,' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --query="INSERT INTO csv FORMAT CSV";
default-eof,,' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_csv_empty_as_default=1 --query="INSERT INTO csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY d";
$CLICKHOUSE_CLIENT --query="DROP TABLE csv";
@ -33,7 +33,7 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (t Nullable(DateTime('Europe/Moscow
echo 'NULL, NULL
"2016-01-01 01:02:03",NUL
"2016-01-02 01:02:03",Nhello' | $CLICKHOUSE_CLIENT --input_format_csv_unquoted_null_literal_as_null=1 --query="INSERT INTO csv FORMAT CSV";
"2016-01-02 01:02:03",Nhello' | $CLICKHOUSE_CLIENT --input_format_csv_unquoted_null_literal_as_null=1 --input_format_csv_empty_as_default=1 --query="INSERT INTO csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s NULLS LAST";
$CLICKHOUSE_CLIENT --query="DROP TABLE csv";

View File

@ -23,3 +23,11 @@ cv bn m","","as""df'gh","",456,"2016-01-02"
"as""df'gh","","zx
cv bn m","",789,"2016-01-04"
"qwe,rty","","","",9876543210,"2016-01-03"
==== check raw ====
"qwe,rty","as""df'gh","","zx
cv bn m",123,"2016-01-01"
"as""df\'gh","","zx
cv bn m","qwe,rty",456,"2016-01-02"
"zx\cv\bn m","qwe,rty","as""df'gh","",789,"2016-01-04"
"","zx
cv bn m","qwe,rty","as""df'gh",9876543210,"2016-01-03"

View File

@ -50,6 +50,30 @@ format_template_rows_between_delimiter = ','";
$CLICKHOUSE_CLIENT --query="SELECT * FROM template2 ORDER BY n FORMAT CSV";
echo "==== check raw ===="
echo -ne '{prefix} \n${data}\n $$ suffix $$\n' > "$CURDIR"/00938_template_input_format_resultset.tmp
echo -ne 'n:\t${n:Escaped}, s1:\t${0:Raw}\t, s2:\t${1:Quoted}, s3:\t${s3:JSON}, s4:\t${3:CSV}, d:\t${d:Escaped}\t' > "$CURDIR"/00938_template_input_format_row.tmp
$CLICKHOUSE_CLIENT --query="TRUNCATE TABLE template1";
echo "{prefix}"' '"
n: 123, s1: qwe,rty , s2: 'as\"df\\'gh', s3: \"\", s4: \"zx
cv bn m\", d: 2016-01-01 ;
n: 456, s1: as\"df\\'gh , s2: '', s3: \"zx\\ncv\\tbn m\", s4: \"qwe,rty\", d: 2016-01-02 ;
n: 9876543210, s1: , s2: 'zx\\ncv\\tbn m', s3: \"qwe,rty\", s4: \"as\"\"df'gh\", d: 2016-01-03 ;
n: 789, s1: zx\cv\bn m , s2: 'qwe,rty', s3: \"as\\\"df'gh\", s4: \"\", d: 2016-01-04"$'\t'"
$ suffix $" | $CLICKHOUSE_CLIENT --query="INSERT INTO template1 FORMAT Template SETTINGS \
format_template_resultset = '$CURDIR/00938_template_input_format_resultset.tmp', \
format_template_row = '$CURDIR/00938_template_input_format_row.tmp', \
format_template_rows_between_delimiter = ';\n'";
$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT CSV";
$CLICKHOUSE_CLIENT --query="DROP TABLE template1";
$CLICKHOUSE_CLIENT --query="DROP TABLE template2";
rm "$CURDIR"/00938_template_input_format_resultset.tmp "$CURDIR"/00938_template_input_format_row.tmp

View File

@ -12,6 +12,11 @@
[1, "a"]
[2, "b"]
[3, "c"]
----------
["value", "name"]
[1, "a"]
[2, "b"]
[3, "c"]
4
["name", "c"]
["String", "UInt64"]
@ -31,17 +36,33 @@
8
["first", 1, 2, 0]
["second", 2, 0, 6]
["first", 1, 2, 0]
["second", 2, 0, 6]
9
["first", 1, 2, 8]
["second", 2, 32, 6]
["first", 1, 2, 8]
["second", 2, 32, 6]
10
["first", 1, 16, 8]
["second", 2, 32, 8]
["first", 1, 16, 8]
["second", 2, 32, 8]
11
["v1", "v2", "v3", "v4"]
["String", "UInt8", "UInt16", "UInt8"]
["", 2, 3, 1]
["", 2, 3, 1]
----------
["v1", "v2", "v3", "v4"]
["", 2, 3, 1]
["", 2, 3, 1]
12
["v1", "n.id", "n.name"]
["UInt8", "Array(UInt8)", "Array(String)"]
[16, [15,16,0], ["first","second","third"]]
[16, [15,16,0], ["first","second","third"]]
----------
["v1", "n.id", "n.name"]
[16, [15,16,0], ["first","second","third"]]
[16, [15,16,0], ["first","second","third"]]

View File

@ -10,8 +10,10 @@ SELECT 2;
/* Check Totals */
SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactEachRow;
SELECT 3;
/* Check JSONCompactEachRowWithNamesAndTypes Output */
/* Check JSONCompactEachRowWithNames and JSONCompactEachRowWithNamesAndTypes Output */
SELECT * FROM test_table FORMAT JSONCompactEachRowWithNamesAndTypes;
SELECT '----------';
SELECT * FROM test_table FORMAT JSONCompactEachRowWithNames;
SELECT 4;
/* Check Totals */
SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactEachRowWithNamesAndTypes;
@ -35,30 +37,39 @@ INSERT INTO test_table_2 FORMAT JSONCompactEachRow [16, [15, 16, null], ["first"
SELECT * FROM test_table_2 FORMAT JSONCompactEachRow;
TRUNCATE TABLE test_table_2;
SELECT 8;
/* Check JSONCompactEachRowWithNamesAndTypes Output */
/* Check JSONCompactEachRowWithNamesAndTypes and JSONCompactEachRowWithNamesAndTypes Input */
SET input_format_null_as_default = 0;
INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", 1, "2", null]["second", 2, null, 6];
INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v1", "v2", "v3", "v4"]["first", 1, "2", null]["second", 2, null, 6];
SELECT * FROM test_table FORMAT JSONCompactEachRow;
TRUNCATE TABLE test_table;
SELECT 9;
/* Check input_format_null_as_default = 1 */
SET input_format_null_as_default = 1;
INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", 1, "2", null] ["second", 2, null, 6];
INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v1", "v2", "v3", "v4"]["first", 1, "2", null] ["second", 2, null, 6];
SELECT * FROM test_table FORMAT JSONCompactEachRow;
SELECT 10;
/* Check Header */
TRUNCATE TABLE test_table;
SET input_format_skip_unknown_fields = 1;
INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", 1, 32]["second", 2, "64"];
INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v1", "v2", "invalid_column"]["first", 1, 32]["second", 2, "64"];
SELECT * FROM test_table FORMAT JSONCompactEachRow;
SELECT 11;
TRUNCATE TABLE test_table;
INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"][1, 2, 3]
INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v4", "v2", "v3"][1, 2, 3]
SELECT * FROM test_table FORMAT JSONCompactEachRowWithNamesAndTypes;
SELECT '----------';
SELECT * FROM test_table FORMAT JSONCompactEachRowWithNames;
SELECT 12;
/* Check Nested */
INSERT INTO test_table_2 FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"][16, [15, 16, null], ["first", "second", "third"]];
INSERT INTO test_table_2 FORMAT JSONCompactEachRowWithNames ["v1", "n.id", "n.name"][16, [15, 16, null], ["first", "second", "third"]];
SELECT * FROM test_table_2 FORMAT JSONCompactEachRowWithNamesAndTypes;
SELECT '----------';
SELECT * FROM test_table_2 FORMAT JSONCompactEachRowWithNames;
DROP TABLE IF EXISTS test_table;
DROP TABLE IF EXISTS test_table_2;

View File

@ -1,5 +1,5 @@
CSV
Column 2, name: d, type: Decimal(18, 10), parsed text: "123456789"ERROR
Column 2, name: d, type: Decimal(18, 10), parsed text: " 123456789"ERROR
ERROR: garbage after DateTime: "7, <DOUBLE QUOTE>Hello<DOUBLE QUOTE>"
ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.
ERROR: There is no line feed. "1" found instead.
@ -28,3 +28,14 @@ ERROR: There is no delimiter before field 1: expected "<TAB>", got "7<TAB>Hello<
ERROR: There is no delimiter after last field: expected "<LINE FEED>", got "<TAB>1<LINE FEED>"
ERROR: There is no delimiter after last field: expected "<LINE FEED>", got "Hello<LINE FEED>"
Column 0, name: t, type: DateTime, ERROR: text "<LINE FEED>" is not like DateTime
JSONCompactEachRow
Column 2, name: d, type: Decimal(18, 10), parsed text: " 123456789"ERROR
Column 0, name: t, type: DateTime, parsed text: "<DOUBLE QUOTE>2020-04-21 12:34:56"ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.
ERROR: garbage after DateTime: "7, <DOUBLE QUOTE>Hello<DOUBLE QUOTE>"
ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.
ERROR: There is no closing parenthesis (']') at the end of the row. "," found instead.
Column 1, name: s, type: String, parsed text: <EMPTY>ERROR
ERROR: There is no '[' before the row.
ERROR: garbage after Decimal(18, 10): ";<LINE FEED>"
ERROR: There is no comma. ";" found instead.
ERROR: Closing parenthesis (']') found where comma is expected. It's like your file has less columns than expected.

View File

@ -38,3 +38,19 @@ echo -e '2020-04-21 12:34:567\tHello\t123456789' | "${PARSER[@]}" 2>&1| grep "ER
echo -e '2020-04-21 12:34:56\tHello\t12345678\t1' | "${PARSER[@]}" 2>&1| grep "ERROR"
echo -e '2020-04-21 12:34:56\t\t123Hello' | "${PARSER[@]}" 2>&1| grep "ERROR"
echo -e '2020-04-21 12:34:56\tHello\t12345678\n' | "${PARSER[@]}" 2>&1| grep "ERROR"
PARSER=(${CLICKHOUSE_LOCAL} --query 'SELECT t, s, d FROM table' --structure 't DateTime, s String, d Decimal64(10)' --input-format JSONCompactEachRow)
echo '["2020-04-21 12:34:56", "Hello", 12345678]' | "${PARSER[@]}" 2>&1| grep "ERROR" || echo "JSONCompactEachRow"
echo '["2020-04-21 12:34:56", "Hello", 123456789]' | "${PARSER[@]}" 2>&1| grep "ERROR"
echo '["2020-04-21 12:34:567", "Hello", 123456789]' | "${PARSER[@]}" 2>&1| grep "ERROR"
echo '["2020-04-21 12:34:56"7, "Hello", 123456789]' | "${PARSER[@]}" 2>&1| grep "ERROR"
echo '["2020-04-21 12:34:56", "Hello", 12345678,1]' | "${PARSER[@]}" 2>&1| grep "ERROR"
echo '["2020-04-21 12:34:56",,123Hello]' | "${PARSER[@]}" 2>&1| grep "ERROR"
echo -e '["2020-04-21 12:34:56", "Hello", 12345678\n]' | "${PARSER[@]}" 2>&1| grep "ERROR"
echo -e '"2020-04-21 12:34:56", "Hello", 12345678]' | "${PARSER[@]}" 2>&1| grep "ERROR"
echo -e '["2020-04-21 12:34:56", "Hello", 12345678;' | "${PARSER[@]}" 2>&1| grep "ERROR"
echo -e '["2020-04-21 12:34:56", "Hello", 12345678' | "${PARSER[@]}" 2>&1| grep "ERROR"
echo -e '["2020-04-21 12:34:56", "Hello", 12345678\n' | "${PARSER[@]}" 2>&1| grep "ERROR"
echo -e '["2020-04-21 12:34:56", "Hello"; 12345678\n' | "${PARSER[@]}" 2>&1| grep "ERROR"
echo -e '["2020-04-21 12:34:56", "Hello"\n' | "${PARSER[@]}" 2>&1| grep "ERROR"
echo -e '["2020-04-21 12:34:56", "Hello"]' | "${PARSER[@]}" 2>&1| grep "ERROR"

View File

@ -7,7 +7,21 @@ number
UInt64
0
1
TSVRawWithNames
number
0
1
TSVRawWithNamesAndTypes
number
UInt64
0
1
CSVWithNames
"number"
0
1
CSVWithNamesAndTypes
"number"
"UInt64"
0
1

View File

@ -15,5 +15,14 @@ ${CLICKHOUSE_LOCAL} "${opts[@]}" --format TSVWithNames
echo 'TSVWithNamesAndTypes'
${CLICKHOUSE_LOCAL} "${opts[@]}" --format TSVWithNamesAndTypes
echo 'TSVRawWithNames'
${CLICKHOUSE_LOCAL} "${opts[@]}" --format TSVWithNames
echo 'TSVRawWithNamesAndTypes'
${CLICKHOUSE_LOCAL} "${opts[@]}" --format TSVWithNamesAndTypes
echo 'CSVWithNames'
${CLICKHOUSE_LOCAL} "${opts[@]}" --format CSVWithNames
echo 'CSVWithNamesAndTypes'
${CLICKHOUSE_LOCAL} "${opts[@]}" --format CSVWithNamesAndTypes

View File

@ -2,6 +2,11 @@ zero rows
TSVWithNames
TSVWithNamesAndTypes
CSVWithNames
CSVWithNamesAndTypes
JSONCompactEachRowWithNames
JSONCompactEachRowWithNamesAndTypes
JSONCompactStringsEachRow
JSONCompactStringsEachRowWithNamesAndTypes
multi clickhouse-local one file
TSVWithNames
0
@ -15,3 +20,23 @@ CSVWithNames
0
0
0
CSVWithNamesAndTypes
0
0
0
JSONCompactEachRowWithNames
0
0
0
JSONCompactEachRowWithNamesAndTypes
0
0
0
JSONCompactStringsEachRow
0
0
0
JSONCompactStringsEachRowWithNamesAndTypes
0
0
0

View File

@ -6,26 +6,26 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# zero rows
echo 'zero rows'
for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames; do
for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames CSVWithNamesAndTypes JSONCompactEachRowWithNames JSONCompactEachRowWithNamesAndTypes JSONCompactStringsEachRow JSONCompactStringsEachRowWithNamesAndTypes; do
echo $format
${CLICKHOUSE_LOCAL} --query="
CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format.tsv') AS SELECT * FROM numbers(1) WHERE number < 0;
CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format') AS SELECT * FROM numbers(1) WHERE number < 0;
SELECT * FROM ${format}_01375;
DROP TABLE ${format}_01375;
"
rm 01375_$format.tsv
rm 01375_$format
done
# run multiple times to the same file
echo 'multi clickhouse-local one file'
for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames; do
for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames CSVWithNamesAndTypes JSONCompactEachRowWithNames JSONCompactEachRowWithNamesAndTypes JSONCompactStringsEachRow JSONCompactStringsEachRowWithNamesAndTypes; do
echo $format
for _ in {1..2}; do
${CLICKHOUSE_LOCAL} --query="
CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format.tsv') AS SELECT * FROM numbers(1);
CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format') AS SELECT * FROM numbers(1);
SELECT * FROM ${format}_01375;
DROP TABLE ${format}_01375;
"
done
rm 01375_$format.tsv
rm 01375_$format
done

View File

@ -12,6 +12,11 @@
["1", "a"]
["2", "b"]
["3", "c"]
----------
["value", "name"]
["1", "a"]
["2", "b"]
["3", "c"]
4
["name", "c"]
["String", "UInt64"]
@ -31,17 +36,33 @@
8
["first", "1", "2", "0"]
["second", "2", "0", "6"]
["first", "1", "2", "0"]
["second", "2", "0", "6"]
9
["first", "1", "2", "8"]
["second", "2", "32", "6"]
["first", "1", "2", "8"]
["second", "2", "32", "6"]
10
["first", "1", "16", "8"]
["second", "2", "32", "8"]
["first", "1", "16", "8"]
["second", "2", "32", "8"]
11
["v1", "v2", "v3", "v4"]
["String", "UInt8", "UInt16", "UInt8"]
["", "2", "3", "1"]
["", "2", "3", "1"]
---------
["v1", "v2", "v3", "v4"]
["", "2", "3", "1"]
["", "2", "3", "1"]
12
["v1", "n.id", "n.name"]
["UInt8", "Array(UInt8)", "Array(String)"]
["16", "[15,16,17]", "['first','second','third']"]
["16", "[15,16,17]", "['first','second','third']"]
---------
["v1", "n.id", "n.name"]
["16", "[15,16,17]", "['first','second','third']"]
["16", "[15,16,17]", "['first','second','third']"]

View File

@ -12,8 +12,10 @@ SELECT 2;
/* Check Totals */
SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactStringsEachRow;
SELECT 3;
/* Check JSONCompactStringsEachRowWithNamesAndTypes Output */
/* Check JSONCompactStringsEachRowWithNames and JSONCompactStringsEachRowWithNamesAndTypes Output */
SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes;
SELECT '----------';
SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNames;
SELECT 4;
/* Check Totals */
SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactStringsEachRowWithNamesAndTypes;
@ -37,30 +39,39 @@ INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRow ["16", "[15, 16, 17]",
SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRow;
TRUNCATE TABLE test_table_2;
SELECT 8;
/* Check JSONCompactStringsEachRowWithNamesAndTypes Output */
/* Check JSONCompactStringsEachRowWithNames and JSONCompactStringsEachRowWithNamesAndTypes Input */
SET input_format_null_as_default = 0;
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"]["second", "2", "null", "6"];
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v1", "v2", "v3", "v4"]["first", "1", "2", "null"]["second", "2", "null", "6"];
SELECT * FROM test_table FORMAT JSONCompactStringsEachRow;
TRUNCATE TABLE test_table;
SELECT 9;
/* Check input_format_null_as_default = 1 */
SET input_format_null_as_default = 1;
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"] ["second", "2", "null", "6"];
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v1", "v2", "v3", "v4"]["first", "1", "2", "null"] ["second", "2", "null", "6"];
SELECT * FROM test_table FORMAT JSONCompactStringsEachRow;
SELECT 10;
/* Check Header */
TRUNCATE TABLE test_table;
SET input_format_skip_unknown_fields = 1;
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", "1", "32"]["second", "2", "64"];
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v1", "v2", "invalid_column"]["first", "1", "32"]["second", "2", "64"];
SELECT * FROM test_table FORMAT JSONCompactStringsEachRow;
SELECT 11;
TRUNCATE TABLE test_table;
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"]["1", "2", "3"]
INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v4", "v2", "v3"]["1", "2", "3"]
SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes;
SELECT '---------';
SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNames;
SELECT 12;
/* Check Nested */
INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"]["16", "[15, 16, 17]", "['first', 'second', 'third']"];
INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRowWithNames ["v1", "n.id", "n.name"]["16", "[15, 16, 17]", "['first', 'second', 'third']"];
SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRowWithNamesAndTypes;
SELECT '---------';
SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRowWithNames;
DROP TABLE IF EXISTS test_table;
DROP TABLE IF EXISTS test_table_2;

View File

@ -0,0 +1,4 @@
test\n\t\0\n test\n\t\0\n
test\n\t\0\n test\n\t\0\n
test\n\t\0\n test\n\t\0\n
test\n\t\0\n test\n\t\0\n

View File

@ -0,0 +1,14 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "drop table if exists test_02097"
$CLICKHOUSE_CLIENT -q "create table test_02097 (s String, f FixedString(8)) engine=Memory()"
echo -e "('test\n\t\0\n', 'test\n\t\0\n')" | $CLICKHOUSE_CLIENT -q "insert into test_02097 format Values"
$CLICKHOUSE_CLIENT -q "select * from test_02097 format JSONStringsEachRow" | $CLICKHOUSE_CLIENT -q "insert into test_02097 format JSONStringsEachRow"
$CLICKHOUSE_CLIENT -q "select * from test_02097 format JSONCompactStringsEachRow" | $CLICKHOUSE_CLIENT -q "insert into test_02097 format JSONCompactStringsEachRow"
$CLICKHOUSE_CLIENT -q "select * from test_02097"
$CLICKHOUSE_CLIENT -q "drop table test_02097"

View File

@ -0,0 +1,16 @@
TSVWithNamesAndTypes
OK
OK
OK
CSVWithNamesAndTypes
OK
OK
OK
JSONCompactEachRowWithNamesAndTypes
OK
OK
OK
JSONCompactStringsEachRowWithNamesAndTypes
OK
OK
OK

View File

@ -0,0 +1,33 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02098"
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02098 (x UInt32, y String, z Date) engine=Memory()"
echo "TSVWithNamesAndTypes"
echo -e "x\ty\tz\nString\tDate\tUInt32\ntext\t2020-01-01\t1" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT TSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
echo -e "y\tz\tx\nString\tDate\tUInt32\ntext\t2020-01-01\t1" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT TSVWithNamesAndTypes" && echo 'OK' || echo 'FAIL'
echo -e "x\tz\ty\nUInt32\tString\tDate\n1\ttext\t2020-01-01" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT TSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
echo "CSVWithNamesAndTypes"
echo -e "'x','y','z'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
echo -e "'y','z','x'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" && echo 'OK' || echo 'FAIL'
echo -e "'x','z','y'\n'UInt32','String',Date'\n1,'text','2020-01-01'" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
echo "JSONCompactEachRowWithNamesAndTypes"
echo -e '["x","y","z"]\n["String","Date","UInt32"]\n["text","2020-01-01",1]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
echo -e '["y","z","x"]\n["String","Date","UInt32"]\n["text","2020-01-01",1]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactEachRowWithNamesAndTypes" && echo 'OK' || echo 'FAIL'
echo -e '["x","z","y"]\n["UInt32", "String", "Date"]\n[1, "text","2020-01-01"]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
echo "JSONCompactStringsEachRowWithNamesAndTypes"
echo -e '["x","y","z"]\n["String","Date","UInt32"]\n["text","2020-01-01","1"]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactStringsEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
echo -e '["y","z","x"]\n["String","Date","UInt32"]\n["text","2020-01-01","1"]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactStringsEachRowWithNamesAndTypes" && echo 'OK' || echo 'FAIL'
echo -e '["x","z","y"]\n["UInt32", "String", "Date"]\n["1", "text","2020-01-01"]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactStringsEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
$CLICKHOUSE_CLIENT -q "DROP TABLE test_02098"

View File

@ -0,0 +1,113 @@
TSVRaw
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
TSVRawWithNames
number string date
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
TSVRawWithNamesAndTypes
number string date
UInt64 String Date
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
TabSeparatedRaw
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
TabSeparatedRawWithNames
number string date
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
TabSeparatedRawWithNamesAndTypes
number string date
UInt64 String Date
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
0 0 1970-01-01
1 1 1970-01-02
2 2 1970-01-03
0
\N
2
\N
0
\N
2
\N
0
\N
2
\N
0
\N
2
\N
0
\N
2
\N
0
\N
2
\N
0
\N
2
\N
0
\N
2
\N
0
\N
2
\N
0
\N
2
\N
0
\N
2
\N
nSome text
b1cad4eb4be08a40387c9de70d02fcc2 -
b1cad4eb4be08a40387c9de70d02fcc2 -

View File

@ -0,0 +1,59 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02099"
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02099 (number UInt64, string String, date Date) ENGINE=Memory()"
FORMATS=('TSVRaw' 'TSVRawWithNames' 'TSVRawWithNamesAndTypes' 'TabSeparatedRaw' 'TabSeparatedRawWithNames' 'TabSeparatedRawWithNamesAndTypes')
for format in "${FORMATS[@]}"
do
echo $format
$CLICKHOUSE_CLIENT -q "INSERT INTO test_02099 SELECT number, toString(number), toDate(number) FROM numbers(3)"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02099 FORMAT $format"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02099 FORMAT $format" | $CLICKHOUSE_CLIENT -q "INSERT INTO test_02099 FORMAT $format"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02099"
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02099"
done
$CLICKHOUSE_CLIENT -q "DROP TABLE test_02099"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_nullable_02099"
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_nullable_02099 ENGINE=Memory() AS SELECT number % 2 ? NULL : number from numbers(4)";
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_nullable_02099 FORMAT TSVRaw"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_nullable_02099 FORMAT TSVRaw" | $CLICKHOUSE_CLIENT -q "INSERT INTO test_nullable_02099 FORMAT TSVRaw"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_nullable_02099"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_nullable_02099 FORMAT TSV" | $CLICKHOUSE_CLIENT -q "INSERT INTO test_nullable_02099 FORMAT TSVRaw"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_nullable_02099 FORMAT TSVRaw" | $CLICKHOUSE_CLIENT -q "INSERT INTO test_nullable_02099 FORMAT TSV"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_nullable_02099"
$CLICKHOUSE_CLIENT -q "DROP TABLE test_nullable_02099"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_nullable_string_02099"
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_nullable_string_02099 (s Nullable(String)) ENGINE=Memory()";
echo 'nSome text' | $CLICKHOUSE_CLIENT -q "INSERT INTO test_nullable_string_02099 FORMAT TSVRaw"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_nullable_string_02099"
$CLICKHOUSE_CLIENT -q "DROP TABLE test_nullable_string_02099"
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_parallel_parsing_02099"
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_parallel_parsing_02099 (x UInt64, a Array(UInt64), s String) ENGINE=Memory()";
$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(1000000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=0 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_parallel_parsing_02099 ORDER BY x" | md5sum
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_parallel_parsing_02099"
$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(1000000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=1 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_parallel_parsing_02099 ORDER BY x" | md5sum
$CLICKHOUSE_CLIENT -q "DROP TABLE test_parallel_parsing_02099"

View File

@ -0,0 +1,14 @@
CSV
\N
TSV
\N
TSVRaw
\N
TSKV
\N
JSONCompactEachRow
\N
JSONEachRow
\N
Values
\N

View File

@ -0,0 +1,21 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02100"
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02100 (x LowCardinality(Nullable(String)) DEFAULT 'default') ENGINE=Memory()"
FORMATS=('CSV' 'TSV' 'TSVRaw' 'TSKV' 'JSONCompactEachRow' 'JSONEachRow' 'Values')
for format in "${FORMATS[@]}"
do
echo $format
$CLICKHOUSE_CLIENT -q "SELECT NULL as x FORMAT $format" | $CLICKHOUSE_CLIENT -q "INSERT INTO test_02100 FORMAT $format"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02100"
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02100"
done
$CLICKHOUSE_CLIENT -q "DROP TABLE test_02100"

View File

@ -0,0 +1,16 @@
TSV
1 42
2 0
3 42
4 0
CSV
1 42
2 0
3 42
4 0
JSONEachRow
1 42
2 0
JSONCompactEachRow
1 42
2 0

View File

@ -0,0 +1,39 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02101"
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02101 (x UInt64, y UInt64 DEFAULT 42) ENGINE=Memory()"
echo 'TSV'
echo -e 'x\ty\n1\t' | $CLICKHOUSE_CLIENT --input_format_tsv_empty_as_default=1 --input_format_defaults_for_omitted_fields=1 -q "INSERT INTO test_02101 FORMAT TSVWithNames"
echo -e 'x\ty\n2\t' | $CLICKHOUSE_CLIENT --input_format_tsv_empty_as_default=1 --input_format_defaults_for_omitted_fields=0 -q "INSERT INTO test_02101 FORMAT TSVWithNames"
echo -e 'x\tz\n3\t123' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT TSVWithNames"
echo -e 'x\tz\n4\t123' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT TSVWithNames"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02101 ORDER BY x"
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02101"
echo 'CSV'
echo -e '"x","y"\n1,' | $CLICKHOUSE_CLIENT --input_format_csv_empty_as_default=1 --input_format_defaults_for_omitted_fields=1 -q "INSERT INTO test_02101 FORMAT CSVWithNames"
echo -e '"x","y"\n2,' | $CLICKHOUSE_CLIENT --input_format_csv_empty_as_default=1 --input_format_defaults_for_omitted_fields=0 -q "INSERT INTO test_02101 FORMAT CSVWithNames"
echo -e '"x","z"\n3,123' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT CSVWithNames"
echo -e '"x","z"\n4,123' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT CSVWithNames"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02101 ORDER BY x"
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02101"
echo 'JSONEachRow'
echo -e '{"x" : 1, "z" : 123}' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT JSONEachRow"
echo -e '{"x" : 2, "z" : 123}' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT JSONEachRow"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02101 ORDER BY x"
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02101"
echo 'JSONCompactEachRow'
echo -e '["x", "z"], [1, 123]' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT JSONCompactEachRowWithNames"
echo -e '["x", "z"], [2, 123]' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT JSONCompactEachRowWithNames"
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02101 ORDER BY x"
$CLICKHOUSE_CLIENT -q "DROP TABLE test_02101"

View File

@ -1,12 +1,28 @@
JSONEachRow, false
7251839681e559f5a92db107571bb357 -
e0a3c9978a92a277f2fff4664f3c1749 -
JSONEachRow, true
7251839681e559f5a92db107571bb357 -
e0a3c9978a92a277f2fff4664f3c1749 -
JSONCompactEachRow, false
ba1081a754a06ef6563840b2d8d4d327 -
0c1efbbc25a5bd90a2ecea559d283667 -
JSONCompactEachRow, true
ba1081a754a06ef6563840b2d8d4d327 -
0c1efbbc25a5bd90a2ecea559d283667 -
JSONCompactStringsEachRow, false
0c1efbbc25a5bd90a2ecea559d283667 -
JSONCompactStringsEachRow, true
0c1efbbc25a5bd90a2ecea559d283667 -
JSONCompactEachRowWithNames, false
b9e4f8ecadbb650245d1762f4187ee0a -
JSONCompactEachRowWithNames, true
b9e4f8ecadbb650245d1762f4187ee0a -
JSONCompactStringsEachRowWithNames, false
b9e4f8ecadbb650245d1762f4187ee0a -
JSONCompactStringsEachRowWithNames, true
b9e4f8ecadbb650245d1762f4187ee0a -
JSONCompactEachRowWithNamesAndTypes, false
8b41f7375999b53d4c9607398456fe5b -
JSONCompactEachRowWithNamesAndTypes, true
8b41f7375999b53d4c9607398456fe5b -
JSONCompactStringsEachRowWithNamesAndTypes, false
31ded3cd9971b124450fb5a44a8bce63 -
8b41f7375999b53d4c9607398456fe5b -
JSONCompactStringsEachRowWithNamesAndTypes, true
31ded3cd9971b124450fb5a44a8bce63 -
8b41f7375999b53d4c9607398456fe5b -

View File

@ -6,15 +6,15 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
FORMATS=('JSONEachRow' 'JSONCompactEachRow' 'JSONCompactStringsEachRowWithNamesAndTypes')
FORMATS=('JSONEachRow' 'JSONCompactEachRow' 'JSONCompactStringsEachRow' 'JSONCompactEachRowWithNames' 'JSONCompactStringsEachRowWithNames' 'JSONCompactEachRowWithNamesAndTypes' 'JSONCompactStringsEachRowWithNamesAndTypes')
for format in "${FORMATS[@]}"
do
echo "$format, false";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
"SELECT ClientEventTime::DateTime('Europe/Moscow') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum
"SELECT ClientEventTime::DateTime('Europe/Moscow') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c LIMIT 3000000 Format $format" | md5sum
echo "$format, true";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=true -q \
"SELECT ClientEventTime::DateTime('Europe/Moscow') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum
"SELECT ClientEventTime::DateTime('Europe/Moscow') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c LIMIT 3000000 Format $format" | md5sum
done

View File

@ -0,0 +1,20 @@
TSVWithNamesAndTypes, false
7c1feeaae418e502d66fcc8e31946f2e -
TSVWithNamesAndTypes, true
7c1feeaae418e502d66fcc8e31946f2e -
CSVWithNamesAndTypes, false
7c1feeaae418e502d66fcc8e31946f2e -
CSVWithNamesAndTypes, true
7c1feeaae418e502d66fcc8e31946f2e -
JSONStringsEachRow, false
7c1feeaae418e502d66fcc8e31946f2e -
JSONStringsEachRow, true
7c1feeaae418e502d66fcc8e31946f2e -
JSONCompactEachRowWithNamesAndTypes, false
7c1feeaae418e502d66fcc8e31946f2e -
JSONCompactEachRowWithNamesAndTypes, true
7c1feeaae418e502d66fcc8e31946f2e -
JSONCompactStringsEachRowWithNamesAndTypes, false
7c1feeaae418e502d66fcc8e31946f2e -
JSONCompactStringsEachRowWithNamesAndTypes, true
7c1feeaae418e502d66fcc8e31946f2e -

View File

@ -0,0 +1,32 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
FORMATS=('TSVWithNamesAndTypes' 'CSVWithNamesAndTypes' 'JSONStringsEachRow' 'JSONCompactEachRowWithNamesAndTypes' 'JSONCompactStringsEachRowWithNamesAndTypes')
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names"
for format in "${FORMATS[@]}"
do
# Columns are permuted
$CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Europe/Moscow'), b String) ENGINE=Memory()"
echo "$format, false";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
"SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Europe/Moscow') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 5000 Format $format" | \
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names FORMAT $format SETTINGS input_format_null_as_default=0"
$CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names"
$CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Europe/Moscow'), b String) ENGINE=Memory()"
echo "$format, true";
$CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \
"SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Europe/Moscow') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 5000 Format $format" | \
$CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names FORMAT $format SETTINGS input_format_null_as_default=0"
$CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum
$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names"
done