Merge pull request #38477 from Avogar/sql-insert-format

Add SQLInsert output format
This commit is contained in:
Kruglov Pavel 2022-07-04 15:06:33 +02:00 committed by GitHub
commit 4080f055b6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 329 additions and 62 deletions

View File

@ -11,68 +11,69 @@ results of a `SELECT`, and to perform `INSERT`s into a file-backed table.
The supported formats are:
| Format | Input | Output |
|-------------------------------------------------------------------------------------------|-------|--------|
| [TabSeparated](#tabseparated) | ✔ | ✔ |
| [TabSeparatedRaw](#tabseparatedraw) | ✔ | ✔ |
| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ |
| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ |
| [TabSeparatedRawWithNames](#tabseparatedrawwithnames) | ✔ | ✔ |
| [TabSeparatedRawWithNamesAndTypes](#tabseparatedrawwithnamesandtypes) | ✔ | ✔ |
| [Template](#format-template) | ✔ | ✔ |
| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ |
| [CSV](#csv) | ✔ | ✔ |
| [CSVWithNames](#csvwithnames) | ✔ | ✔ |
| [CSVWithNamesAndTypes](#csvwithnamesandtypes) | ✔ | ✔ |
| [CustomSeparated](#format-customseparated) | ✔ | ✔ |
| [CustomSeparatedWithNames](#customseparatedwithnames) | ✔ | ✔ |
| [CustomSeparatedWithNamesAndTypes](#customseparatedwithnamesandtypes) | ✔ | ✔ |
| [Values](#data-format-values) | ✔ | ✔ |
| [Vertical](#vertical) | ✗ | ✔ |
| [JSON](#json) | ✗ | ✔ |
| [JSONAsString](#jsonasstring) | ✔ | ✗ |
| [JSONStrings](#jsonstrings) | ✗ | ✔ |
| [JSONColumns](#jsoncolumns) | ✔ | ✔ |
| [JSONColumnsWithMetadata](#jsoncolumnswithmetadata) | ✗ | ✔ |
| [JSONCompact](#jsoncompact) | ✗ | ✔ |
| [JSONCompactStrings](#jsoncompactstrings) | ✗ | ✔ |
| [JSONCompactColumns](#jsoncompactcolumns) | ✔ | ✔ |
| [JSONEachRow](#jsoneachrow) | ✔ | ✔ |
| [JSONEachRowWithProgress](#jsoneachrowwithprogress) | ✗ | ✔ |
| [JSONStringsEachRow](#jsonstringseachrow) | ✔ | ✔ |
| [JSONStringsEachRowWithProgress](#jsonstringseachrowwithprogress) | ✗ | ✔ |
| [JSONCompactEachRow](#jsoncompacteachrow) | ✔ | ✔ |
| [JSONCompactEachRowWithNames](#jsoncompacteachrowwithnames) | ✔ | ✔ |
| [JSONCompactEachRowWithNamesAndTypes](#jsoncompacteachrowwithnamesandtypes) | ✔ | ✔ |
| [JSONCompactStringsEachRow](#jsoncompactstringseachrow) | ✔ | ✔ |
| [JSONCompactStringsEachRowWithNames](#jsoncompactstringseachrowwithnames) | ✔ | ✔ |
| [JSONCompactStringsEachRowWithNamesAndTypes](#jsoncompactstringseachrowwithnamesandtypes) | ✔ | ✔ |
| [TSKV](#tskv) | ✔ | ✔ |
| [Pretty](#pretty) | ✗ | ✔ |
| [PrettyCompact](#prettycompact) | ✗ | ✔ |
| [PrettyCompactMonoBlock](#prettycompactmonoblock) | ✗ | ✔ |
| [PrettyNoEscapes](#prettynoescapes) | ✗ | ✔ |
| [PrettySpace](#prettyspace) | ✗ | ✔ |
| [Prometheus](#prometheus) | ✗ | ✔ |
| [Protobuf](#protobuf) | ✔ | ✔ |
| [ProtobufSingle](#protobufsingle) | ✔ | ✔ |
| [Avro](#data-format-avro) | ✔ | ✔ |
| [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ |
| [Parquet](#data-format-parquet) | ✔ | ✔ |
| [Arrow](#data-format-arrow) | ✔ | ✔ |
| [ArrowStream](#data-format-arrow-stream) | ✔ | ✔ |
| [ORC](#data-format-orc) | ✔ | ✔ |
| [RowBinary](#rowbinary) | ✔ | ✔ |
| [RowBinaryWithNames](#rowbinarywithnamesandtypes) | ✔ | ✔ |
| [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ |
| [Native](#native) | ✔ | ✔ |
| [Null](#null) | ✗ | ✔ |
| [XML](#xml) | ✗ | ✔ |
| [CapnProto](#capnproto) | ✔ | ✔ |
| [LineAsString](#lineasstring) | ✔ | ✗ |
| [Regexp](#data-format-regexp) | ✔ | ✗ |
| [RawBLOB](#rawblob) | ✔ | ✔ |
| [MsgPack](#msgpack) | ✔ | ✔ |
| [MySQLDump](#mysqldump) | ✔ | ✗ |
|-------------------------------------------------------------------------------------------|------|--------|
| [TabSeparated](#tabseparated) | ✔ | ✔ |
| [TabSeparatedRaw](#tabseparatedraw) | ✔ | ✔ |
| [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ |
| [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ |
| [TabSeparatedRawWithNames](#tabseparatedrawwithnames) | ✔ | ✔ |
| [TabSeparatedRawWithNamesAndTypes](#tabseparatedrawwithnamesandtypes) | ✔ | ✔ |
| [Template](#format-template) | ✔ | ✔ |
| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ |
| [CSV](#csv) | ✔ | ✔ |
| [CSVWithNames](#csvwithnames) | ✔ | ✔ |
| [CSVWithNamesAndTypes](#csvwithnamesandtypes) | ✔ | ✔ |
| [CustomSeparated](#format-customseparated) | ✔ | ✔ |
| [CustomSeparatedWithNames](#customseparatedwithnames) | ✔ | ✔ |
| [CustomSeparatedWithNamesAndTypes](#customseparatedwithnamesandtypes) | ✔ | ✔ |
| [SQLInsert](#sqlinsert) | ✗ | ✔ |
| [Values](#data-format-values) | ✔ | ✔ |
| [Vertical](#vertical) | ✗ | ✔ |
| [JSON](#json) | ✗ | ✔ |
| [JSONAsString](#jsonasstring) | ✔ | ✗ |
| [JSONStrings](#jsonstrings) | ✗ | ✔ |
| [JSONColumns](#jsoncolumns) | ✔ | ✔ |
| [JSONColumnsWithMetadata](#jsoncolumnswithmetadata) | ✗ | ✔ |
| [JSONCompact](#jsoncompact) | ✗ | ✔ |
| [JSONCompactStrings](#jsoncompactstrings) | ✗ | ✔ |
| [JSONCompactColumns](#jsoncompactcolumns) | ✔ | ✔ |
| [JSONEachRow](#jsoneachrow) | ✔ | ✔ |
| [JSONEachRowWithProgress](#jsoneachrowwithprogress) | ✗ | ✔ |
| [JSONStringsEachRow](#jsonstringseachrow) | ✔ | ✔ |
| [JSONStringsEachRowWithProgress](#jsonstringseachrowwithprogress) | ✗ | ✔ |
| [JSONCompactEachRow](#jsoncompacteachrow) | ✔ | ✔ |
| [JSONCompactEachRowWithNames](#jsoncompacteachrowwithnames) | ✔ | ✔ |
| [JSONCompactEachRowWithNamesAndTypes](#jsoncompacteachrowwithnamesandtypes) | ✔ | ✔ |
| [JSONCompactStringsEachRow](#jsoncompactstringseachrow) | ✔ | ✔ |
| [JSONCompactStringsEachRowWithNames](#jsoncompactstringseachrowwithnames) | ✔ | ✔ |
| [JSONCompactStringsEachRowWithNamesAndTypes](#jsoncompactstringseachrowwithnamesandtypes) | ✔ | ✔ |
| [TSKV](#tskv) | ✔ | ✔ |
| [Pretty](#pretty) | ✗ | ✔ |
| [PrettyCompact](#prettycompact) | ✗ | ✔ |
| [PrettyCompactMonoBlock](#prettycompactmonoblock) | ✗ | ✔ |
| [PrettyNoEscapes](#prettynoescapes) | ✗ | ✔ |
| [PrettySpace](#prettyspace) | ✗ | ✔ |
| [Prometheus](#prometheus) | ✗ | ✔ |
| [Protobuf](#protobuf) | ✔ | ✔ |
| [ProtobufSingle](#protobufsingle) | ✔ | ✔ |
| [Avro](#data-format-avro) | ✔ | ✔ |
| [AvroConfluent](#data-format-avro-confluent) | ✔ | ✗ |
| [Parquet](#data-format-parquet) | ✔ | ✔ |
| [Arrow](#data-format-arrow) | ✔ | ✔ |
| [ArrowStream](#data-format-arrow-stream) | ✔ | ✔ |
| [ORC](#data-format-orc) | ✔ | ✔ |
| [RowBinary](#rowbinary) | ✔ | ✔ |
| [RowBinaryWithNames](#rowbinarywithnamesandtypes) | ✔ | ✔ |
| [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ |
| [Native](#native) | ✔ | ✔ |
| [Null](#null) | ✗ | ✔ |
| [XML](#xml) | ✗ | ✔ |
| [CapnProto](#capnproto) | ✔ | ✔ |
| [LineAsString](#lineasstring) | ✔ | ✗ |
| [Regexp](#data-format-regexp) | ✔ | ✗ |
| [RawBLOB](#rawblob) | ✔ | ✔ |
| [MsgPack](#msgpack) | ✔ | ✔ |
| [MySQLDump](#mysqldump) | ✔ | ✗ |
You can control some format processing parameters with the ClickHouse settings. For more information read the [Settings](../operations/settings/settings.md) section.
@ -468,6 +469,34 @@ Also prints the header row with column names, similar to [TabSeparatedWithNames]
Also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes).
## SQLInsert {#sqlinsert}
Outputs data as a sequence of `INSERT INTO table (columns...) VALUES (...), (...) ...;` statements.
Example:
```sql
SELECT number AS x, number + 1 AS y, 'Hello' AS z FROM numbers(10) FORMAT SQLInsert SETTINGS output_format_sql_insert_max_batch_size = 2
```
```sql
INSERT INTO table (x, y, z) VALUES (0, 1, 'Hello'), (1, 2, 'Hello');
INSERT INTO table (x, y, z) VALUES (2, 3, 'Hello'), (3, 4, 'Hello');
INSERT INTO table (x, y, z) VALUES (4, 5, 'Hello'), (5, 6, 'Hello');
INSERT INTO table (x, y, z) VALUES (6, 7, 'Hello'), (7, 8, 'Hello');
INSERT INTO table (x, y, z) VALUES (8, 9, 'Hello'), (9, 10, 'Hello');
```
To read data output by this format ypu can use [MySQLDump](#mysqldump) input format.
### SQLInsert format settings {#sqlinsert-format-settings}
- [output_format_sql_insert_max_batch_size](../operations/settings/settings.md#output_format_sql_insert_max_batch_size) - The maximum number of rows in one INSERT statement. Default value - `65505`.
- [output_format_sql_insert_table_name](../operations/settings/settings.md#output_format_sql_insert_table_name) - The name of table in the output INSERT query. Default value - `'table'`.
- [output_format_sql_insert_include_column_names](../operations/settings/settings.md#output_format_sql_insert_include_column_names) - Include column names in INSERT query. Default value - `true`.
- [output_format_sql_insert_use_replace](../operations/settings/settings.md#output_format_sql_insert_use_replace) - Use REPLACE statement instead of INSERT. Default value - `false`.
- [output_format_sql_insert_quote_names](../operations/settings/settings.md#output_format_sql_insert_quote_names) - Quote column names with "\`" characters . Default value - `true`.
## JSON {#json}
Outputs data in JSON format. Besides data tables, it also outputs column names and types, along with some additional information: the total number of output rows, and the number of rows that could have been output if there werent a LIMIT. Example:

View File

@ -4637,3 +4637,35 @@ Possible values:
- 1 — Enabled.
Default value: 1.
## SQLInsert format settings {$sqlinsert-format-settings}
### output_format_sql_insert_max_batch_size {#output_format_sql_insert_max_batch_size}
The maximum number of rows in one INSERT statement.
Default value: `65505`.
### output_format_sql_insert_table_name {#output_format_sql_insert_table_name}
The name of table that will be used in the output INSERT statement.
Default value: `'table''`.
### output_format_sql_insert_include_column_names {#output_format_sql_insert_include_column_names}
Include column names in INSERT statement.
Default value: `true`.
### output_format_sql_insert_use_replace {#output_format_sql_insert_use_replace}
Use REPLACE keyword instead of INSERT.
Default value: `false`.
### output_format_sql_insert_quote_names {#output_format_sql_insert_quote_names}
Quote column names with "`" characters
Default value: `true`.

View File

@ -763,6 +763,12 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
\
M(String, input_format_mysql_dump_table_name, "", "Name of the table in MySQL dump from which to read data", 0) \
M(Bool, input_format_mysql_dump_map_column_names, true, "Match columns from table in MySQL dump and columns from ClickHouse table by names", 0) \
\
M(UInt64, output_format_sql_insert_max_batch_size, DEFAULT_BLOCK_SIZE, "The maximum number of rows in one INSERT statement.", 0) \
M(String, output_format_sql_insert_table_name, "table", "The name of table in the output INSERT query", 0) \
M(Bool, output_format_sql_insert_include_column_names, true, "Include column names in INSERT query", 0) \
M(Bool, output_format_sql_insert_use_replace, false, "Use REPLACE statement instead of INSERT", 0) \
M(Bool, output_format_sql_insert_quote_names, true, "Quote column names with '`' characters", 0) \
// End of FORMAT_FACTORY_SETTINGS
// Please add settings non-related to formats into the COMMON_SETTINGS above.

View File

@ -158,6 +158,11 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.column_names_for_schema_inference = settings.column_names_for_schema_inference;
format_settings.mysql_dump.table_name = settings.input_format_mysql_dump_table_name;
format_settings.mysql_dump.map_column_names = settings.input_format_mysql_dump_map_column_names;
format_settings.sql_insert.max_batch_size = settings.output_format_sql_insert_max_batch_size;
format_settings.sql_insert.include_column_names = settings.output_format_sql_insert_include_column_names;
format_settings.sql_insert.table_name = settings.output_format_sql_insert_table_name;
format_settings.sql_insert.use_replace = settings.output_format_sql_insert_use_replace;
format_settings.sql_insert.quote_names = settings.output_format_sql_insert_quote_names;
/// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context
if (format_settings.schema.is_server)

View File

@ -1,6 +1,7 @@
#pragma once
#include <Core/Names.h>
#include <Core/Defines.h>
#include <base/types.h>
@ -274,6 +275,15 @@ struct FormatSettings
String table_name;
bool map_column_names = true;
} mysql_dump;
struct
{
UInt64 max_batch_size = DEFAULT_BLOCK_SIZE;
String table_name = "table";
bool include_column_names = true;
bool use_replace = false;
bool quote_names = true;
} sql_insert;
};
}

View File

@ -82,6 +82,7 @@ void registerOutputFormatMySQLWire(FormatFactory & factory);
void registerOutputFormatMarkdown(FormatFactory & factory);
void registerOutputFormatPostgreSQLWire(FormatFactory & factory);
void registerOutputFormatPrometheus(FormatFactory & factory);
void registerOutputFormatSQLInsert(FormatFactory & factory);
/// Input only formats.
@ -205,6 +206,7 @@ void registerFormats()
registerOutputFormatPostgreSQLWire(factory);
registerOutputFormatCapnProto(factory);
registerOutputFormatPrometheus(factory);
registerOutputFormatSQLInsert(factory);
registerInputFormatRegexp(factory);
registerInputFormatJSONAsString(factory);

View File

@ -0,0 +1,102 @@
#include <Processors/Formats/Impl/SQLInsertRowOutputFormat.h>
#include <IO/WriteHelpers.h>
namespace DB
{
SQLInsertRowOutputFormat::SQLInsertRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_)
: IRowOutputFormat(header_, out_, params_), column_names(header_.getNames()), format_settings(format_settings_)
{
}
void SQLInsertRowOutputFormat::writeRowStartDelimiter()
{
if (rows_in_line == 0)
printLineStart();
writeChar('(', out);
}
void SQLInsertRowOutputFormat::printLineStart()
{
if (format_settings.sql_insert.use_replace)
writeCString("REPLACE INTO ", out);
else
writeCString("INSERT INTO ", out);
writeString(format_settings.sql_insert.table_name, out);
if (format_settings.sql_insert.include_column_names)
printColumnNames();
writeCString(" VALUES ", out);
}
void SQLInsertRowOutputFormat::printColumnNames()
{
writeCString(" (", out);
for (size_t i = 0; i != column_names.size(); ++i)
{
if (format_settings.sql_insert.quote_names)
writeChar('`', out);
writeString(column_names[i], out);
if (format_settings.sql_insert.quote_names)
writeChar('`', out);
if (i + 1 != column_names.size())
writeCString(", ", out);
}
writeChar(')', out);
}
void SQLInsertRowOutputFormat::writeField(const IColumn & column, const ISerialization & serialization, size_t row_num)
{
serialization.serializeTextQuoted(column, row_num, out, format_settings);
}
void SQLInsertRowOutputFormat::writeFieldDelimiter()
{
writeCString(", ", out);
}
void SQLInsertRowOutputFormat::writeRowEndDelimiter()
{
writeChar(')', out);
++rows_in_line;
}
void SQLInsertRowOutputFormat::writeRowBetweenDelimiter()
{
if (rows_in_line >= format_settings.sql_insert.max_batch_size)
{
writeCString(";\n", out);
rows_in_line = 0;
}
else
{
writeCString(", ", out);
}
}
void SQLInsertRowOutputFormat::writeSuffix()
{
writeCString(";\n", out);
}
void registerOutputFormatSQLInsert(FormatFactory & factory)
{
factory.registerOutputFormat("SQLInsert", [](
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
const FormatSettings & settings)
{
return std::make_shared<SQLInsertRowOutputFormat>(buf, sample, params, settings);
});
}
}

View File

@ -0,0 +1,43 @@
#pragma once
#include <Core/Block.h>
#include <Formats/FormatSettings.h>
#include <Processors/Formats/IRowOutputFormat.h>
namespace DB
{
class WriteBuffer;
class SQLInsertRowOutputFormat : public IRowOutputFormat
{
public:
SQLInsertRowOutputFormat(
WriteBuffer & out_,
const Block & header_,
const RowOutputFormatParams & params_,
const FormatSettings & format_settings_);
String getName() const override { return "SQLInsertRowOutputFormat"; }
/// https://www.iana.org/assignments/media-types/text/tab-separated-values
String getContentType() const override { return "text/tab-separated-values; charset=UTF-8"; }
protected:
void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override;
virtual void writeFieldDelimiter() override;
virtual void writeRowStartDelimiter() override;
virtual void writeRowEndDelimiter() override;
virtual void writeRowBetweenDelimiter() override;
virtual void writeSuffix() override;
void printLineStart();
void printColumnNames();
size_t rows_in_line = 0;
Names column_names;
const FormatSettings format_settings;
};
}

View File

@ -0,0 +1,23 @@
INSERT INTO table (`x`, `y`, `z`) VALUES (0, 0, 'Hello'), (1, 1, 'Hello'), (2, 2, 'Hello'), (3, 0, 'Hello'), (4, 1, 'Hello');
INSERT INTO table (`x`, `y`, `z`) VALUES (0, 0, 'Hello');
INSERT INTO table (`x`, `y`, `z`) VALUES (1, 1, 'Hello');
INSERT INTO table (`x`, `y`, `z`) VALUES (2, 2, 'Hello');
INSERT INTO table (`x`, `y`, `z`) VALUES (3, 0, 'Hello');
INSERT INTO table (`x`, `y`, `z`) VALUES (4, 1, 'Hello');
INSERT INTO table (`x`, `y`, `z`) VALUES (0, 0, 'Hello'), (1, 1, 'Hello');
INSERT INTO table (`x`, `y`, `z`) VALUES (2, 2, 'Hello'), (3, 0, 'Hello');
INSERT INTO table (`x`, `y`, `z`) VALUES (4, 1, 'Hello');
INSERT INTO table VALUES (0, 0, 'Hello'), (1, 1, 'Hello'), (2, 2, 'Hello'), (3, 0, 'Hello'), (4, 1, 'Hello');
REPLACE INTO table (`x`, `y`, `z`) VALUES (0, 0, 'Hello'), (1, 1, 'Hello'), (2, 2, 'Hello'), (3, 0, 'Hello'), (4, 1, 'Hello');
INSERT INTO test (`x`, `y`, `z`) VALUES (0, 0, 'Hello'), (1, 1, 'Hello'), (2, 2, 'Hello'), (3, 0, 'Hello'), (4, 1, 'Hello');
INSERT INTO test (x, y, z) VALUES (0, 0, 'Hello'), (1, 1, 'Hello'), (2, 2, 'Hello'), (3, 0, 'Hello'), (4, 1, 'Hello');
0 0 Hello
1 1 Hello
2 2 Hello
3 0 Hello
4 1 Hello
0 0 Hello
1 1 Hello
2 2 Hello
3 0 Hello
4 1 Hello

View File

@ -0,0 +1,13 @@
-- Tags: no-parallel
select number as x, number % 3 as y, 'Hello' as z from numbers(5) format SQLInsert;
select number as x, number % 3 as y, 'Hello' as z from numbers(5) format SQLInsert settings output_format_sql_insert_max_batch_size=1;
select number as x, number % 3 as y, 'Hello' as z from numbers(5) format SQLInsert settings output_format_sql_insert_max_batch_size=2;
select number as x, number % 3 as y, 'Hello' as z from numbers(5) format SQLInsert settings output_format_sql_insert_include_column_names=0;
select number as x, number % 3 as y, 'Hello' as z from numbers(5) format SQLInsert settings output_format_sql_insert_use_replace=1;
select number as x, number % 3 as y, 'Hello' as z from numbers(5) format SQLInsert settings output_format_sql_insert_table_name='test';
select number as x, number % 3 as y, 'Hello' as z from numbers(5) format SQLInsert settings output_format_sql_insert_table_name='test', output_format_sql_insert_quote_names=0;
insert into function file(02322_data.sql, 'SQLInsert') select number as x, number % 3 as y, 'Hello' as z from numbers(5) settings output_format_sql_insert_max_batch_size=2, output_format_sql_insert_quote_names=0, engine_file_truncate_on_insert=1;
select * from file(02322_data.sql, 'MySQLDump');
insert into function file(02322_data.sql, 'SQLInsert') select number, number % 3, 'Hello' from numbers(5) settings output_format_sql_insert_max_batch_size=2, engine_file_truncate_on_insert=1;
select * from file(02322_data.sql, 'MySQLDump');

View File

@ -121,6 +121,7 @@ SATA
SERIALIZABLE
SIMD
SMALLINT
SQLInsert
SQLSTATE
SSSE
Schemas
@ -411,6 +412,7 @@ simdjson
skippingerrors
sparsehash
sql
sqlinsert
src
stacktraces
statbox