2021-10-14 10:32:49 +00:00
|
|
|
#include <Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h>
|
|
|
|
|
2019-12-05 10:13:40 +00:00
|
|
|
#include <IO/ReadHelpers.h>
|
2021-12-15 11:30:57 +00:00
|
|
|
#include <IO/PeekableReadBuffer.h>
|
2021-10-14 10:32:49 +00:00
|
|
|
#include <IO/Operators.h>
|
2019-12-05 10:13:40 +00:00
|
|
|
#include <Formats/FormatFactory.h>
|
2021-10-14 10:32:49 +00:00
|
|
|
#include <Formats/verbosePrintString.h>
|
2022-05-06 16:48:48 +00:00
|
|
|
#include <Formats/JSONUtils.h>
|
2022-07-13 15:57:55 +00:00
|
|
|
#include <Formats/EscapingRuleUtils.h>
|
2021-10-20 11:48:54 +00:00
|
|
|
#include <Formats/registerWithNamesAndTypes.h>
|
2019-12-05 10:13:40 +00:00
|
|
|
#include <DataTypes/NestedUtils.h>
|
2021-03-09 14:46:52 +00:00
|
|
|
#include <DataTypes/Serializations/SerializationNullable.h>
|
2021-12-15 11:30:57 +00:00
|
|
|
#include <DataTypes/DataTypeFactory.h>
|
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
|
|
|
#include <Poco/JSON/Parser.h>
|
2019-12-05 10:13:40 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2021-10-14 10:32:49 +00:00
|
|
|
JSONCompactEachRowRowInputFormat::JSONCompactEachRowRowInputFormat(
|
|
|
|
const Block & header_,
|
|
|
|
ReadBuffer & in_,
|
|
|
|
Params params_,
|
|
|
|
bool with_names_,
|
|
|
|
bool with_types_,
|
|
|
|
bool yield_strings_,
|
|
|
|
const FormatSettings & format_settings_)
|
2021-12-15 11:30:57 +00:00
|
|
|
: RowInputFormatWithNamesAndTypes(
|
|
|
|
header_,
|
|
|
|
in_,
|
2022-03-02 17:22:12 +00:00
|
|
|
params_,
|
2022-05-22 12:17:16 +00:00
|
|
|
false,
|
2021-12-15 11:30:57 +00:00
|
|
|
with_names_,
|
|
|
|
with_types_,
|
|
|
|
format_settings_,
|
|
|
|
std::make_unique<JSONCompactEachRowFormatReader>(in_, yield_strings_, format_settings_))
|
2019-12-05 10:13:40 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
void JSONCompactEachRowRowInputFormat::syncAfterError()
|
|
|
|
{
|
|
|
|
skipToUnescapedNextLineOrEOF(*in);
|
|
|
|
}
|
|
|
|
|
|
|
|
JSONCompactEachRowFormatReader::JSONCompactEachRowFormatReader(ReadBuffer & in_, bool yield_strings_, const FormatSettings & format_settings_)
|
|
|
|
: FormatWithNamesAndTypesReader(in_, format_settings_), yield_strings(yield_strings_)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
void JSONCompactEachRowFormatReader::skipRowStartDelimiter()
|
2020-06-11 00:51:27 +00:00
|
|
|
{
|
2021-10-14 10:32:49 +00:00
|
|
|
skipWhitespaceIfAny(*in);
|
|
|
|
assertChar('[', *in);
|
2020-06-11 00:51:27 +00:00
|
|
|
}
|
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
void JSONCompactEachRowFormatReader::skipFieldDelimiter()
|
2019-12-05 10:13:40 +00:00
|
|
|
{
|
2021-10-14 10:32:49 +00:00
|
|
|
skipWhitespaceIfAny(*in);
|
|
|
|
assertChar(',', *in);
|
|
|
|
}
|
2020-06-11 00:51:27 +00:00
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
void JSONCompactEachRowFormatReader::skipRowEndDelimiter()
|
2021-10-14 10:32:49 +00:00
|
|
|
{
|
|
|
|
skipWhitespaceIfAny(*in);
|
|
|
|
assertChar(']', *in);
|
2019-12-05 10:13:40 +00:00
|
|
|
|
2021-10-14 10:32:49 +00:00
|
|
|
skipWhitespaceIfAny(*in);
|
|
|
|
if (!in->eof() && (*in->position() == ',' || *in->position() == ';'))
|
|
|
|
++in->position();
|
2019-12-05 10:13:40 +00:00
|
|
|
|
2021-10-14 10:32:49 +00:00
|
|
|
skipWhitespaceIfAny(*in);
|
|
|
|
}
|
2019-12-05 10:13:40 +00:00
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
void JSONCompactEachRowFormatReader::skipField()
|
2021-10-14 10:32:49 +00:00
|
|
|
{
|
|
|
|
skipWhitespaceIfAny(*in);
|
2021-12-15 11:30:57 +00:00
|
|
|
skipJSONField(*in, "skipped_field");
|
2021-10-14 10:32:49 +00:00
|
|
|
}
|
2019-12-05 10:13:40 +00:00
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
void JSONCompactEachRowFormatReader::skipHeaderRow()
|
2021-10-14 10:32:49 +00:00
|
|
|
{
|
|
|
|
skipRowStartDelimiter();
|
|
|
|
do
|
2019-12-05 10:13:40 +00:00
|
|
|
{
|
2021-12-15 11:30:57 +00:00
|
|
|
skipField();
|
2021-10-14 10:32:49 +00:00
|
|
|
skipWhitespaceIfAny(*in);
|
2019-12-05 10:13:40 +00:00
|
|
|
}
|
2021-10-14 10:32:49 +00:00
|
|
|
while (checkChar(',', *in));
|
|
|
|
|
|
|
|
skipRowEndDelimiter();
|
2019-12-05 10:13:40 +00:00
|
|
|
}
|
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
std::vector<String> JSONCompactEachRowFormatReader::readHeaderRow()
|
2019-12-05 10:13:40 +00:00
|
|
|
{
|
2021-10-14 10:32:49 +00:00
|
|
|
skipRowStartDelimiter();
|
|
|
|
std::vector<String> fields;
|
2021-12-15 11:30:57 +00:00
|
|
|
String field;
|
2021-10-14 10:32:49 +00:00
|
|
|
do
|
2019-12-05 10:13:40 +00:00
|
|
|
{
|
2021-12-15 11:30:57 +00:00
|
|
|
skipWhitespaceIfAny(*in);
|
|
|
|
readJSONString(field, *in);
|
|
|
|
fields.push_back(field);
|
2021-10-14 10:32:49 +00:00
|
|
|
skipWhitespaceIfAny(*in);
|
2019-12-05 10:13:40 +00:00
|
|
|
}
|
2021-10-14 10:32:49 +00:00
|
|
|
while (checkChar(',', *in));
|
2019-12-05 10:13:40 +00:00
|
|
|
|
2021-10-14 10:32:49 +00:00
|
|
|
skipRowEndDelimiter();
|
|
|
|
return fields;
|
|
|
|
}
|
2019-12-05 10:13:40 +00:00
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
bool JSONCompactEachRowFormatReader::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool /*is_last_file_column*/, const String & column_name)
|
2021-10-14 10:32:49 +00:00
|
|
|
{
|
|
|
|
skipWhitespaceIfAny(*in);
|
2022-05-18 09:25:26 +00:00
|
|
|
return JSONUtils::readField(*in, column, type, serialization, column_name, format_settings, yield_strings);
|
2019-12-05 10:13:40 +00:00
|
|
|
}
|
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
bool JSONCompactEachRowFormatReader::parseRowStartWithDiagnosticInfo(WriteBuffer & out)
|
2021-10-14 10:32:49 +00:00
|
|
|
{
|
|
|
|
skipWhitespaceIfAny(*in);
|
|
|
|
if (!checkChar('[', *in))
|
|
|
|
{
|
|
|
|
out << "ERROR: There is no '[' before the row.\n";
|
2019-12-05 10:13:40 +00:00
|
|
|
return false;
|
2021-10-14 10:32:49 +00:00
|
|
|
}
|
2019-12-05 10:13:40 +00:00
|
|
|
|
2021-10-14 10:32:49 +00:00
|
|
|
return true;
|
|
|
|
}
|
2019-12-05 10:13:40 +00:00
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
bool JSONCompactEachRowFormatReader::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out)
|
2021-10-14 10:32:49 +00:00
|
|
|
{
|
|
|
|
try
|
2019-12-05 10:13:40 +00:00
|
|
|
{
|
2021-10-14 10:32:49 +00:00
|
|
|
skipWhitespaceIfAny(*in);
|
|
|
|
assertChar(',', *in);
|
|
|
|
}
|
|
|
|
catch (const DB::Exception &)
|
|
|
|
{
|
|
|
|
if (*in->position() == ']')
|
2019-12-05 10:13:40 +00:00
|
|
|
{
|
2021-10-14 10:32:49 +00:00
|
|
|
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";
|
2019-12-05 10:13:40 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2021-10-14 10:32:49 +00:00
|
|
|
out << "ERROR: There is no comma. ";
|
|
|
|
verbosePrintString(in->position(), in->position() + 1, out);
|
|
|
|
out << " found instead.\n";
|
2019-12-05 10:13:40 +00:00
|
|
|
}
|
2021-10-14 10:32:49 +00:00
|
|
|
return false;
|
2019-12-05 10:13:40 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
bool JSONCompactEachRowFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out)
|
2019-12-05 10:13:40 +00:00
|
|
|
{
|
2021-08-27 03:00:12 +00:00
|
|
|
skipWhitespaceIfAny(*in);
|
2021-10-14 10:32:49 +00:00
|
|
|
|
|
|
|
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() == ';'))
|
2021-08-27 03:00:12 +00:00
|
|
|
++in->position();
|
2019-12-05 10:13:40 +00:00
|
|
|
|
2021-08-27 03:00:12 +00:00
|
|
|
skipWhitespaceIfAny(*in);
|
2021-10-14 10:32:49 +00:00
|
|
|
return true;
|
2019-12-05 10:13:40 +00:00
|
|
|
}
|
|
|
|
|
2022-04-07 12:17:48 +00:00
|
|
|
JSONCompactEachRowRowSchemaReader::JSONCompactEachRowRowSchemaReader(
|
|
|
|
ReadBuffer & in_, bool with_names_, bool with_types_, bool yield_strings_, const FormatSettings & format_settings_)
|
2022-04-13 19:13:40 +00:00
|
|
|
: FormatWithNamesAndTypesSchemaReader(in_, format_settings_, with_names_, with_types_, &reader)
|
2022-03-29 17:37:31 +00:00
|
|
|
, reader(in_, yield_strings_, format_settings_)
|
2021-12-15 11:30:57 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
DataTypes JSONCompactEachRowRowSchemaReader::readRowAndGetDataTypes()
|
|
|
|
{
|
2022-01-20 13:31:24 +00:00
|
|
|
if (first_row)
|
|
|
|
first_row = false;
|
|
|
|
else
|
|
|
|
{
|
|
|
|
skipWhitespaceIfAny(in);
|
|
|
|
/// ',' and ';' are possible between the rows.
|
|
|
|
if (!in.eof() && (*in.position() == ',' || *in.position() == ';'))
|
|
|
|
++in.position();
|
|
|
|
}
|
|
|
|
|
2021-12-16 18:48:38 +00:00
|
|
|
skipWhitespaceIfAny(in);
|
2021-12-15 11:30:57 +00:00
|
|
|
if (in.eof())
|
|
|
|
return {};
|
|
|
|
|
2022-07-13 15:57:55 +00:00
|
|
|
return JSONUtils::readRowAndGetDataTypesForJSONCompactEachRow(in, format_settings, reader.yieldStrings());
|
|
|
|
}
|
|
|
|
|
|
|
|
void JSONCompactEachRowRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type, size_t)
|
|
|
|
{
|
|
|
|
transformInferredJSONTypesIfNeeded(type, new_type, format_settings);
|
2021-12-15 11:30:57 +00:00
|
|
|
}
|
|
|
|
|
2021-10-14 10:32:49 +00:00
|
|
|
void registerInputFormatJSONCompactEachRow(FormatFactory & factory)
|
2019-12-05 10:13:40 +00:00
|
|
|
{
|
2021-10-14 10:32:49 +00:00
|
|
|
for (bool yield_strings : {true, false})
|
2019-12-05 10:13:40 +00:00
|
|
|
{
|
2021-10-20 12:47:20 +00:00
|
|
|
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
|
2020-09-02 04:05:02 +00:00
|
|
|
{
|
2021-10-20 12:47:20 +00:00
|
|
|
factory.registerInputFormat(format_name, [with_names, with_types, yield_strings](
|
2021-10-14 10:32:49 +00:00
|
|
|
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);
|
2021-10-20 12:47:20 +00:00
|
|
|
});
|
2021-10-14 10:32:49 +00:00
|
|
|
};
|
2020-09-02 04:05:02 +00:00
|
|
|
|
2021-10-20 12:47:20 +00:00
|
|
|
registerWithNamesAndTypes(yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", register_func);
|
2022-05-13 13:51:28 +00:00
|
|
|
markFormatWithNamesAndTypesSupportsSamplingColumns(yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", factory);
|
2019-12-05 10:13:40 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
void registerJSONCompactEachRowSchemaReader(FormatFactory & factory)
|
|
|
|
{
|
|
|
|
for (bool json_strings : {false, true})
|
|
|
|
{
|
|
|
|
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
|
|
|
|
{
|
2022-03-24 12:54:12 +00:00
|
|
|
factory.registerSchemaReader(format_name, [=](ReadBuffer & buf, const FormatSettings & settings)
|
2021-12-15 11:30:57 +00:00
|
|
|
{
|
|
|
|
return std::make_shared<JSONCompactEachRowRowSchemaReader>(buf, with_names, with_types, json_strings, settings);
|
|
|
|
});
|
|
|
|
};
|
|
|
|
registerWithNamesAndTypes(json_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", register_func);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-10-14 10:32:49 +00:00
|
|
|
void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory)
|
2019-12-05 10:13:40 +00:00
|
|
|
{
|
2021-10-20 12:47:20 +00:00
|
|
|
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
|
2021-10-20 11:48:54 +00:00
|
|
|
{
|
2021-10-27 19:16:34 +00:00
|
|
|
/// In case when we have names and/or types in the first two/one rows,
|
|
|
|
/// we need to read at least one more row of actual data. So, set
|
|
|
|
/// the minimum of rows for segmentation engine according to
|
|
|
|
/// parameters with_names and with_types.
|
2021-10-20 12:47:20 +00:00
|
|
|
size_t min_rows = 1 + int(with_names) + int(with_types);
|
|
|
|
factory.registerFileSegmentationEngine(format_name, [min_rows](ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
|
2021-10-20 11:48:54 +00:00
|
|
|
{
|
2022-05-18 09:25:26 +00:00
|
|
|
return JSONUtils::fileSegmentationEngineJSONCompactEachRow(in, memory, min_chunk_size, min_rows);
|
2021-10-20 12:47:20 +00:00
|
|
|
});
|
2021-10-20 11:48:54 +00:00
|
|
|
};
|
|
|
|
|
2021-10-20 12:47:20 +00:00
|
|
|
registerWithNamesAndTypes("JSONCompactEachRow", register_func);
|
|
|
|
registerWithNamesAndTypes("JSONCompactStringsEachRow", register_func);
|
2019-12-05 10:13:40 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|