mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge pull request #60015 from azat/values-quote-escape
Fix INSERT into SQLite with single quote (by escaping single quotes with a quote instead of backslash)
This commit is contained in:
commit
df48106cd5
@ -1146,6 +1146,8 @@ class IColumn;
|
||||
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) \
|
||||
\
|
||||
M(Bool, output_format_values_escape_quote_with_quote, false, "If true escape ' with '', otherwise quoted with \\'", 0) \
|
||||
\
|
||||
M(Bool, output_format_bson_string_as_string, false, "Use BSON String type instead of Binary for String columns.", 0) \
|
||||
M(Bool, input_format_bson_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format BSON.", 0) \
|
||||
\
|
||||
|
@ -334,9 +334,12 @@ bool SerializationString::tryDeserializeTextEscaped(IColumn & column, ReadBuffer
|
||||
return read<bool>(column, [&](ColumnString::Chars & data) { readEscapedStringInto(data, istr); return true; });
|
||||
}
|
||||
|
||||
void SerializationString::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
void SerializationString::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeQuotedString(assert_cast<const ColumnString &>(column).getDataAt(row_num), ostr);
|
||||
if (settings.values.escape_quote_with_quote)
|
||||
writeQuotedStringPostgreSQL(assert_cast<const ColumnString &>(column).getDataAt(row_num).toView(), ostr);
|
||||
else
|
||||
writeQuotedString(assert_cast<const ColumnString &>(column).getDataAt(row_num), ostr);
|
||||
}
|
||||
|
||||
|
||||
|
@ -181,6 +181,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se
|
||||
format_settings.values.allow_data_after_semicolon = settings.input_format_values_allow_data_after_semicolon;
|
||||
format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions;
|
||||
format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions;
|
||||
format_settings.values.escape_quote_with_quote = settings.output_format_values_escape_quote_with_quote;
|
||||
format_settings.with_names_use_header = settings.input_format_with_names_use_header;
|
||||
format_settings.with_types_use_header = settings.input_format_with_types_use_header;
|
||||
format_settings.write_statistics = settings.output_format_write_statistics;
|
||||
|
@ -361,6 +361,7 @@ struct FormatSettings
|
||||
bool deduce_templates_of_expressions = true;
|
||||
bool accurate_types_of_literals = true;
|
||||
bool allow_data_after_semicolon = false;
|
||||
bool escape_quote_with_quote = false;
|
||||
} values;
|
||||
|
||||
enum class ORCCompression
|
||||
|
@ -19,6 +19,20 @@
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
using namespace DB;
|
||||
|
||||
ContextPtr makeSQLiteWriteContext(ContextPtr context)
|
||||
{
|
||||
auto write_context = Context::createCopy(context);
|
||||
write_context->setSetting("output_format_values_escape_quote_with_quote", Field(true));
|
||||
return write_context;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -43,6 +57,7 @@ StorageSQLite::StorageSQLite(
|
||||
, database_path(database_path_)
|
||||
, sqlite_db(sqlite_db_)
|
||||
, log(getLogger("StorageSQLite (" + table_id_.table_name + ")"))
|
||||
, write_context(makeSQLiteWriteContext(getContext()))
|
||||
{
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
|
||||
@ -144,7 +159,7 @@ public:
|
||||
|
||||
sqlbuf << ") VALUES ";
|
||||
|
||||
auto writer = FormatFactory::instance().getOutputFormat("Values", sqlbuf, metadata_snapshot->getSampleBlock(), storage.getContext());
|
||||
auto writer = FormatFactory::instance().getOutputFormat("Values", sqlbuf, metadata_snapshot->getSampleBlock(), storage.write_context);
|
||||
writer->write(block);
|
||||
|
||||
sqlbuf << ";";
|
||||
|
@ -47,10 +47,13 @@ public:
|
||||
const String & table);
|
||||
|
||||
private:
|
||||
friend class SQLiteSink; /// for write_context
|
||||
|
||||
String remote_table_name;
|
||||
String database_path;
|
||||
SQLitePtr sqlite_db;
|
||||
LoggerPtr log;
|
||||
ContextPtr write_context;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -29,7 +29,7 @@ CREATE TABLE default.sqlite_table3\n(\n `col1` String,\n `col2` Int32\n)\n
|
||||
not a null 2
|
||||
3
|
||||
4
|
||||
line6 6
|
||||
line\'6 6
|
||||
7
|
||||
test table function
|
||||
line1 1
|
||||
|
@ -76,7 +76,7 @@ ${CLICKHOUSE_CLIENT} --query='DROP TABLE IF EXISTS sqlite_table3'
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE sqlite_table3 (col1 String, col2 Int32) ENGINE = SQLite('${DB_PATH}', 'table3')"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query='SHOW CREATE TABLE sqlite_table3;' | sed -r 's/(.*SQLite)(.*)/\1/'
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO sqlite_table3 VALUES ('line6', 6);"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO sqlite_table3 VALUES ('line\'6', 6);"
|
||||
${CLICKHOUSE_CLIENT} --query="INSERT INTO sqlite_table3 VALUES (NULL, 7);"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query='SELECT * FROM sqlite_table3 ORDER BY col2'
|
||||
|
@ -0,0 +1,3 @@
|
||||
('foo')('foo\'bar')('foo\'\'bar')
|
||||
output_format_values_escape_quote_with_quote=1
|
||||
('foo')('foo''bar')('foo''''bar')
|
12
tests/queries/0_stateless/02993_values_escape_quote.sql
Normal file
12
tests/queries/0_stateless/02993_values_escape_quote.sql
Normal file
@ -0,0 +1,12 @@
|
||||
select 'foo' format Values;
|
||||
select 'foo\'bar' format Values;
|
||||
select 'foo\'\'bar' format Values;
|
||||
|
||||
select '\noutput_format_values_escape_quote_with_quote=1' format LineAsString;
|
||||
set output_format_values_escape_quote_with_quote=1;
|
||||
|
||||
select 'foo' format Values;
|
||||
select 'foo\'bar' format Values;
|
||||
select 'foo\'\'bar' format Values;
|
||||
-- fix no newline at end of file
|
||||
select '' format LineAsString;
|
Loading…
Reference in New Issue
Block a user