Remove IdentifierQuotingStyle::None

This commit is contained in:
Tuan Pham Anh 2024-09-25 03:28:38 +00:00
parent 0a71befde4
commit 6a83b5cca7
10 changed files with 19 additions and 93 deletions

View File

@ -164,7 +164,7 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
std::string table_name = params.get("table_name");
LOG_TRACE(log, "DB name: '{}', table name: '{}'", db_name, table_name);
auto quoting_style = IdentifierQuotingStyle::None;
auto quoting_style = IdentifierQuotingStyle::Backticks;
#if USE_ODBC
quoting_style = getQuotingStyle(connection_handler);
#endif

View File

@ -38,7 +38,7 @@ IdentifierQuotingStyle getQuotingStyle(nanodbc::ConnectionHolderPtr connection)
{
auto identifier_quote = getIdentifierQuote(connection);
if (identifier_quote.empty())
return IdentifierQuotingStyle::None;
return IdentifierQuotingStyle::Backticks;
else if (identifier_quote[0] == '`')
return IdentifierQuotingStyle::Backticks;
else if (identifier_quote[0] == '"')

View File

@ -243,7 +243,7 @@ protected:
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Failed to parse quoting style from '{}' for service {}",
character, BridgeHelperMixin::serviceAlias());
else if (character.empty())
quote_style = IdentifierQuotingStyle::None;
quote_style = IdentifierQuotingStyle::Backticks;
else if (character[0] == '`')
quote_style = IdentifierQuotingStyle::Backticks;
else if (character[0] == '"')

View File

