Quote all identifiers when querying external database #2635

This commit is contained in:
Alexey Milovidov 2018-07-16 04:02:46 +03:00 committed by alexey-milovidov
parent 9af2a9179e
commit 245b114363
17 changed files with 107 additions and 51 deletions

View File

@ -50,7 +50,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
table{config.getString(config_prefix + ".table")},
where{config.getString(config_prefix + ".where", "")},
update_field{config.getString(config_prefix + ".update_field", "")},
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::Backticks},
query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks},
sample_block{sample_block}, context(context),
is_local{isLocalAddress({ host, port }, config.getInt("tcp_port", 0))},
pool{is_local ? nullptr : createPool(host, port, secure, db, user, password, context)},
@ -67,7 +67,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionar
db{other.db}, table{other.table},
where{other.where},
update_field{other.update_field},
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::Backticks},
query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks},
sample_block{other.sample_block}, context(other.context),
is_local{other.is_local},
pool{is_local ? nullptr : createPool(host, port, secure, db, user, password, context)},

View File

@ -22,7 +22,7 @@ ExternalQueryBuilder::ExternalQueryBuilder(
const std::string & db,
const std::string & table,
const std::string & where,
QuotingStyle quoting_style)
IdentifierQuotingStyle quoting_style)
: dict_struct(dict_struct), db(db), table(table), where(where), quoting_style(quoting_style)
{
}
@ -32,15 +32,15 @@ void ExternalQueryBuilder::writeQuoted(const std::string & s, WriteBuffer & out)
{
switch (quoting_style)
{
case None:
case IdentifierQuotingStyle::None:
writeString(s, out);
break;
case Backticks:
case IdentifierQuotingStyle::Backticks:
writeBackQuotedString(s, out);
break;
case DoubleQuotes:
case IdentifierQuotingStyle::DoubleQuotes:
writeDoubleQuotedString(s, out);
break;
}
@ -138,7 +138,7 @@ std::string ExternalQueryBuilder::composeLoadAllQuery() const
}
std::string ExternalQueryBuilder::composeUpdateQuery(const std::string &update_field, const std::string &time_point) const
std::string ExternalQueryBuilder::composeUpdateQuery(const std::string & update_field, const std::string & time_point) const
{
std::string out = composeLoadAllQuery();
std::string update_query;

View File

@ -3,6 +3,7 @@
#include <string>
#include <Formats/FormatSettings.h>
#include <Columns/IColumn.h>
#include <Parsers/IdentifierQuotingStyle.h>
namespace DB
@ -21,16 +22,7 @@ struct ExternalQueryBuilder
const std::string & table;
const std::string & where;
/// Method to quote identifiers.
/// NOTE There could be differences in escaping rules inside quotes. Escaping rules may not match that required by specific external DBMS.
enum QuotingStyle
{
None, /// Write as-is, without quotes.
Backticks, /// `mysql` style
DoubleQuotes /// "postgres" style
};
QuotingStyle quoting_style;
IdentifierQuotingStyle quoting_style;
ExternalQueryBuilder(
@ -38,7 +30,7 @@ struct ExternalQueryBuilder
const std::string & db,
const std::string & table,
const std::string & where,
QuotingStyle quoting_style);
IdentifierQuotingStyle quoting_style);
/** Generate a query to load all data. */
std::string composeLoadAllQuery() const;

View File

