Allow data in square brackets in JSONAsString format

This commit is contained in:
Pavel Kruglov 2021-06-23 16:17:34 +03:00
parent 93f15e443d
commit 92e6df7b89
13 changed files with 164 additions and 13 deletions

View File

@ -162,14 +162,12 @@ InputFormatPtr FormatFactory::getInput(
if (settings.max_memory_usage_for_user && settings.min_chunk_bytes_for_parallel_parsing * settings.max_threads * 2 > settings.max_memory_usage_for_user)
parallel_parsing = false;
if (parallel_parsing && name == "JSONEachRow")
if (parallel_parsing)
{
/// FIXME ParallelParsingBlockInputStream doesn't support formats with non-trivial readPrefix() and readSuffix()
/// For JSONEachRow we can safely skip whitespace characters
skipWhitespaceIfAny(buf);
if (buf.eof() || *buf.position() == '[')
parallel_parsing = false; /// Disable it for JSONEachRow if data is in square brackets (see JSONEachRowRowInputFormat)
const auto & non_trivial_prefix_and_suffix_checker = getCreators(name).non_trivial_prefix_and_suffix_checker;
/// Disable parallel parsing for input formats with non-trivial readPrefix() and readSuffix().
if (non_trivial_prefix_and_suffix_checker && non_trivial_prefix_and_suffix_checker(buf))
parallel_parsing = false;
}
if (parallel_parsing)
@ -396,6 +394,14 @@ void FormatFactory::registerInputFormatProcessor(const String & name, InputProce
target = std::move(input_creator);
}
void FormatFactory::registerNonTrivialPrefixAndSuffixChecker(const String & name, NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker)
{
auto & target = dict[name].non_trivial_prefix_and_suffix_checker;
if (target)
throw Exception("FormatFactory: Non trivial prefix and suffix checker " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
target = std::move(non_trivial_prefix_and_suffix_checker);
}
void FormatFactory::registerOutputFormatProcessor(const String & name, OutputProcessorCreator output_creator)
{
auto & target = dict[name].output_processor_creator;

View File

@ -93,6 +93,11 @@ private:
const RowOutputFormatParams & params,
const FormatSettings & settings)>;
/// Some input formats can have non trivial readPrefix() and readSuffix(),
/// so in some cases there is no possibility to use parallel parsing.
/// The checker should return true if parallel parsing should be disabled.
using NonTrivialPrefixAndSuffixChecker = std::function<bool(ReadBuffer & buf)>;
struct Creators
{
InputCreator input_creator;
@ -102,6 +107,7 @@ private:
FileSegmentationEngine file_segmentation_engine;
bool supports_parallel_formatting{false};
bool is_column_oriented{false};
NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker;
};
using FormatsDictionary = std::unordered_map<String, Creators>;
@ -166,6 +172,8 @@ public:
void registerOutputFormat(const String & name, OutputCreator output_creator);
void registerFileSegmentationEngine(const String & name, FileSegmentationEngine file_segmentation_engine);
void registerNonTrivialPrefixAndSuffixChecker(const String & name, NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker);
void registerInputFormatProcessor(const String & name, InputProcessorCreator input_creator);
void registerOutputFormatProcessor(const String & name, OutputProcessorCreator output_creator);

View File

@ -83,4 +83,11 @@ std::pair<bool, size_t> fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, D
return {loadAtPosition(in, memory, pos), number_of_rows};
}
bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf)
{
/// For JSONEachRow we can safely skip whitespace characters
skipWhitespaceIfAny(buf);
return buf.eof() || *buf.position() == '[';
}
}

View File

@ -5,4 +5,6 @@ namespace DB
std::pair<bool, size_t> fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size);
bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf);
}

View File

@ -79,6 +79,9 @@ void registerInputFormatProcessorJSONAsString(FormatFactory & factory);
void registerInputFormatProcessorLineAsString(FormatFactory & factory);
void registerInputFormatProcessorCapnProto(FormatFactory & factory);
/// Non trivial prefix and suffix checkers for disabling parallel parsing.
void registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(FormatFactory & factory);
void registerNonTrivialPrefixAndSuffixCheckerJSONAsString(FormatFactory & factory);
void registerFormats()
{
@ -152,6 +155,9 @@ void registerFormats()
#if !defined(ARCADIA_BUILD)
registerInputFormatProcessorCapnProto(factory);
#endif
registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(factory);
registerNonTrivialPrefixAndSuffixCheckerJSONAsString(factory);
}
}

View File

