mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 17:44:23 +00:00
Allow data in square brackets in JSONAsString format
This commit is contained in:
parent
93f15e443d
commit
92e6df7b89
@ -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;
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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() == '[';
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -359,4 +359,10 @@ void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory)
|
||||
factory.registerFileSegmentationEngine("JSONStringsEachRow", &fileSegmentationEngineJSONEachRowImpl);
|
||||
}
|
||||
|
||||
void registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(FormatFactory & factory)
|
||||
{
|
||||
factory.registerNonTrivialPrefixAndSuffixChecker("JSONEachRow", nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl);
|
||||
factory.registerNonTrivialPrefixAndSuffixChecker("JSONStringsEachRow", nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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"
|
||||
|
||||
|
@ -0,0 +1,2 @@
|
||||
{"id": 1, "name": "name1"}
|
||||
{"id": 2, "name": "name2"}
|
@ -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;
|
@ -0,0 +1 @@
|
||||
1000000
|
9
tests/queries/0_stateless/01926_json_as_string_array.sh
Executable file
9
tests/queries/0_stateless/01926_json_as_string_array.sh
Executable 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'
|
Loading…
Reference in New Issue
Block a user