mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #10465 from kekekekule/ddl_settings
Add custom settings support in DDL-queries for CREATE DICTIONARY
This commit is contained in:
commit
4c17542096
@ -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`):
|
||||||
|
|
||||||
|
@ -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)
|
||||||
|
@ -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`):
|
||||||
|
|
||||||
|
@ -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)
|
||||||
|
@ -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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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"; }
|
||||||
|
|
||||||
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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:
|
||||||
|
@ -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;
|
||||||
|
@ -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;
|
||||||
};
|
};
|
||||||
|
@ -0,0 +1,2 @@
|
|||||||
|
INITIALIZING DICTIONARY
|
||||||
|
END
|
@ -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;
|
Loading…
Reference in New Issue
Block a user