Add setting to obtain object name as column value in JSONObjectEachRow format

This commit is contained in:
avogar 2022-09-22 16:48:54 +00:00
parent 2c83abaaba
commit d3d06251a3
11 changed files with 128 additions and 17 deletions

View File

@ -772,6 +772,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(Bool, output_format_json_array_of_rows, false, "Output a JSON array of all rows in JSONEachRow(Compact) format.", 0) \
M(Bool, output_format_json_validate_utf8, false, "Validate UTF-8 sequences in JSON output formats, doesn't impact formats JSON/JSONCompact/JSONColumnsWithMetadata, they always validate utf8", 0) \
\
M(String, format_json_object_each_row_column_for_object_name, "", "The name of column that will be used as object names in JSONObjectEachRow format. Column type should be String", 0) \
\
M(UInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.", 0) \
M(UInt64, output_format_pretty_max_column_pad_width, 250, "Maximum width to pad all values in a column in Pretty formats.", 0) \
M(UInt64, output_format_pretty_max_value_width, 10000, "Maximum width of value to display in Pretty formats. If greater - it will be cut.", 0) \

View File

@ -100,6 +100,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.json.try_infer_numbers_from_strings = settings.input_format_json_try_infer_numbers_from_strings;
format_settings.json.validate_types_from_metadata = settings.input_format_json_validate_types_from_metadata;
format_settings.json.validate_utf8 = settings.output_format_json_validate_utf8;
format_settings.json_object_each_row.column_for_object_name = settings.format_json_object_each_row_column_for_object_name;
format_settings.null_as_default = settings.input_format_null_as_default;
format_settings.decimal_trailing_zeros = settings.output_format_decimal_trailing_zeros;
format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size;

View File

@ -155,6 +155,11 @@ struct FormatSettings
bool validate_utf8 = false;
} json;
struct
{
String column_for_object_name;
} json_object_each_row;
struct
{
UInt64 row_group_size = 1000000;

View File

@ -214,7 +214,7 @@ bool JSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi
seen_columns.assign(num_columns, false);
nested_prefix_length = 0;
readRowStart();
readRowStart(columns);
readJSONObject(columns);
const auto & header = getPort().getHeader();

View File

@ -48,7 +48,7 @@ private:
void readJSONObject(MutableColumns & columns);
void readNestedData(const String & name, MutableColumns & columns);
virtual void readRowStart() {}
virtual void readRowStart(MutableColumns &) {}
virtual bool checkEndOfData(bool is_first_row);
const FormatSettings format_settings;
@ -66,10 +66,6 @@ private:
/// the nested column names are 'n.i' and 'n.s' and the nested prefix is 'n.'
size_t nested_prefix_length = 0;
/// Set of columns for which the values were read. The rest will be filled with default values.
std::vector<UInt8> read_columns;
/// Set of columns which already met in row. Exception is thrown if there are more than one column with the same name.
std::vector<UInt8> seen_columns;
/// These sets may be different, because if null_as_default=1 read_columns[i] will be false and seen_columns[i] will be true
/// for row like {..., "non-nullable column name" : null, ...}
@ -85,6 +81,12 @@ private:
bool yield_strings;
protected:
/// Set of columns for which the values were read. The rest will be filled with default values.
std::vector<UInt8> read_columns;
/// Set of columns which already met in row. Exception is thrown if there are more than one column with the same name.
std::vector<UInt8> seen_columns;
/// This flag is needed to know if data is in square brackets.
bool data_in_square_brackets = false;
};

View File

@ -2,12 +2,39 @@
#include <Formats/JSONUtils.h>
#include <Formats/FormatFactory.h>
#include <Formats/EscapingRuleUtils.h>
#include <DataTypes/DataTypeString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
std::optional<size_t> getColumnIndexForJSONObjectEachRowObjectName(const Block & header, const FormatSettings & format_settings)
{
if (format_settings.json_object_each_row.column_for_object_name.empty())
return std::nullopt;
if (!header.has(format_settings.json_object_each_row.column_for_object_name))
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Column name '{}' from setting format_json_object_each_row_column_for_object_name doesn't exists in header",
format_settings.json_object_each_row.column_for_object_name);
size_t index = header.getPositionByName(format_settings.json_object_each_row.column_for_object_name);
if (!isStringOrFixedString(header.getDataTypes()[index]))
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Column '{}' from setting json_object_each_row_column_for_object_name must have String type",
format_settings.json_object_each_row.column_for_object_name);
return index;
}
JSONObjectEachRowInputFormat::JSONObjectEachRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_)
: JSONEachRowRowInputFormat(in_, header_, params_, format_settings_, false)
: JSONEachRowRowInputFormat(in_, header_, params_, format_settings_, false), field_index_for_object_name(getColumnIndexForJSONObjectEachRowObjectName(header_, format_settings_))
{
}
@ -16,9 +43,15 @@ void JSONObjectEachRowInputFormat::readPrefix()
JSONUtils::skipObjectStart(*in);
}
void JSONObjectEachRowInputFormat::readRowStart()
void JSONObjectEachRowInputFormat::readRowStart(MutableColumns & columns)
{
JSONUtils::readFieldName(*in);
auto object_name = JSONUtils::readFieldName(*in);
if (field_index_for_object_name)
{
columns[*field_index_for_object_name]->insertData(object_name.data(), object_name.size());
seen_columns[*field_index_for_object_name] = true;
read_columns[*field_index_for_object_name] = true;
}
}
bool JSONObjectEachRowInputFormat::checkEndOfData(bool is_first_row)
@ -30,7 +63,6 @@ bool JSONObjectEachRowInputFormat::checkEndOfData(bool is_first_row)
return false;
}
JSONObjectEachRowSchemaReader::JSONObjectEachRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
: IRowWithNamesSchemaReader(in_, format_settings_)
{
@ -53,7 +85,10 @@ NamesAndTypesList JSONObjectEachRowSchemaReader::readRowAndGetNamesAndDataTypes(
JSONUtils::skipComma(in);
JSONUtils::readFieldName(in);
return JSONUtils::readRowAndGetNamesAndDataTypesForJSONEachRow(in, format_settings, false);
auto names_and_types = JSONUtils::readRowAndGetNamesAndDataTypesForJSONEachRow(in, format_settings, false);
if (!format_settings.json_object_each_row.column_for_object_name.empty())
names_and_types.emplace_front(format_settings.json_object_each_row.column_for_object_name, std::make_shared<DataTypeString>());
return names_and_types;
}
void JSONObjectEachRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)