@ -245,9 +245,10 @@ IMPLEMENT_SETTING_ENUM(
ErrorCodes::BAD_ARGUMENTS,
{{"throw", GroupArrayActionWhenLimitReached::THROW}, {"discard", GroupArrayActionWhenLimitReached::DISCARD}})
IMPLEMENT_SETTING_ENUM(IdentifierQuotingStyle, ErrorCodes::BAD_ARGUMENTS,
{{"None", IdentifierQuotingStyle::None},
{"Backticks", IdentifierQuotingStyle::Backticks},
IMPLEMENT_SETTING_ENUM(
IdentifierQuotingStyle,
ErrorCodes::BAD_ARGUMENTS,
{{"Backticks", IdentifierQuotingStyle::Backticks},
{"DoubleQuotes", IdentifierQuotingStyle::DoubleQuotes},
{"BackticksMySQL", IdentifierQuotingStyle::BackticksMySQL}})

View File

@ -53,10 +53,6 @@ void ExternalQueryBuilder::writeQuoted(const std::string & s, WriteBuffer & out)
{
switch (quoting_style)
{
case IdentifierQuotingStyle::None:
writeString(s, out);
break;
case IdentifierQuotingStyle::Backticks:
writeBackQuotedString(s, out);
break;

View File

@ -5,6 +5,7 @@
#include <IO/WriteHelpers.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/IdentifierQuotingStyle.h>
#include <Poco/String.h>
#include <Common/SensitiveDataMasker.h>
#include <Common/SipHash.h>
@ -220,23 +221,18 @@ String IAST::getColumnNameWithoutAlias() const
void IAST::FormatSettings::writeIdentifier(const String & name, bool ambiguous) const
{
if (!ambiguous && identifier_quoting_rule == IdentifierQuotingRule::UserDisplay)
{
// Treat `name` as ambiguous if it is one of the keywords
const auto & keywords = getAllKeyWords();
ambiguous = std::find(keywords.begin(), keywords.end(), Poco::toUpper(name)) != keywords.end();
}
bool must_quote
= (identifier_quoting_rule == IdentifierQuotingRule::Always
|| (ambiguous && identifier_quoting_rule == IdentifierQuotingRule::WhenNecessary));
switch (identifier_quoting_style)
{
case IdentifierQuotingStyle::None:
{
if (must_quote)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Incompatible arguments: identifier_quoting_rule = {} && identifier_quoting_style == {} && ambiguous = {}",
identifier_quoting_rule,
identifier_quoting_style,
ambiguous);
writeString(name, ostr);
break;
}
case IdentifierQuotingStyle::Backticks:
{
if (must_quote)

View File

@ -8,10 +8,9 @@ namespace DB
/// NOTE There could be differences in escaping rules inside quotes. Escaping rules may not match that required by specific external DBMS.
enum class IdentifierQuotingStyle : uint8_t
{
None, /// Write as-is, without quotes.
Backticks, /// `clickhouse` style
DoubleQuotes, /// "postgres" style
BackticksMySQL, /// `mysql` style, most same as Backticks, but it uses '``' to escape '`'
Backticks, /// `clickhouse` style
DoubleQuotes, /// "postgres" style
BackticksMySQL, /// `mysql` style, most same as Backticks, but it uses '``' to escape '`'
};
enum class IdentifierQuotingRule : uint8_t

View File

@ -386,11 +386,7 @@ String transformQueryForExternalDatabaseImpl(
ASTPtr select_ptr = select;
dropAliases(select_ptr);
IdentifierQuotingRule identifier_quoting_rule = IdentifierQuotingRule::WhenNecessary;
if (identifier_quoting_style != IdentifierQuotingStyle::None)
{
identifier_quoting_rule = IdentifierQuotingRule::Always;
}
IdentifierQuotingRule identifier_quoting_rule = IdentifierQuotingRule::Always;
WriteBufferFromOwnString out;
IAST::FormatSettings settings(
/*ostr_=*/out,

View File

@ -4,13 +4,6 @@ CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_d
CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month
CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
Settings: when_necessary & None
Settings: user_display & None
CREATE TABLE default.uk_price_paid\n(\n Table String,\n Engine String,\n price UInt32,\n date Date,\n postcode1 LowCardinality(String),\n postcode2 LowCardinality(String),\n type Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n is_new UInt8,\n duration Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n addr1 String,\n addr2 String,\n street LowCardinality(String),\n locality LowCardinality(String),\n town LowCardinality(String),\n district LowCardinality(String),\n county LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n price UInt32,\n date Date,\n addr1 String,\n addr2 String,\n street LowCardinality(String),\n town LowCardinality(String),\n district LowCardinality(String),\n county LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid
CREATE TABLE default.uk_prices_aggs_dest\n(\n month Date,\n min_price SimpleAggregateFunction(min, UInt32),\n max_price SimpleAggregateFunction(max, UInt32),\n volume AggregateFunction(count, UInt32),\n avg_price AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n month Date,\n min_price SimpleAggregateFunction(min, UInt32),\n max_price SimpleAggregateFunction(max, UInt32),\n volume AggregateFunction(count, UInt32),\n avg_price AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month
CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n date DateTime64,\n variable Decimal32(2),\n fixed Decimal32(2),\n bank Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED())
Settings: always & Backticks
CREATE TABLE `default`.`uk_price_paid`\n(\n `Table` String,\n `Engine` String,\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX `county_index` `county` TYPE set(10) GRANULARITY 1,\n PROJECTION `town_date_projection`\n (\n SELECT \n `town`,\n `date`,\n `price`\n ORDER BY \n `town`,\n `date`\n ),\n PROJECTION `handy_aggs_projection`\n (\n SELECT \n avg(`price`),\n max(`price`),\n sum(`price`)\n GROUP BY `town`\n )\n)\nENGINE = MergeTree\nORDER BY (`postcode1`, `postcode2`, `date`)\nSETTINGS index_granularity = 8192
CREATE MATERIALIZED VIEW `default`.`prices_by_year_view` TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n `price`,\n `date`,\n `addr1`,\n `addr2`,\n `street`,\n `town`,\n `district`,\n `county`\nFROM `default`.`uk_price_paid`

View File

@ -136,61 +136,6 @@ SHOW CREATE uk_prices_aggs_dest;
SHOW CREATE VIEW uk_prices_aggs_view;
SHOW CREATE DICTIONARY uk_mortgage_rates_dict;
-- Show tables, views, dictionaries with output_format_identifier_quoting_rule='when_necessary', output_format_identifier_quoting_style='None'
SELECT('Settings: when_necessary & None');
SHOW CREATE TABLE uk_price_paid
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='None'; -- { serverError BAD_ARGUMENTS}
SHOW CREATE VIEW prices_by_year_view
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='None'; -- { serverError BAD_ARGUMENTS}
SHOW CREATE uk_prices_aggs_dest
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='None'; -- { serverError BAD_ARGUMENTS}
SHOW CREATE VIEW uk_prices_aggs_view
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='None'; -- { serverError BAD_ARGUMENTS}
SHOW CREATE DICTIONARY uk_mortgage_rates_dict
SETTINGS
output_format_identifier_quoting_rule='when_necessary',
output_format_identifier_quoting_style='None'; -- { serverError BAD_ARGUMENTS}
-- Show tables, views, dictionaries with output_format_identifier_quoting_rule='user_display', output_format_identifier_quoting_style='None'
SELECT('Settings: user_display & None');
SHOW CREATE TABLE uk_price_paid
SETTINGS
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='None';
SHOW CREATE VIEW prices_by_year_view
SETTINGS
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='None';
SHOW CREATE uk_prices_aggs_dest
SETTINGS
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='None';
SHOW CREATE VIEW uk_prices_aggs_view
SETTINGS
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='None';
SHOW CREATE DICTIONARY uk_mortgage_rates_dict
SETTINGS
output_format_identifier_quoting_rule='user_display',
output_format_identifier_quoting_style='None';
-- Show tables, views, dictionaries with output_format_identifier_quoting_rule='always', output_format_identifier_quoting_style='Backticks'
SELECT('Settings: always & Backticks');
SHOW CREATE TABLE uk_price_paid