@ -35,7 +35,7 @@ MySQLDictionarySource::MySQLDictionarySource(const DictionaryStructure & dict_st
dont_check_update_time{config.getBool(config_prefix + ".dont_check_update_time", false)},
sample_block{sample_block},
pool{config, config_prefix},
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::Backticks},
query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks},
load_all_query{query_builder.composeLoadAllQuery()},
invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}
{
@ -53,7 +53,7 @@ MySQLDictionarySource::MySQLDictionarySource(const MySQLDictionarySource & other
dont_check_update_time{other.dont_check_update_time},
sample_block{other.sample_block},
pool{other.pool},
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::Backticks},
query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks},
load_all_query{other.load_all_query}, last_modification{other.last_modification},
invalidate_query{other.invalidate_query}, invalidate_query_response{other.invalidate_query_response}
{

View File

@ -29,7 +29,7 @@ ODBCDictionarySource::ODBCDictionarySource(const DictionaryStructure & dict_stru
where{config.getString(config_prefix + ".where", "")},
update_field{config.getString(config_prefix + ".update_field", "")},
sample_block{sample_block},
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::None}, /// NOTE Better to obtain quoting style via ODBC interface.
query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::None}, /// NOTE Better to obtain quoting style via ODBC interface.
load_all_query{query_builder.composeLoadAllQuery()},
invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}
{
@ -58,7 +58,7 @@ ODBCDictionarySource::ODBCDictionarySource(const ODBCDictionarySource & other)
update_field{other.update_field},
sample_block{other.sample_block},
pool{other.pool},
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::None},
query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::None},
load_all_query{other.load_all_query},
invalidate_query{other.invalidate_query}, invalidate_query_response{other.invalidate_query_response}
{

View File

@ -394,11 +394,13 @@ inline void writeBackQuotedString(const String & s, WriteBuffer & buf)
writeAnyQuotedString<'`'>(s, buf);
}
/// The same, but backquotes apply only if there are characters that do not match the identifier without backquotes.
inline void writeProbablyBackQuotedString(const String & s, WriteBuffer & buf)
/// The same, but quotes apply only if there are characters that do not match the identifier without quotes.
template <typename F>
inline void writeProbablyQuotedStringImpl(const String & s, WriteBuffer & buf, F && write_quoted_string)
{
if (s.empty() || !isValidIdentifierBegin(s[0]))
writeBackQuotedString(s, buf);
write_quoted_string(s, buf);
else
{
const char * pos = s.data() + 1;
@ -407,12 +409,22 @@ inline void writeProbablyBackQuotedString(const String & s, WriteBuffer & buf)
if (!isWordCharASCII(*pos))
break;
if (pos != end)
writeBackQuotedString(s, buf);
write_quoted_string(s, buf);
else
writeString(s, buf);
}
}
inline void writeProbablyBackQuotedString(const String & s, WriteBuffer & buf)
{
writeProbablyQuotedStringImpl(s, buf, [](const String & s, WriteBuffer & buf) { return writeBackQuotedString(s, buf); });
}
inline void writeProbablyDoubleQuotedString(const String & s, WriteBuffer & buf)
{
writeProbablyQuotedStringImpl(s, buf, [](const String & s, WriteBuffer & buf) { return writeDoubleQuotedString(s, buf); });
}
/** Outputs the string in for the CSV format.
* Rules:

View File

@ -13,7 +13,7 @@ void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, Form
settings.ostr << (settings.hilite ? hilite_identifier : "");
WriteBufferFromOStream wb(settings.ostr, 32);
writeProbablyBackQuotedString(name, wb);
settings.writeIdentifier(name, wb);
wb.next();
settings.ostr << (settings.hilite ? hilite_none : "");

View File

@ -6,6 +6,18 @@
namespace DB
{
void ASTWithAlias::writeAlias(const String & name, const FormatSettings & settings) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_alias : "");
WriteBufferFromOStream wb(settings.ostr, 32);
settings.writeIdentifier(name, wb);
wb.next();
settings.ostr << (settings.hilite ? hilite_none : "");
}
void ASTWithAlias::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
if (!alias.empty())
@ -14,7 +26,7 @@ void ASTWithAlias::formatImpl(const FormatSettings & settings, FormatState & sta
if (!state.printed_asts_with_alias.emplace(frame.current_select, alias).second)
{
WriteBufferFromOStream wb(settings.ostr, 32);
writeProbablyBackQuotedString(alias, wb);
settings.writeIdentifier(alias, wb);
return;
}
}
@ -27,7 +39,7 @@ void ASTWithAlias::formatImpl(const FormatSettings & settings, FormatState & sta
if (!alias.empty())
{
writeAlias(alias, settings.ostr, settings.hilite);
writeAlias(alias, settings);
if (frame.need_parens)
settings.ostr <<')';
}

View File

@ -32,6 +32,8 @@ public:
protected:
virtual void appendColumnNameImpl(WriteBuffer & ostr) const = 0;
void writeAlias(const String & name, const FormatSettings & settings) const;
};
/// helper for setting aliases and chaining result to other functions

View File

@ -13,6 +13,7 @@ namespace ErrorCodes
{
extern const int TOO_BIG_AST;
extern const int TOO_DEEP_AST;
extern const int BAD_ARGUMENTS;
}
@ -36,18 +37,6 @@ String backQuoteIfNeed(const String & x)
}
void IAST::writeAlias(const String & name, std::ostream & s, bool hilite) const
{
s << (hilite ? hilite_keyword : "") << " AS " << (hilite ? hilite_alias : "");
WriteBufferFromOStream wb(s, 32);
writeProbablyBackQuotedString(name, wb);
wb.next();
s << (hilite ? hilite_none : "");
}
size_t IAST::checkSize(size_t max_size) const
{
size_t res = 1;
@ -109,4 +98,37 @@ String IAST::getColumnName() const
return write_buffer.str();
}
void IAST::FormatSettings::writeIdentifier(const String & name, WriteBuffer & out) const
{
switch (identifier_quoting_style)
{
case IdentifierQuotingStyle::None:
{
if (always_quote_identifiers)
throw Exception("Incompatible arguments: always_quote_identifiers = true && identifier_quoting_style == IdentifierQuotingStyle::None",
ErrorCodes::BAD_ARGUMENTS);
writeString(name, out);
break;
}
case IdentifierQuotingStyle::Backticks:
{
if (always_quote_identifiers)
writeBackQuotedString(name, out);
else
writeProbablyBackQuotedString(name, out);
break;
}
case IdentifierQuotingStyle::DoubleQuotes:
{
if (always_quote_identifiers)
writeDoubleQuotedString(name, out);
else
writeProbablyDoubleQuotedString(name, out);
break;
}
case
}
}
}

View File

@ -7,6 +7,7 @@
#include <Core/Types.h>
#include <Common/Exception.h>
#include <Parsers/StringRange.h>
#include <Parsers/IdentifierQuotingStyle.h>
class SipHash;
@ -150,16 +151,20 @@ public:
struct FormatSettings
{
std::ostream & ostr;
bool hilite;
bool hilite = false;
bool one_line;
bool always_quote_identifiers = false;
IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks;
char nl_or_ws;
FormatSettings(std::ostream & ostr_, bool hilite_, bool one_line_)
: ostr(ostr_), hilite(hilite_), one_line(one_line_)
FormatSettings(std::ostream & ostr_, bool one_line_)
: ostr(ostr_), one_line(one_line_)
{
nl_or_ws = one_line ? ' ' : '\n';
}
void writeIdentifier(const String & name, WriteBuffer & out) const;
};
/// State. For example, a set of nodes can be remembered, which we already walk through.
@ -194,8 +199,6 @@ public:
ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
}
void writeAlias(const String & name, std::ostream & s, bool hilite) const;
void cloneChildren();
public:

View File

@ -7,7 +7,9 @@ namespace DB
void formatAST(const IAST & ast, std::ostream & s, bool hilite, bool one_line)
{
IAST::FormatSettings settings(s, hilite, one_line);
IAST::FormatSettings settings(s, one_line);
settings.hilite = hilite;
ast.format(settings);
}

View File

@ -12,6 +12,7 @@ namespace DB
*/
void formatAST(const IAST & ast, std::ostream & s, bool hilite = true, bool one_line = false);
inline std::ostream & operator<<(std::ostream & os, const IAST & ast)
{
formatAST(ast, os, false, true);

View File

@ -57,7 +57,8 @@ BlockInputStreams StorageMySQL::read(
{
check(column_names);
processed_stage = QueryProcessingStage::FetchColumns;
String query = transformQueryForExternalDatabase(*query_info.query, getColumns().ordinary, remote_database_name, remote_table_name, context);
String query = transformQueryForExternalDatabase(
*query_info.query, getColumns().ordinary, IdentifierQuotingStyle::Backticks, remote_database_name, remote_table_name, context);
Block sample_block;
for (const String & name : column_names)

View File

@ -44,7 +44,7 @@ BlockInputStreams StorageODBC::read(
check(column_names);
processed_stage = QueryProcessingStage::FetchColumns;
String query = transformQueryForExternalDatabase(
*query_info.query, getColumns().ordinary, remote_database_name, remote_table_name, context);
*query_info.query, getColumns().ordinary, IdentifierQuotingStyle::DoubleQuotes, remote_database_name, remote_table_name, context);
Block sample_block;
for (const String & name : column_names)

View File

@ -1,3 +1,4 @@
#include <sstream>
#include <Common/typeid_cast.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTFunction.h>
@ -5,7 +6,6 @@
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/queryToString.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Storages/transformQueryForExternalDatabase.h>
@ -57,6 +57,7 @@ static bool isCompatible(const IAST & node)
String transformQueryForExternalDatabase(
const IAST & query,
const NamesAndTypesList & available_columns,
IdentifierQuotingStyle identifier_quoting_style,
const String & database,
const String & table,
const Context & context)
@ -105,7 +106,13 @@ String transformQueryForExternalDatabase(
}
}
return queryToString(select);
std::stringstream out;
IAST::FormatSettings settings(out, true);
settings.always_quote_identifiers = true;
settings.identifier_quoting_style = identifier_quoting_style;
select->format(settings);
return out.str();
}
}

View File

@ -2,6 +2,7 @@
#include <Core/Types.h>
#include <Core/NamesAndTypes.h>
#include <Parsers/IdentifierQuotingStyle.h>
namespace DB
@ -27,6 +28,7 @@ class Context;
String transformQueryForExternalDatabase(
const IAST & query,
const NamesAndTypesList & available_columns,
IdentifierQuotingStyle identifier_quoting_style,
const String & database,
const String & table,
const Context & context);