mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Refactor and improve TSV, CSV and JSONCompactEachRow formats, fix some bugs in formats
This commit is contained in:
parent
6c6dd7a0c8
commit
324dfd4f81
@ -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)
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -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) \
|
||||
|
@ -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);
|
||||
|
@ -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; }
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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); });
|
||||
}
|
||||
|
||||
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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); });
|
||||
}
|
||||
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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>;
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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();
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -185,7 +185,7 @@ void registerInputFormatJSONAsString(FormatFactory & factory)
|
||||
|
||||
void registerFileSegmentationEngineJSONAsString(FormatFactory & factory)
|
||||
{
|
||||
factory.registerFileSegmentationEngine("JSONAsString", &fileSegmentationEngineJSONEachRowImpl);
|
||||
factory.registerFileSegmentationEngine("JSONAsString", &fileSegmentationEngineJSONEachRow);
|
||||
}
|
||||
|
||||
void registerNonTrivialPrefixAndSuffixCheckerJSONAsString(FormatFactory & factory)
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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());
|
||||
|
@ -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;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -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>);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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'; }
|
||||
};
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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)
|
||||
{
|
||||
|
265
src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp
Normal file
265
src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp
Normal 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);
|
||||
}
|
||||
|
||||
|
||||
}
|
73
src/Processors/Formats/RowInputFormatWithNamesAndTypes.h
Normal file
73
src/Processors/Formats/RowInputFormatWithNamesAndTypes.h
Normal 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);
|
||||
|
||||
}
|
@ -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"
|
||||
|
@ -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;
|
||||
|
@ -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";
|
||||
|
@ -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"
|
||||
|
@ -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
|
||||
|
||||
|
@ -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"]]
|
||||
|
@ -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;
|
||||
|
@ -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.
|
||||
|
@ -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"
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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']"]
|
||||
|
@ -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;
|
||||
|
@ -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
|
14
tests/queries/0_stateless/02097_json_strings_deserialization.sh
Executable file
14
tests/queries/0_stateless/02097_json_strings_deserialization.sh
Executable 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"
|
||||
|
@ -0,0 +1,16 @@
|
||||
TSVWithNamesAndTypes
|
||||
OK
|
||||
OK
|
||||
OK
|
||||
CSVWithNamesAndTypes
|
||||
OK
|
||||
OK
|
||||
OK
|
||||
JSONCompactEachRowWithNamesAndTypes
|
||||
OK
|
||||
OK
|
||||
OK
|
||||
JSONCompactStringsEachRowWithNamesAndTypes
|
||||
OK
|
||||
OK
|
||||
OK
|
33
tests/queries/0_stateless/02098_with_types_use_header.sh
Executable file
33
tests/queries/0_stateless/02098_with_types_use_header.sh
Executable 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"
|
113
tests/queries/0_stateless/02099_tsv_raw_format.reference
Normal file
113
tests/queries/0_stateless/02099_tsv_raw_format.reference
Normal 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 -
|
59
tests/queries/0_stateless/02099_tsv_raw_format.sh
Executable file
59
tests/queries/0_stateless/02099_tsv_raw_format.sh
Executable 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"
|
||||
|
@ -0,0 +1,14 @@
|
||||
CSV
|
||||
\N
|
||||
TSV
|
||||
\N
|
||||
TSVRaw
|
||||
\N
|
||||
TSKV
|
||||
\N
|
||||
JSONCompactEachRow
|
||||
\N
|
||||
JSONEachRow
|
||||
\N
|
||||
Values
|
||||
\N
|
21
tests/queries/0_stateless/02100_low_cardinality_nullable_null_default.sh
Executable file
21
tests/queries/0_stateless/02100_low_cardinality_nullable_null_default.sh
Executable 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"
|
||||
|
@ -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
|
39
tests/queries/0_stateless/02101_empty_as_default_and_omitted_fields.sh
Executable file
39
tests/queries/0_stateless/02101_empty_as_default_and_omitted_fields.sh
Executable 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"
|
||||
|
@ -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 -
|
||||
|
@ -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
|
||||
|
@ -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 -
|
32
tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.sh
Executable file
32
tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.sh
Executable 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
|
Loading…
Reference in New Issue
Block a user