@ -34,6 +34,35 @@ void JSONAsStringRowInputFormat::resetParser()
buf.reset();
}
void JSONAsStringRowInputFormat::readPrefix()
{
/// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it.
skipBOMIfExists(buf);
skipWhitespaceIfAny(buf);
if (!buf.eof() && *buf.position() == '[')
{
++buf.position();
data_in_square_brackets = true;
}
}
void JSONAsStringRowInputFormat::readSuffix()
{
skipWhitespaceIfAny(buf);
if (data_in_square_brackets)
{
assertChar(']', buf);
skipWhitespaceIfAny(buf);
}
if (!buf.eof() && *buf.position() == ';')
{
++buf.position();
skipWhitespaceIfAny(buf);
}
assertEOF(buf);
}
void JSONAsStringRowInputFormat::readJSONObject(IColumn & column)
{
PeekableReadBufferCheckpoint checkpoint{buf};
@ -113,7 +142,23 @@ void JSONAsStringRowInputFormat::readJSONObject(IColumn & column)
bool JSONAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &)
{
if (!allow_new_rows)
return false;
skipWhitespaceIfAny(buf);
if (!buf.eof())
{
if (!data_in_square_brackets && *buf.position() == ';')
{
/// ';' means the end of query, but it cannot be before ']'.
return allow_new_rows = false;
}
else if (data_in_square_brackets && *buf.position() == ']')
{
/// ']' means the end of query.
return allow_new_rows = false;
}
}
if (!buf.eof())
readJSONObject(*columns[0]);
@ -143,4 +188,9 @@ void registerFileSegmentationEngineJSONAsString(FormatFactory & factory)
factory.registerFileSegmentationEngine("JSONAsString", &fileSegmentationEngineJSONEachRowImpl);
}
void registerNonTrivialPrefixAndSuffixCheckerJSONAsString(FormatFactory & factory)
{
factory.registerNonTrivialPrefixAndSuffixChecker("JSONAsString", nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl);
}
}

View File

@ -22,10 +22,17 @@ public:
String getName() const override { return "JSONAsStringRowInputFormat"; }
void resetParser() override;
void readPrefix() override;
void readSuffix() override;
private:
void readJSONObject(IColumn & column);
PeekableReadBuffer buf;
/// This flag is needed to know if data is in square brackets.
bool data_in_square_brackets = false;
bool allow_new_rows = true;
};
}

View File

@ -359,4 +359,10 @@ void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory)
factory.registerFileSegmentationEngine("JSONStringsEachRow", &fileSegmentationEngineJSONEachRowImpl);
}
void registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(FormatFactory & factory)
{
factory.registerNonTrivialPrefixAndSuffixChecker("JSONEachRow", nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl);
factory.registerNonTrivialPrefixAndSuffixChecker("JSONStringsEachRow", nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl);
}
}

View File

@ -28,21 +28,59 @@ echo '
"array" : [3, 2, 1],
"map" : {
"z" : 1,
"y" : 2,
"y" : 2,
"x" : 3
}
}
{
"id" : 3,
"date" : "01.03.2020",
"string" : "one more string",
"array" : [3,1,2],
"id" : 3,
"date" : "01.03.2020",
"string" : "one more string",
"array" : [3,1,2],
"map" : {
"{" : 1,
"{" : 1,
"}}" : 2
}
}' | $CLICKHOUSE_CLIENT --query="INSERT INTO json_as_string FORMAT JSONAsString";
echo '
[
{
"id" : 1,
"date" : "01.01.2020",
"string" : "123{{{\"\\",
"array" : [1, 2, 3],
"map": {
"a" : 1,
"b" : 2,
"c" : 3
}
},
{
"id" : 2,
"date" : "01.02.2020",
"string" : "{another\"
string}}",
"array" : [3, 2, 1],
"map" : {
"z" : 1,
"y" : 2,
"x" : 3
}
}
{
"id" : 3,
"date" : "01.03.2020",
"string" : "one more string",
"array" : [3,1,2],
"map" : {
"{" : 1,
"}}" : 2
}
}
]' | $CLICKHOUSE_CLIENT --query="INSERT INTO json_as_string FORMAT JSONAsString";
$CLICKHOUSE_CLIENT --query="SELECT * FROM json_as_string";
$CLICKHOUSE_CLIENT --query="DROP TABLE json_as_string"

View File

@ -0,0 +1,2 @@
{"id": 1, "name": "name1"}
{"id": 2, "name": "name2"}

View File

@ -0,0 +1,9 @@
DROP TABLE IF EXISTS json_square_brackets;
CREATE TABLE json_square_brackets (field String) ENGINE = Memory;
INSERT INTO json_square_brackets FORMAT JSONAsString [{"id": 1, "name": "name1"}, {"id": 2, "name": "name2"}];
INSERT INTO json_square_brackets FORMAT JSONAsString[];
INSERT INTO json_square_brackets FORMAT JSONAsString [ ] ;
INSERT INTO json_square_brackets FORMAT JSONEachRow ;
SELECT * FROM json_square_brackets;
DROP TABLE IF EXISTS json_square_brackets;

View File

@ -0,0 +1 @@
1000000

View File

@ -0,0 +1,9 @@
#!/usr/bin/env bash
set -e
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_LOCAL} --query "SELECT '[' || arrayStringConcat(arrayMap(x -> '{\"id\": 1, \"name\": \"name1\"}', range(1000000)), ',') || ']'" | ${CLICKHOUSE_LOCAL} --query "SELECT count() FROM table" --input-format JSONAsString --structure 'field String'