Merge pull request #10465 from kekekekule/ddl_settings

Add custom settings support in DDL-queries for CREATE DICTIONARY
This commit is contained in:
alexey-milovidov 2020-04-26 20:25:29 +03:00 committed by GitHub
commit 4c17542096
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 204 additions and 15 deletions

View File

@ -35,11 +35,7 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
The source is configured in the `source` section. The source is configured in the `source` section.
For source types For source types [Local file](#dicts-external_dicts_dict_sources-local_file), [Executable file](#dicts-external_dicts_dict_sources-executable), [HTTP(s)](#dicts-external_dicts_dict_sources-http), [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse)
[Local file](#dicts-external_dicts_dict_sources-local_file),
[Executable file](#dicts-external_dicts_dict_sources-executable),
[HTTP(s)](#dicts-external_dicts_dict_sources-http),
[ClickHouse](#dicts-external_dicts_dict_sources-clickhouse)
optional settings are available: optional settings are available:
``` xml ``` xml
@ -53,6 +49,12 @@ optional settings are available:
</settings> </settings>
</source> </source>
``` ```
or
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SETTINGS(format_csv_allow_single_quotes = 0)
```
Types of sources (`source_type`): Types of sources (`source_type`):

View File

@ -37,6 +37,20 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
La source est configurée dans le `source` section. La source est configurée dans le `source` section.
Pour les sources de types [Fichier Local](#dicts-external_dicts_dict_sources-local_file), [Fichier exécutable](#dicts-external_dicts_dict_sources-executable), [HTTP(s)](#dicts-external_dicts_dict_sources-http), [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse)
les paramètres optionnels sont possibles:
``` xml
@@ -53,6 +49,12 @@ optional settings are available:
</settings>
</source>
```
ou
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SETTINGS(format_csv_allow_single_quotes = 1)
```
Les Types de sources (`source_type`): Les Types de sources (`source_type`):
- [Fichier Local](#dicts-external_dicts_dict_sources-local_file) - [Fichier Local](#dicts-external_dicts_dict_sources-local_file)

View File

@ -30,11 +30,7 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
Источник настраивается в разделе `source`. Источник настраивается в разделе `source`.
Для типов источников Для типов источников [Локальный файл](#dicts-external_dicts_dict_sources-local_file), [Исполняемый файл](#dicts-external_dicts_dict_sources-executable), [HTTP(s)](#dicts-external_dicts_dict_sources-http), [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse)
[Локальный файл](#dicts-external_dicts_dict_sources-local_file),
[Исполняемый файл](#dicts-external_dicts_dict_sources-executable),
[HTTP(s)](#dicts-external_dicts_dict_sources-http),
[ClickHouse](#dicts-external_dicts_dict_sources-clickhouse)
доступны дополнительные настройки: доступны дополнительные настройки:
``` xml ``` xml
@ -48,6 +44,11 @@ SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration
</settings> </settings>
</source> </source>
``` ```
или
``` sql
SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated'))
SETTINGS(format_csv_allow_single_quotes = 0)
```
Типы источников (`source_type`): Типы источников (`source_type`):

View File

@ -375,13 +375,26 @@ void buildConfigurationFromFunctionWithKeyValueArguments(
* </mysql> * </mysql>
* </source> * </source>
*/ */
void buildSourceConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const ASTFunctionWithKeyValueArguments * source) void buildSourceConfiguration(AutoPtr<Document> doc, AutoPtr<Element> root, const ASTFunctionWithKeyValueArguments * source, const ASTDictionarySettings * settings)
{ {
AutoPtr<Element> outer_element(doc->createElement("source")); AutoPtr<Element> outer_element(doc->createElement("source"));
root->appendChild(outer_element); root->appendChild(outer_element);
AutoPtr<Element> source_element(doc->createElement(source->name)); AutoPtr<Element> source_element(doc->createElement(source->name));
outer_element->appendChild(source_element); outer_element->appendChild(source_element);
buildConfigurationFromFunctionWithKeyValueArguments(doc, source_element, source->elements->as<const ASTExpressionList>()); buildConfigurationFromFunctionWithKeyValueArguments(doc, source_element, source->elements->as<const ASTExpressionList>());
if (settings != nullptr)
{
AutoPtr<Element> settings_element(doc->createElement("settings"));
outer_element->appendChild(settings_element);
for (const auto & [name, value] : settings->changes)
{
AutoPtr<Element> setting_change_element(doc->createElement(name));
settings_element->appendChild(setting_change_element);
AutoPtr<Text> setting_value(doc->createTextNode(getFieldAsString(value)));
setting_change_element->appendChild(setting_value);
}
}
} }
/** Check all AST fields are filled, throws exception /** Check all AST fields are filled, throws exception
@ -454,7 +467,7 @@ DictionaryConfigurationPtr getDictionaryConfigurationFromAST(const ASTCreateQuer
buildPrimaryKeyConfiguration(xml_document, structure_element, complex, pk_attrs, query.dictionary_attributes_list); buildPrimaryKeyConfiguration(xml_document, structure_element, complex, pk_attrs, query.dictionary_attributes_list);
buildLayoutConfiguration(xml_document, current_dictionary, dictionary_layout); buildLayoutConfiguration(xml_document, current_dictionary, dictionary_layout);
buildSourceConfiguration(xml_document, current_dictionary, query.dictionary->source); buildSourceConfiguration(xml_document, current_dictionary, query.dictionary->source, query.dictionary->dict_settings);
buildLifetimeConfiguration(xml_document, current_dictionary, query.dictionary->lifetime); buildLifetimeConfiguration(xml_document, current_dictionary, query.dictionary->lifetime);
if (query.dictionary->range) if (query.dictionary->range)

View File

@ -107,6 +107,34 @@ void ASTDictionaryLayout::formatImpl(const FormatSettings & settings,
settings.ostr << ")"; settings.ostr << ")";
} }
ASTPtr ASTDictionarySettings::clone() const
{
auto res = std::make_shared<ASTDictionarySettings>(*this);
res->children.clear();
res->changes = changes;
return res;
}
void ASTDictionarySettings::formatImpl(const FormatSettings & settings,
FormatState &,
FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "")
<< "SETTINGS"
<< (settings.hilite ? hilite_none : "")
<< "(";
for (auto it = changes.begin(); it != changes.end(); ++it)
{
if (it != changes.begin())
settings.ostr << ", ";
settings.ostr << it->name << " = " << applyVisitor(FieldVisitorToString(), it->value);
}
settings.ostr << (settings.hilite ? hilite_none : "") << ")";
}
ASTPtr ASTDictionary::clone() const ASTPtr ASTDictionary::clone() const
{ {
@ -128,6 +156,9 @@ ASTPtr ASTDictionary::clone() const
if (range) if (range)
res->set(res->range, range->clone()); res->set(res->range, range->clone());
if (dict_settings)
res->set(res->dict_settings, dict_settings->clone());
return res; return res;
} }
@ -166,6 +197,12 @@ void ASTDictionary::formatImpl(const FormatSettings & settings, FormatState & st
settings.ostr << settings.nl_or_ws; settings.ostr << settings.nl_or_ws;
range->formatImpl(settings, state, frame); range->formatImpl(settings, state, frame);
} }
if (dict_settings)
{
settings.ostr << settings.nl_or_ws;
dict_settings->formatImpl(settings, state, frame);
}
} }
} }

View File

@ -5,6 +5,10 @@
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/ASTExpressionList.h> #include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ParserSetQuery.h>
namespace DB namespace DB
{ {
@ -60,6 +64,18 @@ public:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
}; };
class ASTDictionarySettings : public IAST
{
public:
SettingsChanges changes;
String getID(char) const override { return "Dictionary settings"; }
ASTPtr clone() const override;
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
/// AST contains all parts of external dictionary definition except attributes /// AST contains all parts of external dictionary definition except attributes
class ASTDictionary : public IAST class ASTDictionary : public IAST
@ -77,6 +93,8 @@ public:
ASTDictionaryLayout * layout; ASTDictionaryLayout * layout;
/// Range for dictionary (only for range-hashed dictionaries) /// Range for dictionary (only for range-hashed dictionaries)
ASTDictionaryRange * range; ASTDictionaryRange * range;
/// Settings for dictionary (optionally)
ASTDictionarySettings * dict_settings;
String getID(char) const override { return "Dictionary definition"; } String getID(char) const override { return "Dictionary definition"; }

View File

@ -11,6 +11,8 @@
#include <Poco/String.h> #include <Poco/String.h>
#include <Parsers/ParserSetQuery.h>
namespace DB namespace DB
{ {
@ -149,6 +151,31 @@ bool ParserDictionaryLayout::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
return true; return true;
} }
bool ParserDictionarySettings::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserToken s_comma(TokenType::Comma);
SettingsChanges changes;
while (true)
{
if (!changes.empty() && !s_comma.ignore(pos))
break;
changes.push_back(SettingChange{});
if (!ParserSetQuery::parseNameValuePair(changes.back(), pos, expected))
return false;
}
auto query = std::make_shared<ASTDictionarySettings>();
query->changes = std::move(changes);
node = query;
return true;
}
bool ParserDictionary::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool ParserDictionary::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{ {
@ -157,6 +184,7 @@ bool ParserDictionary::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword lifetime_keyword("LIFETIME"); ParserKeyword lifetime_keyword("LIFETIME");
ParserKeyword range_keyword("RANGE"); ParserKeyword range_keyword("RANGE");
ParserKeyword layout_keyword("LAYOUT"); ParserKeyword layout_keyword("LAYOUT");
ParserKeyword settings_keyword("SETTINGS");
ParserToken open(TokenType::OpeningRoundBracket); ParserToken open(TokenType::OpeningRoundBracket);
ParserToken close(TokenType::ClosingRoundBracket); ParserToken close(TokenType::ClosingRoundBracket);
ParserFunctionWithKeyValueArguments key_value_pairs_p; ParserFunctionWithKeyValueArguments key_value_pairs_p;
@ -164,12 +192,14 @@ bool ParserDictionary::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserDictionaryLifetime lifetime_p; ParserDictionaryLifetime lifetime_p;
ParserDictionaryRange range_p; ParserDictionaryRange range_p;
ParserDictionaryLayout layout_p; ParserDictionaryLayout layout_p;
ParserDictionarySettings settings_p;
ASTPtr primary_key; ASTPtr primary_key;
ASTPtr ast_source; ASTPtr ast_source;
ASTPtr ast_lifetime; ASTPtr ast_lifetime;
ASTPtr ast_layout; ASTPtr ast_layout;
ASTPtr ast_range; ASTPtr ast_range;
ASTPtr ast_settings;
/// Primary is required to be the first in dictionary definition /// Primary is required to be the first in dictionary definition
if (primary_key_keyword.ignore(pos) && !expression_list_p.parse(pos, primary_key, expected)) if (primary_key_keyword.ignore(pos) && !expression_list_p.parse(pos, primary_key, expected))
@ -235,6 +265,20 @@ bool ParserDictionary::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
continue; continue;
} }
if (!ast_settings && settings_keyword.ignore(pos, expected))
{
if (!open.ignore(pos))
return false;
if (!settings_p.parse(pos, ast_settings, expected))
return false;
if (!close.ignore(pos))
return false;
continue;
}
break; break;
} }
@ -255,6 +299,9 @@ bool ParserDictionary::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (ast_range) if (ast_range)
query->set(query->range, ast_range); query->set(query->range, ast_range);
if (ast_settings)
query->set(query->dict_settings, ast_settings);
return true; return true;
} }

View File

@ -3,6 +3,8 @@
#include <Parsers/IParser.h> #include <Parsers/IParser.h>
#include <Parsers/IParserBase.h> #include <Parsers/IParserBase.h>
#include <Parsers/ParserSetQuery.h>
namespace DB namespace DB
{ {
@ -37,6 +39,13 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
}; };
class ParserDictionarySettings: public IParserBase
{
protected:
const char * getName() const override { return "settings definition"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// Combines together all parsers from above and also parses primary key and /// Combines together all parsers from above and also parses primary key and
/// dictionary source, which consists of custom key-value pairs: /// dictionary source, which consists of custom key-value pairs:

View File

@ -13,7 +13,7 @@ namespace DB
/// Parse `name = value`. /// Parse `name = value`.
static bool parseNameValuePair(SettingChange & change, IParser::Pos & pos, Expected & expected) bool ParserSetQuery::parseNameValuePair(SettingChange & change, IParser::Pos & pos, Expected & expected)
{ {
ParserIdentifier name_p; ParserIdentifier name_p;
ParserLiteral value_p; ParserLiteral value_p;

View File

@ -14,11 +14,10 @@ class ParserSetQuery : public IParserBase
{ {
public: public:
explicit ParserSetQuery(bool parse_only_internals_ = false) : parse_only_internals(parse_only_internals_) {} explicit ParserSetQuery(bool parse_only_internals_ = false) : parse_only_internals(parse_only_internals_) {}
static bool parseNameValuePair(SettingChange & change, IParser::Pos & pos, Expected & expected);
protected: protected:
const char * getName() const override { return "SET query"; } const char * getName() const override { return "SET query"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
/// Parse the list `name = value` pairs, without SET. /// Parse the list `name = value` pairs, without SET.
bool parse_only_internals; bool parse_only_internals;
}; };

View File

@ -0,0 +1,2 @@
INITIALIZING DICTIONARY
END

View File

@ -0,0 +1,47 @@
DROP DATABASE IF EXISTS database_for_dict;
CREATE DATABASE database_for_dict Engine = Ordinary;
DROP TABLE IF EXISTS database_for_dict.table_for_dict;
CREATE TABLE database_for_dict.table_for_dict
(
key_column UInt64,
second_column UInt64,
third_column String
)
ENGINE = MergeTree()
ORDER BY key_column;
INSERT INTO database_for_dict.table_for_dict VALUES (100500, 10000000, 'Hello world');
DROP DATABASE IF EXISTS ordinary_db;
CREATE DATABASE ordinary_db ENGINE = Ordinary;
DROP DICTIONARY IF EXISTS ordinary_db.dict1;
CREATE DICTIONARY ordinary_db.dict1
(
key_column UInt64 DEFAULT 0,
second_column UInt64 DEFAULT 1,
third_column String DEFAULT 'qqq'
)
PRIMARY KEY key_column
SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict'))
LIFETIME(MIN 1 MAX 10)
LAYOUT(FLAT()) SETTINGS(max_result_bytes=1);
SELECT 'INITIALIZING DICTIONARY';
SELECT dictGetUInt64('ordinary_db.dict1', 'second_column', toUInt64(100500)); -- { serverError 396 }
SELECT 'END';
DROP DICTIONARY IF EXISTS ordinary_db.dict1;
DROP DATABASE IF EXISTS ordinary_db;
DROP TABLE IF EXISTS database_for_dict.table_for_dict;
DROP DATABASE IF EXISTS database_for_dict;