View File

@ -27,8 +27,10 @@ public:
private:
void readPrefix() override;
void readSuffix() override {}
void readRowStart() override;
void readRowStart(MutableColumns & columns) override;
bool checkEndOfData(bool is_first_row) override;
std::optional<size_t> field_index_for_object_name;
};
@ -44,4 +46,6 @@ private:
bool first_row = true;
};
std::optional<size_t> getColumnIndexForJSONObjectEachRowObjectName(const Block & header, const FormatSettings & settings);
}

View File

@ -1,4 +1,5 @@
#include <Processors/Formats/Impl/JSONObjectEachRowRowOutputFormat.h>
#include <Processors/Formats/Impl/JSONObjectEachRowRowInputFormat.h>
#include <Formats/JSONUtils.h>
#include <IO/WriteHelpers.h>
@ -6,10 +7,38 @@ namespace DB
{
JSONObjectEachRowRowOutputFormat::JSONObjectEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & settings_)
: JSONEachRowRowOutputFormat(out_, header_, params_, settings_)
: JSONEachRowRowOutputFormat(out_, header_, params_, settings_), field_index_for_object_name(getColumnIndexForJSONObjectEachRowObjectName(header_, settings_))
{
}
void JSONObjectEachRowRowOutputFormat::writeField(const IColumn & column, const ISerialization & serialization, size_t row)
{
if (field_number == field_index_for_object_name)
{
++field_number;
return;
}
JSONEachRowRowOutputFormat::writeField(column, serialization, row);
}
void JSONObjectEachRowRowOutputFormat::write(const Columns & columns, size_t row)
{
if (field_index_for_object_name)
object_name = columns[*field_index_for_object_name]->getDataAt(row).toString();
else
object_name = "row_" + std::to_string(row + 1);
IRowOutputFormat::write(columns, row);
}
void JSONObjectEachRowRowOutputFormat::writeFieldDelimiter()
{
/// We should not write comma before column that is used for
/// object name and also after it if it's in the first place
if (field_number != field_index_for_object_name && !(field_index_for_object_name == 0 && field_number == 1))
JSONEachRowRowOutputFormat::writeFieldDelimiter();
}
void JSONObjectEachRowRowOutputFormat::writePrefix()
{
JSONUtils::writeObjectStart(*ostr);
@ -17,9 +46,7 @@ void JSONObjectEachRowRowOutputFormat::writePrefix()
void JSONObjectEachRowRowOutputFormat::writeRowStartDelimiter()
{
++row_num;
String title = "row_" + std::to_string(row_num);
JSONUtils::writeCompactObjectStart(*ostr, 1, title.c_str());
JSONUtils::writeCompactObjectStart(*ostr, 1, object_name.c_str());
}
void JSONObjectEachRowRowOutputFormat::writeRowEndDelimiter()

View File

@ -29,6 +29,9 @@ public:
String getName() const override { return "JSONObjectEachRowRowOutputFormat"; }
private:
void write(const Columns & columns, size_t row) override;
void writeField(const IColumn & column, const ISerialization & serialization, size_t row) override;
void writeFieldDelimiter() override;
void writeRowStartDelimiter() override;
void writeRowEndDelimiter() override;
void writeRowBetweenDelimiter() override;
@ -36,7 +39,8 @@ private:
void writePrefix() override;
void writeSuffix() override;
size_t row_num = 0;
std::optional<size_t> field_index_for_object_name;
String object_name;
};
}

View File

@ -0,0 +1,20 @@
{
"name_0": {"number":"0"},
"name_1": {"number":"1"},
"name_2": {"number":"2"}
}
{
"name_0": {"number":"0","x":"1"},
"name_1": {"number":"1","x":"2"},
"name_2": {"number":"2","x":"3"}
}
{
"name_0": {"number":"0"},
"name_1": {"number":"1"},
"name_2": {"number":"2"}
}
name String
number Nullable(Int64)
name_0 0
name_1 1
name_2 2

View File

@ -0,0 +1,11 @@
-- Tags: no-fasttest, no-parallel
set format_json_object_each_row_column_for_object_name='name';
select number, concat('name_', toString(number)) as name from numbers(3) format JSONObjectEachRow;
select number, concat('name_', toString(number)) as name, number + 1 as x from numbers(3) format JSONObjectEachRow;
select concat('name_', toString(number)) as name, number from numbers(3) format JSONObjectEachRow;
insert into function file(02454_data.jsonobjecteachrow) select number, concat('name_', toString(number)) as name from numbers(3) settings engine_file_truncate_on_insert=1;
desc file(02454_data.jsonobjecteachrow);
select * from file(02454_data.jsonobjecteachrow);