diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 826b5ad54de..74f0a03408e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -144,6 +144,7 @@ add_object_library(clickhouse_compression Compression) add_object_library(clickhouse_datastreams DataStreams) add_object_library(clickhouse_datatypes DataTypes) add_object_library(clickhouse_databases Databases) +add_object_library(clickhouse_databases_mysql Databases/MySQL) add_object_library(clickhouse_disks Disks) add_object_library(clickhouse_interpreters Interpreters) add_object_library(clickhouse_interpreters_clusterproxy Interpreters/ClusterProxy) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 3b84dfa8949..1887317c05d 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -1,16 +1,17 @@ +#include + #include #include -#include #include #include #include +#include #include #include #include #include #include #include -#include "DatabaseFactory.h" #include #include @@ -18,12 +19,6 @@ # include "config_core.h" #endif -#if USE_MYSQL -# include -# include -#endif - - namespace DB { @@ -94,42 +89,8 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String return std::make_shared(database_name, context); #if USE_MYSQL - else if (engine_name == "MySQL") - { - const ASTFunction * engine = engine_define->engine; - - if (!engine->arguments || engine->arguments->children.size() != 4) - throw Exception("MySQL Database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.", - ErrorCodes::BAD_ARGUMENTS); - - - ASTs & arguments = engine->arguments->children; - arguments[1] = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[1], context); - - const auto & host_name_and_port = safeGetLiteralValue(arguments[0], "MySQL"); - const auto & database_name_in_mysql = safeGetLiteralValue(arguments[1], "MySQL"); - const auto & mysql_user_name = safeGetLiteralValue(arguments[2], "MySQL"); - const auto & mysql_user_password = safeGetLiteralValue(arguments[3], "MySQL"); - - try - { - const auto & [remote_host_name, remote_port] = parseAddress(host_name_and_port, 3306); - auto mysql_pool = mysqlxx::Pool(database_name_in_mysql, remote_host_name, mysql_user_name, mysql_user_password, remote_port); - - auto mysql_database = std::make_shared( - context, database_name, metadata_path, engine_define, database_name_in_mysql, std::move(mysql_pool)); - - mysql_database->empty(); /// test database is works fine. - return mysql_database; - } - catch (...) - { - const auto & exception_message = getCurrentExceptionMessage(true); - throw Exception("Cannot create MySQL database, because " + exception_message, ErrorCodes::CANNOT_CREATE_DATABASE); - } - } - + return createMySQLDatabase(database_name, metadata_path, engine_define, context); #endif else if (engine_name == "Lazy") diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp new file mode 100644 index 00000000000..65e5a76ae98 --- /dev/null +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.cpp @@ -0,0 +1,97 @@ +//#include +// +//#include +//#include +//#include +//#include +//#include +//#include +//#include +// +//namespace DB +//{ +// +//static MasterStatusInfo fetchMasterStatus(const mysqlxx::PoolWithFailover::Entry & connection) +//{ +// Block header +// { +// {std::make_shared(), "File"}, +// {std::make_shared(), "Position"}, +// {std::make_shared(), "Binlog_Do_DB"}, +// {std::make_shared(), "Binlog_Ignore_DB"}, +// {std::make_shared(), "Executed_Gtid_Set"}, +// }; +// +// MySQLBlockInputStream input(connection, "SHOW MASTER STATUS;", header, DEFAULT_BLOCK_SIZE); +// Block master_status = input.read(); +// +// if (!master_status || master_status.rows() != 1) +// throw Exception("Unable to get master status from MySQL.", ErrorCodes::LOGICAL_ERROR); +// +// return MasterStatusInfo +// { +// (*master_status.getByPosition(0).column)[0].safeGet(), +// (*master_status.getByPosition(1).column)[0].safeGet(), +// (*master_status.getByPosition(2).column)[0].safeGet(), +// (*master_status.getByPosition(3).column)[0].safeGet(), +// (*master_status.getByPosition(4).column)[0].safeGet() +// }; +//} +// +//static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entry & connection, const std::string & database) +//{ +// Block header{{std::make_shared(), "table_name"}}; +// String query = "SELECT TABLE_NAME AS table_name FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA = " + quoteString(database); +// +// std::vector tables_in_db; +// MySQLBlockInputStream input(connection, query, header, DEFAULT_BLOCK_SIZE); +// +// while (Block block = input.read()) +// { +// tables_in_db.reserve(tables_in_db.size() + block.rows()); +// for (size_t index = 0; index < block.rows(); ++index) +// tables_in_db.emplace_back((*block.getByPosition(0).column)[index].safeGet()); +// } +// +// return tables_in_db; +//} +// +//DatabaseMaterializeMySQL::DatabaseMaterializeMySQL( +// const Context & context, const String & database_name_, const String & metadata_path_, +// const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_) +// : IDatabase(database_name_) +// , global_context(context.getGlobalContext()), metadata_path(metadata_path_) +// , database_engine_define(database_engine_define_->clone()), mysql_database_name(mysql_database_name_), pool(std::move(pool_)) +//{ +// try +// { +// mysqlxx::PoolWithFailover::Entry connection = pool.get(); +// +// connection->query("FLUSH TABLES;").execute(); +// connection->query("FLUSH TABLES WITH READ LOCK;").execute(); +// +// MasterStatusInfo master_status = fetchMasterStatus(connection); +// connection->query("SET SESSION TRANSACTION ISOLATION LEVEL REPEATABLE READ;").execute(); +// connection->query("START TRANSACTION /*!40100 WITH CONSISTENT SNAPSHOT */;").execute(); +// +// std::vector tables_in_db = fetchTablesInDB(connection, mysql_database_name); +// connection->query("UNLOCK TABLES;"); +// +// for (const auto & dumping_table_name : tables_in_db) +// { +// /// TODO: 查询表结构, 根据不同的模式创建对应的表(暂时只支持多version即可) +//// connection->query("SHOW CREATE TABLE " + doubleQuoteString()) +// MySQLBlockInputStream input( +// "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(dumping_table_name)); +// /// TODO: 查询所有数据写入对应表中(全量dump) +// /// TODO: 启动slave, 监听事件 +// } +// } +// catch (...) +// { +// throw; +// } +//} +// +// +//} diff --git a/src/Databases/MySQL/DatabaseMaterializeMySQL.h b/src/Databases/MySQL/DatabaseMaterializeMySQL.h new file mode 100644 index 00000000000..bb7e6193451 --- /dev/null +++ b/src/Databases/MySQL/DatabaseMaterializeMySQL.h @@ -0,0 +1,30 @@ +//#pragma once +// +//#include +//#include +//#include +//#include +//#include +// +//namespace DB +//{ +// +//class DatabaseMaterializeMySQL : public IDatabase +//{ +//public: +// DatabaseMaterializeMySQL( +// const Context & context, const String & database_name_, const String & metadata_path_, +// const ASTStorage * database_engine_define_, const String & mysql_database_name_, mysqlxx::Pool && pool_); +// +// String getEngineName() const override { return "MySQL"; } +// +//private: +// const Context & global_context; +// String metadata_path; +// ASTPtr database_engine_define; +// String mysql_database_name; +// +// mutable mysqlxx::Pool pool; +//}; +// +//} diff --git a/src/Databases/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp similarity index 99% rename from src/Databases/DatabaseMySQL.cpp rename to src/Databases/MySQL/DatabaseMySQL.cpp index 3b026bf9468..ff12446331a 100644 --- a/src/Databases/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -9,7 +9,7 @@ # include # include # include -# include +# include # include # include # include @@ -70,6 +70,7 @@ DatabaseMySQL::DatabaseMySQL( , database_name_in_mysql(database_name_in_mysql_) , mysql_pool(std::move(pool)) { + empty(); /// test database is works fine. } bool DatabaseMySQL::empty() const diff --git a/src/Databases/DatabaseMySQL.h b/src/Databases/MySQL/DatabaseMySQL.h similarity index 100% rename from src/Databases/DatabaseMySQL.h rename to src/Databases/MySQL/DatabaseMySQL.h diff --git a/src/Databases/MySQL/MasterStatusInfo.cpp b/src/Databases/MySQL/MasterStatusInfo.cpp new file mode 100644 index 00000000000..5c53db22bc4 --- /dev/null +++ b/src/Databases/MySQL/MasterStatusInfo.cpp @@ -0,0 +1,6 @@ +#include + +namespace DB +{ + +} diff --git a/src/Databases/MySQL/MasterStatusInfo.h b/src/Databases/MySQL/MasterStatusInfo.h new file mode 100644 index 00000000000..aea50aa3bdd --- /dev/null +++ b/src/Databases/MySQL/MasterStatusInfo.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include + +namespace DB +{ + +struct MasterStatusInfo +{ + String binlog_file; + UInt64 binlog_position; + String binlog_do_db; + String binlog_ignore_db; + String executed_gtid_set; + + MasterStatusInfo( + String binlog_file_, UInt64 binlog_position_, String binlog_do_db_, String binlog_ignore_db_, String executed_gtid_set_); + + +}; + + +std::shared_ptr fetchMasterStatusInfo(mysqlxx::Connection * connection); + +} + diff --git a/src/Databases/MySQL/createMySQLDatabase.cpp b/src/Databases/MySQL/createMySQLDatabase.cpp new file mode 100644 index 00000000000..24417d83791 --- /dev/null +++ b/src/Databases/MySQL/createMySQLDatabase.cpp @@ -0,0 +1,84 @@ +#include + +#if USE_MYSQL + +#include +#include +#include +#include +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int CANNOT_CREATE_DATABASE; +} + +static inline String safeGetLiteralValue(const ASTPtr & ast) +{ + if (!ast || !ast->as()) + throw Exception("Database engine MySQL requested literal argument.", ErrorCodes::BAD_ARGUMENTS); + + return ast->as()->value.safeGet(); +} + +/*static inline bool materializeMySQLDatabase(const ASTSetQuery * settings) +{ + if (!settings || settings->changes.empty()) + return false; + + for (const auto & change : settings->changes) + { + if (change.name == "materialize_data") + { + if (change.value.getType() == Field::Types::String) + return change.value.safeGet() == "true"; ///TODO: ignore case + } + + } + return false; +}*/ + +DatabasePtr createMySQLDatabase(const String & database_name, const String & metadata_path, const ASTStorage * define, Context & context) +{ + const ASTFunction * engine = define->engine; + if (!engine->arguments || engine->arguments->children.size() != 4) + throw Exception( "MySQL Database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.", + ErrorCodes::BAD_ARGUMENTS); + + ASTs & arguments = engine->arguments->children; + arguments[1] = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[1], context); + + const auto & host_name_and_port = safeGetLiteralValue(arguments[0]); + const auto & mysql_database_name = safeGetLiteralValue(arguments[1]); + const auto & mysql_user_name = safeGetLiteralValue(arguments[2]); + const auto & mysql_user_password = safeGetLiteralValue(arguments[3]); + + try + { + const auto & [remote_host_name, remote_port] = parseAddress(host_name_and_port, 3306); + auto mysql_pool = mysqlxx::Pool(mysql_database_name, remote_host_name, mysql_user_name, mysql_user_password, remote_port); + + /*if (materializeMySQLDatabase(define->settings)) + return std::make_shared( + context, database_name, metadata_path, define, mysql_database_name, std::move(mysql_pool));*/ + + return std::make_shared(context, database_name, metadata_path, define, mysql_database_name, std::move(mysql_pool)); + } + catch (...) + { + const auto & exception_message = getCurrentExceptionMessage(true); + throw Exception("Cannot create MySQL database, because " + exception_message, ErrorCodes::CANNOT_CREATE_DATABASE); + } +} + +} + +#endif diff --git a/src/Databases/MySQL/createMySQLDatabase.h b/src/Databases/MySQL/createMySQLDatabase.h new file mode 100644 index 00000000000..f85e96428fe --- /dev/null +++ b/src/Databases/MySQL/createMySQLDatabase.h @@ -0,0 +1,19 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + +#if USE_MYSQL + +#include +#include + +namespace DB +{ + +DatabasePtr createMySQLDatabase(const String & database_name, const String & metadata_path, const ASTStorage * define, Context & context); + +} + +#endif diff --git a/src/Parsers/CMakeLists.txt b/src/Parsers/CMakeLists.txt index b83fc20e818..13e460da4e4 100644 --- a/src/Parsers/CMakeLists.txt +++ b/src/Parsers/CMakeLists.txt @@ -1,5 +1,6 @@ include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) add_headers_and_sources(clickhouse_parsers .) +add_headers_and_sources(clickhouse_parsers ./MySQL) add_library(clickhouse_parsers ${clickhouse_parsers_headers} ${clickhouse_parsers_sources}) target_link_libraries(clickhouse_parsers PUBLIC clickhouse_common_io) diff --git a/src/Parsers/MySQL/ASTCreateDefines.cpp b/src/Parsers/MySQL/ASTCreateDefines.cpp new file mode 100644 index 00000000000..c01602b05c6 --- /dev/null +++ b/src/Parsers/MySQL/ASTCreateDefines.cpp @@ -0,0 +1,95 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ParserCreateDefine : public IParserBase +{ +protected: + + const char * getName() const override { return "table property (column, index, constraint)"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + MySQLParser::ParserDeclareIndex p_declare_index; + MySQLParser::ParserDeclareColumn p_declare_column; + MySQLParser::ParserDeclareConstraint p_declare_constraint; + + if (likely(!p_declare_index.parse(pos, node, expected))) + { + if (likely(!p_declare_constraint.parse(pos, node, expected))) + { + if (!p_declare_column.parse(pos, node, expected)) + return false; + } + } + + return true; + } +}; + +ASTPtr ASTCreateDefines::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (columns) + res->set(res->columns, columns->clone()); + + if (indices) + res->set(res->indices, indices->clone()); + + if (constraints) + res->set(res->constraints, constraints->clone()); + + return res; +} + +bool ParserCreateDefines::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr create_defines; + ParserList create_defines_parser(std::make_unique(), std::make_unique(TokenType::Comma), false); + + if (!create_defines_parser.parse(pos, create_defines, expected)) + return false; + + ASTPtr columns = std::make_shared(); + ASTPtr indices = std::make_shared(); + ASTPtr constraints = std::make_shared(); + + for (const auto & create_define : create_defines->children) + { + if (create_define->as()) + columns->children.push_back(create_define); + else if (create_define->as()) + indices->children.push_back(create_define); + else if (create_define->as()) + constraints->children.push_back(create_define); + else + return false; + } + + auto res = std::make_shared(); + if (!columns->children.empty()) + res->set(res->columns, columns); + if (!indices->children.empty()) + res->set(res->indices, indices); + if (!constraints->children.empty()) + res->set(res->constraints, constraints); + + node = res; + return true; +} + +} + +} + diff --git a/src/Parsers/MySQL/ASTCreateDefines.h b/src/Parsers/MySQL/ASTCreateDefines.h new file mode 100644 index 00000000000..72adb1c745a --- /dev/null +++ b/src/Parsers/MySQL/ASTCreateDefines.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + + +class ASTCreateDefines : public IAST +{ +public: + ASTExpressionList * columns = nullptr; + ASTExpressionList * indices = nullptr; + ASTExpressionList * constraints = nullptr; + + ASTPtr clone() const override; + + String getID(char) const override { return "Create definitions"; } +}; + +class ParserCreateDefines : public IParserBase +{ +protected: + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + + const char * getName() const override { return "table property list (column, index, constraint)"; } +}; + +} + +} + diff --git a/src/Parsers/MySQL/ASTCreateQuery.cpp b/src/Parsers/MySQL/ASTCreateQuery.cpp new file mode 100644 index 00000000000..a200ce2e0ff --- /dev/null +++ b/src/Parsers/MySQL/ASTCreateQuery.cpp @@ -0,0 +1,128 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +ASTPtr ASTCreateQuery::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (columns_list) + { + res->columns_list = columns_list->clone(); + res->children.emplace_back(res->columns_list); + } + + if (table_options) + { + res->table_options = table_options->clone(); + res->children.emplace_back(res->table_options); + } + + if (partition_options) + { + res->partition_options = partition_options->clone(); + res->children.emplace_back(res->partition_options); + } + + return res; +} + +bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr table; + ASTPtr like_table; + ASTPtr columns_list; + ASTPtr table_options; + ASTPtr partition_options; + bool is_temporary = false; + bool if_not_exists = false; + + if (!ParserKeyword("CREATE").ignore(pos, expected)) + return false; + + if (ParserKeyword("TEMPORARY").ignore(pos, expected)) + is_temporary = true; + + if (!ParserKeyword("TABLE").ignore(pos, expected)) + return false; + + if (ParserKeyword("IF NOT EXISTS").ignore(pos, expected)) + if_not_exists = true; + + if (!ParserCompoundIdentifier(true).parse(pos, table, expected)) + return false; + + if (ParserKeyword("LIKE").ignore(pos, expected)) + { + if (!ParserCompoundIdentifier(true).parse(pos, like_table, expected)) + return false; + } + + if (ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)) + { + if (ParserKeyword("LIKE").ignore(pos, expected)) + { + if (!ParserCompoundIdentifier(true).parse(pos, like_table, expected)) + return false; + + if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) + return false; + } + else + { + if (!ParserCreateDefines().parse(pos, columns_list, expected)) + return false; + + if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) + return false; + + ParserDeclareTableOptions().parse(pos, table_options, expected); + ParserDeclarePartitionOptions().parse(pos, partition_options, expected); + } + } + + auto create_query = std::make_shared(); + + create_query->temporary = is_temporary; + create_query->if_not_exists = if_not_exists; + + StorageID table_id = getTableIdentifier(table); + create_query->table = table_id.table_name; + create_query->database = table_id.database_name; + create_query->like_table = like_table; + create_query->columns_list = columns_list; + create_query->table_options = table_options; + create_query->partition_options = partition_options; + + if (create_query->like_table) + create_query->children.emplace_back(create_query->like_table); + + if (create_query->columns_list) + create_query->children.emplace_back(create_query->columns_list); + + if (create_query->table_options) + create_query->children.emplace_back(create_query->table_options); + + if (create_query->partition_options) + create_query->children.emplace_back(create_query->partition_options); + + node = create_query; + return true; +} +} + +} diff --git a/src/Parsers/MySQL/ASTCreateQuery.h b/src/Parsers/MySQL/ASTCreateQuery.h new file mode 100644 index 00000000000..2d1deb23d4b --- /dev/null +++ b/src/Parsers/MySQL/ASTCreateQuery.h @@ -0,0 +1,40 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTCreateQuery : public IAST +{ +public: + bool temporary{false}; + bool if_not_exists{false}; + + String table; + String database; + ASTPtr like_table; + ASTPtr columns_list; + ASTPtr table_options; + ASTPtr partition_options; + + ASTPtr clone() const override; + + String getID(char) const override { return "create query"; } +}; + +class ParserCreateQuery : public IParserBase +{ +protected: + const char * getName() const override { return "create query"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareColumn.cpp b/src/Parsers/MySQL/ASTDeclareColumn.cpp new file mode 100644 index 00000000000..295818a66c5 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareColumn.cpp @@ -0,0 +1,98 @@ +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +ASTPtr ASTDeclareColumn::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (data_type) + { + res->data_type = data_type->clone(); + res->children.emplace_back(res->data_type); + } + + if (column_options) + { + res->column_options = column_options->clone(); + res->children.emplace_back(res->column_options); + } + + return res; +} + +bool ParserDeclareColumn::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr column_name; + ASTPtr column_data_type; + ASTPtr column_options; + + ParserExpression p_expression; + ParserIdentifier p_identifier; + + if (!p_identifier.parse(pos, column_name, expected)) + return false; + + if (!p_expression.parse(pos, column_data_type, expected)) + return false; + + if (!parseColumnDeclareOptions(pos, column_options, expected)) + return false; + + auto declare_column = std::make_shared(); + declare_column->name = column_name->as()->name; + declare_column->data_type = column_data_type; + declare_column->column_options = column_options; + + if (declare_column->data_type) + declare_column->children.emplace_back(declare_column->data_type); + + if (declare_column->column_options) + declare_column->children.emplace_back(declare_column->column_options); + + node = declare_column; + return true; +} +bool ParserDeclareColumn::parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserDeclareOption p_non_generate_options{ + { + OptionDescribe("NULL", "is_null", std::make_unique()), + OptionDescribe("NOT NULL", "is_null", std::make_unique()), + OptionDescribe("DEFAULT", "default", std::make_unique()), + OptionDescribe("AUTO_INCREMENT", "auto_increment", std::make_unique()), + OptionDescribe("UNIQUE", "unique_key", std::make_unique()), + OptionDescribe("UNIQUE KEY", "unique_key", std::make_unique()), + OptionDescribe("KEY", "primary_key", std::make_unique()), + OptionDescribe("PRIMARY KEY", "primary_key", std::make_unique()), + OptionDescribe("COMMENT", "comment", std::make_unique()), + OptionDescribe("COLLATE", "collate", std::make_unique()), + OptionDescribe("COLUMN_FORMAT", "column_format", std::make_unique()), + OptionDescribe("STORAGE", "storage", std::make_unique()), + OptionDescribe("AS", "generated", std::make_unique()), + OptionDescribe("GENERATED ALWAYS AS", "generated", std::make_unique()), + OptionDescribe("STORED", "is_stored", std::make_unique()), + OptionDescribe("VIRTUAL", "is_stored", std::make_unique()), + OptionDescribe("", "reference", std::make_unique()), + OptionDescribe("", "constraint", std::make_unique()), + } + }; + + return p_non_generate_options.parse(pos, node, expected); +} + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareColumn.h b/src/Parsers/MySQL/ASTDeclareColumn.h new file mode 100644 index 00000000000..25103072e71 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareColumn.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTDeclareColumn : public IAST +{ +public: + String name; + ASTPtr data_type; + ASTPtr column_options; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "Column definition"; } +}; + +class ParserDeclareColumn : public IParserBase +{ +protected: + const char * getName() const override { return "index declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + + bool parseColumnDeclareOptions(Pos & pos, ASTPtr & node, Expected & expected); +}; + + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareConstraint.cpp b/src/Parsers/MySQL/ASTDeclareConstraint.cpp new file mode 100644 index 00000000000..0f447fb3b40 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareConstraint.cpp @@ -0,0 +1,74 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +ASTPtr ASTDeclareConstraint::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (check_expression) + { + res->check_expression = check_expression->clone(); + res->children.emplace_back(res->check_expression); + } + + return res; +} + +bool ParserDeclareConstraint::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + bool enforced = true; + ASTPtr constraint_symbol; + ASTPtr index_check_expression; + ParserExpression p_expression; + + if (ParserKeyword("CONSTRAINT").ignore(pos, expected)) + { + if (!ParserKeyword("CHECK").checkWithoutMoving(pos, expected)) + ParserIdentifier().parse(pos, constraint_symbol, expected); + } + + + if (!ParserKeyword("CHECK").ignore(pos, expected)) + return false; + + if (!p_expression.parse(pos, index_check_expression, expected)) + return false; + + if (ParserKeyword("NOT").ignore(pos, expected)) + { + if (!ParserKeyword("ENFORCED").ignore(pos, expected)) + return false; + + enforced = false; + } + else + { + enforced = true; + ParserKeyword("ENFORCED").ignore(pos, expected); + } + + auto declare_constraint = std::make_shared(); + declare_constraint->enforced = enforced; + declare_constraint->check_expression = index_check_expression; + + if (constraint_symbol) + declare_constraint->constraint_name = constraint_symbol->as()->name; + + node = declare_constraint; + return true; +} + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareConstraint.h b/src/Parsers/MySQL/ASTDeclareConstraint.h new file mode 100644 index 00000000000..6af40e499eb --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareConstraint.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTDeclareConstraint : public IAST +{ +public: + bool enforced{true}; + String constraint_name; + ASTPtr check_expression; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "constraint declaration"; } +}; + +class ParserDeclareConstraint : public IParserBase +{ +protected: + const char * getName() const override { return "constraint declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} + +} + diff --git a/src/Parsers/MySQL/ASTDeclareIndex.cpp b/src/Parsers/MySQL/ASTDeclareIndex.cpp new file mode 100644 index 00000000000..592e01e23a3 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareIndex.cpp @@ -0,0 +1,235 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +struct ParserIndexColumn : public IParserBase +{ +protected: + const char * getName() const override { return "index column"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + ParserExpression p_expression; + + if (!p_expression.parse(pos, node, expected)) + return false; + + ParserKeyword("ASC").ignore(pos, expected); + ParserKeyword("DESC").ignore(pos, expected); + return true; + } +}; + +ASTPtr ASTDeclareIndex::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (index_columns) + { + res->index_columns = index_columns->clone(); + res->children.emplace_back(res->index_columns); + } + + + if (index_options) + { + res->index_options = index_options->clone(); + res->children.emplace_back(res->index_options); + } + + + if (reference_definition) + { + res->reference_definition = reference_definition->clone(); + res->children.emplace_back(res->reference_definition); + } + + return res; +} +bool ParserDeclareIndex::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + String index_name; + String index_type; + ASTPtr index_columns; + ASTPtr index_options; + ASTPtr declare_reference; + ParserIndexColumn p_expression; + + ParserDeclareOption p_index_options{ + { + OptionDescribe("KEY_BLOCK_SIZE", "key_block_size", std::make_unique()), + OptionDescribe("USING", "index_type", std::make_unique()), + OptionDescribe("WITH PARSER", "index_parser", std::make_unique()), + OptionDescribe("COMMENT", "comment", std::make_unique()), + OptionDescribe("VISIBLE", "visible", std::make_unique()), + OptionDescribe("INVISIBLE", "visible", std::make_unique()), + } + }; + + if (!parseDeclareOrdinaryIndex(pos, index_name, index_type, expected)) + { + if (!parseDeclareConstraintIndex(pos, index_name, index_type, expected)) + return false; + } + + ParserToken s_opening_round(TokenType::OpeningRoundBracket); + ParserToken s_closing_round(TokenType::ClosingRoundBracket); + + if (!s_opening_round.ignore(pos, expected)) + return false; + + ParserList p_index_columns(std::make_unique(), std::make_unique(TokenType::Comma)); + + if (!p_index_columns.parse(pos, index_columns, expected)) + return false; + + if (!s_closing_round.ignore(pos, expected)) + return false; + + if (index_type != "FOREIGN") + p_index_options.parse(pos, index_options, expected); + else + { + if (!ParserDeclareReference().parse(pos, declare_reference, expected)) + return false; + } + + auto declare_index = std::make_shared(); + declare_index->index_name = index_name; + declare_index->index_type = index_type; + declare_index->index_columns = index_columns; + declare_index->index_options = index_options; + declare_index->reference_definition = declare_reference; + + if (declare_index->index_columns) + declare_index->children.emplace_back(declare_index->index_columns); + + if (declare_index->index_options) + declare_index->children.emplace_back(declare_index->index_options); + + if (declare_index->reference_definition) + declare_index->children.emplace_back(declare_index->reference_definition); + + node = declare_index; + return true; +} +bool ParserDeclareIndex::parseDeclareOrdinaryIndex(IParser::Pos & pos, String & index_name, String & index_type, Expected & expected) +{ + ASTPtr temp_node; + ParserKeyword k_key("KEY"); + ParserKeyword k_index("INDEX"); + + ParserExpression p_expression; + ParserIdentifier p_identifier; + + if (ParserKeyword("SPATIAL").ignore(pos, expected)) + { + if (!k_key.ignore(pos, expected)) + k_index.ignore(pos, expected); + + index_type = "SPATIAL"; + if (p_identifier.parse(pos, temp_node, expected)) + index_name = temp_node->as()->name; + } + else if (ParserKeyword("FULLTEXT").ignore(pos, expected)) + { + if (!k_key.ignore(pos, expected)) + k_index.ignore(pos, expected); + + index_type = "FULLTEXT"; + if (p_identifier.parse(pos, temp_node, expected)) + index_name = temp_node->as()->name; + } + else + { + if (!k_key.ignore(pos, expected)) + { + if (!k_index.ignore(pos, expected)) + return false; + } + + index_type = "BTREE"; /// default index type + if (p_identifier.parse(pos, temp_node, expected)) + index_name = temp_node->as()->name; + + if (ParserKeyword("USING").ignore(pos, expected)) + { + if (!p_identifier.parse(pos, temp_node, expected)) + return false; + + index_type = temp_node->as()->name; + } + } + + return true; +} + +bool ParserDeclareIndex::parseDeclareConstraintIndex(IParser::Pos & pos, String & index_name, String & index_type, Expected & expected) +{ + ASTPtr temp_node; + ParserIdentifier p_identifier; + + if (ParserKeyword("CONSTRAINT").ignore(pos, expected)) + { + + if (!ParserKeyword("PRIMARY").checkWithoutMoving(pos, expected) && !ParserKeyword("UNIQUE").checkWithoutMoving(pos, expected) + && !ParserKeyword("FOREIGN").checkWithoutMoving(pos, expected)) + { + if (!p_identifier.parse(pos, temp_node, expected)) + return false; + + index_name = temp_node->as()->name; + } + } + + if (ParserKeyword("UNIQUE").ignore(pos, expected)) + { + if (!ParserKeyword("KEY").ignore(pos, expected)) + ParserKeyword("INDEX").ignore(pos, expected); + + if (p_identifier.parse(pos, temp_node, expected)) + index_name = temp_node->as()->name; /// reset index_name + + index_type = "UNIQUE_BTREE"; /// default btree index_type + if (ParserKeyword("USING").ignore(pos, expected)) + { + if (!p_identifier.parse(pos, temp_node, expected)) + return false; + + index_type = temp_node->as()->name; + } + } + else if (ParserKeyword("PRIMARY KEY").ignore(pos, expected)) + { + index_type = "PRIMARY_KEY_BTREE"; /// default btree index_type + if (ParserKeyword("USING").ignore(pos, expected)) + { + if (!p_identifier.parse(pos, temp_node, expected)) + return false; + + index_type = temp_node->as()->name; + } + } + else if (ParserKeyword("FOREIGN KEY").ignore(pos, expected)) + { + index_type = "FOREIGN"; + if (p_identifier.parse(pos, temp_node, expected)) + index_name = temp_node->as()->name; /// reset index_name + } + + return true; +} +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareIndex.h b/src/Parsers/MySQL/ASTDeclareIndex.h new file mode 100644 index 00000000000..d3dc3741c0f --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareIndex.h @@ -0,0 +1,43 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTDeclareIndex: public IAST +{ +public: + String index_name; + String index_type; + ASTPtr index_columns; + ASTPtr index_options; + ASTPtr reference_definition; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "index declaration"; } +}; + +class ParserDeclareIndex : public IParserBase +{ +protected: + const char * getName() const override { return "index declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + + bool parseDeclareOrdinaryIndex(Pos & pos, String & index_name, String & index_type, Expected & expected); + + bool parseDeclareConstraintIndex(Pos & pos, String & index_name, String & index_type, Expected & expected); + +}; + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareOption.cpp b/src/Parsers/MySQL/ASTDeclareOption.cpp new file mode 100644 index 00000000000..347fd5d479b --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareOption.cpp @@ -0,0 +1,142 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +bool ParserDeclareOption::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + std::unordered_map changes; + std::unordered_map> usage_parsers_cached; + usage_parsers_cached.reserve(options_collection.size()); + + const auto & get_parser_from_cache = [&](const char * usage_name) + { + auto iterator = usage_parsers_cached.find(usage_name); + if (iterator == usage_parsers_cached.end()) + iterator = usage_parsers_cached.insert(std::make_pair(usage_name, std::make_shared(usage_name))).first; + + return iterator->second; + }; + + while (true) + { + ASTPtr value; + bool found{false}; + for (const auto & option_describe : options_collection) + { + if (strlen(option_describe.usage_name) == 0) + { + if (option_describe.value_parser->parse(pos, value, expected)) + { + found = true; + changes.insert(std::make_pair(option_describe.option_name, value)); + } + } + else if (get_parser_from_cache(option_describe.usage_name)->ignore(pos, expected)) + { + ParserToken{TokenType::Equals}.ignore(pos, expected); + + if (!option_describe.value_parser->parse(pos, value, expected)) + return false; + + /*const auto & changes_iterator = changes.find(option_describe.option_name); + if (changes_iterator != changes.end()) + throw Exception("Duplicate options declare", ErrorCodes::)*/ + found = true; + changes.insert(std::make_pair(option_describe.option_name, value)); + } + } + + if (!found) + break; + + ParserToken{TokenType::Comma}.ignore(pos, expected); + } + + if (!changes.empty()) + { + auto options_declare = std::make_shared(); + options_declare->changes = changes; + + node = options_declare; + } + + return true; +} + +ASTPtr ASTDeclareOptions::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + res->changes.clear(); + + for (const auto & [name, value] : this->changes) + res->changes.insert(std::make_pair(name, value->clone())); + + return res; +} + +bool ParserAlwaysTrue::parseImpl(IParser::Pos & /*pos*/, ASTPtr & node, Expected & /*expected*/) +{ + node = std::make_shared(Field(UInt64(1))); + return true; +} + +bool ParserAlwaysFalse::parseImpl(IParser::Pos & /*pos*/, ASTPtr & node, Expected & /*expected*/) +{ + node = std::make_shared(Field(UInt64(0))); + return true; +} + +bool ParserCharsetName::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &) +{ + /// Identifier in backquotes or in double quotes + if (pos->type == TokenType::QuotedIdentifier) + { + ReadBufferFromMemory buf(pos->begin, pos->size()); + String s; + + if (*pos->begin == '`') + readBackQuotedStringWithSQLStyle(s, buf); + else + readDoubleQuotedStringWithSQLStyle(s, buf); + + if (s.empty()) /// Identifiers "empty string" are not allowed. + return false; + + node = std::make_shared(s); + ++pos; + return true; + } + else if (pos->type == TokenType::BareWord) + { + const char * begin = pos->begin; + + while (true) + { + if (isWhitespaceASCII(*pos->end)) + break; + else + { + ++pos; + } + } + + node = std::make_shared(String(begin, pos->end)); + ++pos; + return true; + } + + return false; +} +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareOption.h b/src/Parsers/MySQL/ASTDeclareOption.h new file mode 100644 index 00000000000..fa6c25bd914 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareOption.h @@ -0,0 +1,76 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +struct OptionDescribe +{ + const char * usage_name; + String option_name; + std::shared_ptr value_parser; + + OptionDescribe(const char * usage_name_, const String & option_name_, const std::shared_ptr & value_parser_) + :usage_name(usage_name_), option_name(option_name_), value_parser(value_parser_) + { + } +}; + +class ASTDeclareOptions : public IAST +{ +public: + std::unordered_map changes; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "options declaration"; } +}; + +class ParserAlwaysTrue : public IParserBase +{ +public: + const char * getName() const override { return "always true"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +class ParserAlwaysFalse : public IParserBase +{ +public: + const char * getName() const override { return "always false"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +/// Copy and paste from ParserIdentifier, +/// the difference is that multiple tokens are glued if there is no whitespace ASCII between them +struct ParserCharsetName : public IParserBase +{ +protected: + const char * getName() const override { return "charset name"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected &) override; +}; + +class ParserDeclareOption : public IParserBase +{ +protected: + std::vector options_collection; + + const char * getName() const override { return "option declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +public: + ParserDeclareOption(const std::vector & options_collection_) : options_collection(options_collection_) {} +}; + +} + + +} diff --git a/src/Parsers/MySQL/ASTDeclarePartition.cpp b/src/Parsers/MySQL/ASTDeclarePartition.cpp new file mode 100644 index 00000000000..68a56109e64 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclarePartition.cpp @@ -0,0 +1,127 @@ +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +ASTPtr ASTDeclarePartition::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (options) + { + res->options = options->clone(); + res->children.emplace_back(res->options); + } + + if (less_than) + { + res->less_than = less_than->clone(); + res->children.emplace_back(res->less_than); + } + + if (in_expression) + { + res->in_expression = in_expression->clone(); + res->children.emplace_back(res->in_expression); + } + + if (subpartitions) + { + res->subpartitions = subpartitions->clone(); + res->children.emplace_back(res->subpartitions); + } + + return res; +} + +bool ParserDeclarePartition::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + if (!ParserKeyword{"PARTITION"}.ignore(pos, expected)) + return false; + + ASTPtr options; + ASTPtr less_than; + ASTPtr in_expression; + ASTPtr partition_name; + + ParserExpression p_expression; + ParserIdentifier p_identifier; + + if (!p_identifier.parse(pos, partition_name, expected)) + return false; + + ParserKeyword p_values("VALUES"); + if (p_values.ignore(pos, expected)) + { + if (ParserKeyword{"IN"}.ignore(pos, expected)) + { + if (!p_expression.parse(pos, in_expression, expected)) + return false; + } + else if (ParserKeyword{"LESS THAN"}.ignore(pos, expected)) + { + if (!p_expression.parse(pos, less_than, expected)) + return false; + } + } + + if (!ParserDeclareOption{ + { + OptionDescribe("ENGINE", "engine", std::make_shared()), + OptionDescribe("STORAGE ENGINE", "engine", std::make_shared()), + OptionDescribe("COMMENT", "comment", std::make_shared()), + OptionDescribe("DATA DIRECTORY", "data_directory", std::make_shared()), + OptionDescribe("INDEX DIRECTORY", "index_directory", std::make_shared()), + OptionDescribe("MAX_ROWS", "max_rows", std::make_shared()), + OptionDescribe("MIN_ROWS", "min_rows", std::make_shared()), + OptionDescribe("TABLESPACE", "tablespace", std::make_shared()), + } + }.parse(pos, options, expected)) + return false; + + ASTPtr subpartitions; + if (ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)) + { + if (!DB::ParserList(std::make_unique(), std::make_unique(TokenType::Comma)) + .parse(pos, subpartitions, expected)) + return false; + + if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) + return false; + } + + auto partition_declare = std::make_shared(); + partition_declare->options = options; + partition_declare->less_than = less_than; + partition_declare->in_expression = in_expression; + partition_declare->subpartitions = subpartitions; + partition_declare->partition_name = partition_name->as()->name; + + if (partition_declare->options) + partition_declare->children.emplace_back(partition_declare->options); + + if (partition_declare->less_than) + partition_declare->children.emplace_back(partition_declare->less_than); + + if (partition_declare->in_expression) + partition_declare->children.emplace_back(partition_declare->in_expression); + + if (partition_declare->subpartitions) + partition_declare->children.emplace_back(partition_declare->subpartitions); + + node = partition_declare; + return true; +} +} + +} diff --git a/src/Parsers/MySQL/ASTDeclarePartition.h b/src/Parsers/MySQL/ASTDeclarePartition.h new file mode 100644 index 00000000000..ef076dcd6cf --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclarePartition.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTDeclarePartition : public IAST +{ +public: + String partition_name; + ASTPtr less_than; + ASTPtr in_expression; + ASTPtr options; + ASTPtr subpartitions; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "partition declaration"; } +}; + +class ParserDeclarePartition : public IParserBase +{ +protected: + const char * getName() const override { return "partition declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} + +} diff --git a/src/Parsers/MySQL/ASTDeclarePartitionOptions.cpp b/src/Parsers/MySQL/ASTDeclarePartitionOptions.cpp new file mode 100644 index 00000000000..ff1ffb93f0c --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclarePartitionOptions.cpp @@ -0,0 +1,184 @@ +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +ASTPtr ASTDeclarePartitionOptions::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (partition_numbers) + { + res->partition_numbers = partition_numbers->clone(); + res->children.emplace_back(res->partition_numbers); + } + + if (partition_expression) + { + res->partition_expression = partition_expression->clone(); + res->children.emplace_back(res->partition_expression); + } + + if (subpartition_numbers) + { + res->subpartition_numbers = subpartition_numbers->clone(); + res->children.emplace_back(res->subpartition_numbers); + } + + if (subpartition_expression) + { + res->subpartition_expression = subpartition_expression->clone(); + res->children.emplace_back(res->subpartition_expression); + } + + return res; +} + +bool ParserDeclarePartitionOptions::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + String partition_type; + ASTPtr partition_numbers; + ASTPtr partition_expression; + String subpartition_type; + ASTPtr subpartition_numbers; + ASTPtr subpartition_expression; + ASTPtr declare_partitions; + + if (!ParserKeyword("PARTITION BY").ignore(pos, expected)) + return false; + + if (!parsePartitionExpression(pos, partition_type, partition_expression, expected)) + return false; + + if (ParserKeyword("PARTITIONS").ignore(pos, expected)) + { + ParserLiteral p_literal; + if (!p_literal.parse(pos, partition_numbers, expected)) + return false; + } + + if (ParserKeyword("SUBPARTITION BY").ignore(pos, expected)) + { + if (!parsePartitionExpression(pos, subpartition_type, subpartition_expression, expected, true)) + return false; + + if (ParserKeyword("SUBPARTITIONS").ignore(pos, expected)) + { + ParserLiteral p_literal; + if (!p_literal.parse(pos, subpartition_numbers, expected)) + return false; + } + } + + if (ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)) + { + if (!ParserList(std::make_unique(), std::make_unique(TokenType::Comma)) + .parse(pos, declare_partitions, expected)) + return false; + + if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) + return false; + } + + auto declare_partition_options = std::make_shared(); + declare_partition_options->partition_type = partition_type; + declare_partition_options->partition_numbers = partition_numbers; + declare_partition_options->partition_expression = partition_expression; + declare_partition_options->subpartition_type = subpartition_type; + declare_partition_options->subpartition_numbers = subpartition_numbers; + declare_partition_options->subpartition_expression = subpartition_expression; + declare_partition_options->declare_partitions = declare_partitions; + + if (declare_partition_options->partition_numbers) + declare_partition_options->children.emplace_back(declare_partition_options->partition_numbers); + + if (declare_partition_options->partition_expression) + declare_partition_options->children.emplace_back(declare_partition_options->partition_expression); + + if (declare_partition_options->subpartition_numbers) + declare_partition_options->children.emplace_back(declare_partition_options->subpartition_numbers); + + if (declare_partition_options->subpartition_expression) + declare_partition_options->children.emplace_back(declare_partition_options->subpartition_expression); + + if (declare_partition_options->declare_partitions) + declare_partition_options->children.emplace_back(declare_partition_options->declare_partitions); + + node = declare_partition_options; + return true; +} + +bool ParserDeclarePartitionOptions::parsePartitionExpression(Pos & pos, std::string & type, ASTPtr & node, Expected & expected, bool subpartition) +{ + ASTPtr expression; + ParserExpression p_expression; + if (!subpartition && ParserKeyword("LIST").ignore(pos, expected)) + { + type = "list"; + ParserKeyword("COLUMNS").ignore(pos, expected); + if (!p_expression.parse(pos, expression, expected)) + return false; + } + else if (!subpartition && ParserKeyword("RANGE").ignore(pos, expected)) + { + type = "range"; + ParserKeyword("COLUMNS").ignore(pos, expected); + if (!p_expression.parse(pos, expression, expected)) + return false; + } + else + { + if (ParserKeyword("LINEAR").ignore(pos, expected)) + type = "linear_"; + + if (ParserKeyword("KEY").ignore(pos, expected)) + { + type += "key"; + + if (ParserKeyword("ALGORITHM").ignore(pos, expected)) + { + if (!ParserToken(TokenType::Equals).ignore(pos, expected)) + return false; + + ASTPtr algorithm; + ParserLiteral p_literal; + if (!p_literal.parse(pos, algorithm, expected) || !algorithm->as()) + return false; + + UInt64 algorithm_type = algorithm->as()->value.safeGet(); + + if (algorithm_type != 1 && algorithm_type != 2) + return false; + + type += "_" + toString(algorithm_type); + } + + if (!p_expression.parse(pos, expression, expected)) + return false; + } + else if (ParserKeyword("HASH").ignore(pos, expected)) + { + type += "hash"; + if (!p_expression.parse(pos, expression, expected)) + return false; + } + else + return false; + } + + node = expression; + return true; +} +} + +} diff --git a/src/Parsers/MySQL/ASTDeclarePartitionOptions.h b/src/Parsers/MySQL/ASTDeclarePartitionOptions.h new file mode 100644 index 00000000000..f560c66534e --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclarePartitionOptions.h @@ -0,0 +1,40 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTDeclarePartitionOptions : public IAST +{ +public: + String partition_type; + ASTPtr partition_numbers; + ASTPtr partition_expression; + String subpartition_type; + ASTPtr subpartition_numbers; + ASTPtr subpartition_expression; + ASTPtr declare_partitions; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "partition options declaration"; } +}; + +class ParserDeclarePartitionOptions : public IParserBase +{ +protected: + const char * getName() const override { return "partition options declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + + bool parsePartitionExpression(Pos & pos, std::string & type, ASTPtr & node, Expected & expected, bool subpartition = false); +}; + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareReference.cpp b/src/Parsers/MySQL/ASTDeclareReference.cpp new file mode 100644 index 00000000000..434b9561eda --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareReference.cpp @@ -0,0 +1,105 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +bool parseReferenceOption(IParser::Pos & pos, ASTDeclareReference::ReferenceOption & option, Expected & expected) +{ + if (ParserKeyword("RESTRICT").ignore(pos, expected)) + option = ASTDeclareReference::RESTRICT; + else if (ParserKeyword("CASCADE").ignore(pos, expected)) + option = ASTDeclareReference::CASCADE; + else if (ParserKeyword("SET NULL").ignore(pos, expected)) + option = ASTDeclareReference::SET_NULL; + else if (ParserKeyword("NO ACTION").ignore(pos, expected)) + option = ASTDeclareReference::NO_ACTION; + else if (ParserKeyword("SET DEFAULT").ignore(pos, expected)) + option = ASTDeclareReference::SET_DEFAULT; + else + return false; + + return true; +} + +ASTPtr ASTDeclareReference::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (reference_expression) + { + res->reference_expression = reference_expression->clone(); + res->children.emplace_back(res->reference_expression); + } + + return res; +} + +bool ParserDeclareReference::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTPtr table_name; + ASTPtr expression; + ParserExpression p_expression; + ParserIdentifier p_identifier; + ASTDeclareReference::MatchKind match_kind = ASTDeclareReference::MATCH_FULL; + ASTDeclareReference::ReferenceOption delete_option = ASTDeclareReference::RESTRICT; + ASTDeclareReference::ReferenceOption update_option = ASTDeclareReference::RESTRICT; + + if (!ParserKeyword("REFERENCES").ignore(pos, expected)) + return false; + + if (!p_identifier.parse(pos, table_name, expected)) + return false; + + if (!p_expression.parse(pos, expression, expected)) + return false; + + if (ParserKeyword("MATCH").ignore(pos, expected)) + { + if (ParserKeyword("FULL").ignore(pos, expected)) + match_kind = ASTDeclareReference::MATCH_FULL; + else if (ParserKeyword("SIMPLE").ignore(pos, expected)) + match_kind = ASTDeclareReference::MATCH_SIMPLE; + else if (ParserKeyword("PARTIAL").ignore(pos, expected)) + match_kind = ASTDeclareReference::MATCH_PARTIAL; + else + return false; + } + + while (true) + { + if (ParserKeyword("ON DELETE").ignore(pos, expected)) + { + if (!parseReferenceOption(pos, delete_option, expected)) + return false; + } + else if (ParserKeyword("ON UPDATE").ignore(pos, expected)) + { + if (!parseReferenceOption(pos, update_option, expected)) + return false; + } + else + break; + } + + auto declare_reference = std::make_shared(); + declare_reference->kind = match_kind; + declare_reference->on_delete_option = delete_option; + declare_reference->on_update_option = update_option; + declare_reference->reference_expression = expression; + declare_reference->reference_table_name = table_name->as()->name; + + node = declare_reference; + return true; +} +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareReference.h b/src/Parsers/MySQL/ASTDeclareReference.h new file mode 100644 index 00000000000..41cd6597dfb --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareReference.h @@ -0,0 +1,52 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTDeclareReference : public IAST +{ +public: + enum MatchKind + { + MATCH_FULL, + MATCH_PARTIAL, + MATCH_SIMPLE + }; + + enum ReferenceOption + { + RESTRICT, + CASCADE, + SET_NULL, + NO_ACTION, + SET_DEFAULT + }; + + MatchKind kind; + String reference_table_name; + ASTPtr reference_expression; + ReferenceOption on_delete_option; + ReferenceOption on_update_option; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "subpartition declaration"; } +}; + +class ParserDeclareReference : public IParserBase +{ +protected: + const char * getName() const override { return "reference declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareSubPartition.cpp b/src/Parsers/MySQL/ASTDeclareSubPartition.cpp new file mode 100644 index 00000000000..25c04779656 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareSubPartition.cpp @@ -0,0 +1,66 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +bool ParserDeclareSubPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (!ParserKeyword{"SUBPARTITION"}.ignore(pos, expected)) + return false; + + ASTPtr options; + ASTPtr logical_name; + ParserIdentifier p_identifier; + + if (!p_identifier.parse(pos, logical_name, expected)) + return false; + + if (!ParserDeclareOption{ + { + OptionDescribe("ENGINE", "engine", std::make_shared()), + OptionDescribe("STORAGE ENGINE", "engine", std::make_shared()), + OptionDescribe("COMMENT", "comment", std::make_shared()), + OptionDescribe("DATA DIRECTORY", "data_directory", std::make_shared()), + OptionDescribe("INDEX DIRECTORY", "index_directory", std::make_shared()), + OptionDescribe("MAX_ROWS", "max_rows", std::make_shared()), + OptionDescribe("MIN_ROWS", "min_rows", std::make_shared()), + OptionDescribe("TABLESPACE", "tablespace", std::make_shared()), + } + }.parse(pos, options, expected)) + return false; + + auto subpartition_declare = std::make_shared(); + subpartition_declare->options = options; + subpartition_declare->logical_name = logical_name->as()->name; + + if (subpartition_declare->options) + subpartition_declare->children.emplace_back(subpartition_declare->options); + + node = subpartition_declare; + return true; +} + +ASTPtr ASTDeclareSubPartition::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (options) + { + res->options = options->clone(); + res->children.emplace_back(res->options); + } + + return res; +} +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareSubPartition.h b/src/Parsers/MySQL/ASTDeclareSubPartition.h new file mode 100644 index 00000000000..018cf2c8c4e --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareSubPartition.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ASTDeclareSubPartition : public IAST +{ +public: + ASTPtr options; + String logical_name; + + ASTPtr clone() const override; + + String getID(char /*delimiter*/) const override { return "subpartition declaration"; } +}; + +class ParserDeclareSubPartition : public IParserBase +{ +protected: + const char * getName() const override { return "subpartition declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareTableOptions.cpp b/src/Parsers/MySQL/ASTDeclareTableOptions.cpp new file mode 100644 index 00000000000..066e8c4c414 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareTableOptions.cpp @@ -0,0 +1,102 @@ +#include + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace MySQLParser +{ + +template +struct ParserBoolOption : public IParserBase +{ +protected: + const char * getName() const override { return "bool option with default"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + if constexpr(allow_default) + { + if (ParserKeyword("DEFAULT").ignore(pos, expected)) + { + node = std::make_shared("DEFAULT"); + return true; + } + } + ParserLiteral p_literal; + if (!p_literal.parse(pos, node, expected) || !node->as()) + return false; + + return !(node->as()->value.safeGet() != 0 && node->as()->value.safeGet() != 1); + } +}; + +struct ParserTablespaceName : public IParserBase +{ +protected: + const char * getName() const override { return "table space name"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + ParserIdentifier p_identifier; + if (!p_identifier.parse(pos, node, expected)) + return false; + + if (ParserKeyword("STORAGE").ignore(pos, expected)) + { + if (!ParserKeyword("DISK").ignore(pos, expected)) + { + if (!ParserKeyword("MEMORY").ignore(pos, expected)) + return false; + } + } + + return true; + } +}; + +bool ParserDeclareTableOptions::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected) +{ + return ParserDeclareOption{ + { + OptionDescribe("AUTO_INCREMENT", "auto_increment", std::make_shared()), + OptionDescribe("AVG_ROW_LENGTH", "avg_row_length", std::make_shared()), + OptionDescribe("CHARACTER SET", "character_set", std::make_shared()), + OptionDescribe("DEFAULT CHARACTER SET", "character_set", std::make_shared()), + OptionDescribe("CHECKSUM", "checksum", std::make_shared>()), + OptionDescribe("COLLATE", "collate", std::make_shared()), + OptionDescribe("DEFAULT COLLATE", "collate", std::make_shared()), + OptionDescribe("COMMENT", "comment", std::make_shared()), + OptionDescribe("COMPRESSION", "compression", std::make_shared()), + OptionDescribe("CONNECTION", "connection", std::make_shared()), + OptionDescribe("DATA DIRECTORY", "data_directory", std::make_shared()), + OptionDescribe("INDEX DIRECTORY", "index_directory", std::make_shared()), + OptionDescribe("DELAY_KEY_WRITE", "delay_key_write", std::make_shared>()), + OptionDescribe("ENCRYPTION", "encryption", std::make_shared()), + OptionDescribe("ENGINE", "engine", std::make_shared()), + OptionDescribe("INSERT_METHOD", "insert_method", std::make_shared()), + OptionDescribe("KEY_BLOCK_SIZE", "key_block_size", std::make_shared()), + OptionDescribe("MAX_ROWS", "max_rows", std::make_shared()), + OptionDescribe("MIN_ROWS", "min_rows", std::make_shared()), + OptionDescribe("PACK_KEYS", "pack_keys", std::make_shared>()), + OptionDescribe("PASSWORD", "password", std::make_shared()), + OptionDescribe("ROW_FORMAT", "row_format", std::make_shared()), + OptionDescribe("STATS_AUTO_RECALC", "stats_auto_recalc", std::make_shared>()), + OptionDescribe("STATS_PERSISTENT", "stats_persistent", std::make_shared>()), + OptionDescribe("STATS_SAMPLE_PAGES", "stats_sample_pages", std::make_shared()), + OptionDescribe("TABLESPACE", "tablespace", std::make_shared()), + OptionDescribe("UNION", "union", std::make_shared()), + } + }.parse(pos, node, expected); +} + +} + +} diff --git a/src/Parsers/MySQL/ASTDeclareTableOptions.h b/src/Parsers/MySQL/ASTDeclareTableOptions.h new file mode 100644 index 00000000000..38697eae754 --- /dev/null +++ b/src/Parsers/MySQL/ASTDeclareTableOptions.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ + +namespace MySQLParser +{ + +class ParserDeclareTableOptions : public IParserBase +{ +protected: + const char * getName() const override { return "table options declaration"; } + + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + +} + +} diff --git a/src/Parsers/MySQL/tests/gtest_column_parser.cpp b/src/Parsers/MySQL/tests/gtest_column_parser.cpp new file mode 100644 index 00000000000..ef6371f71d9 --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_column_parser.cpp @@ -0,0 +1,60 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserColumn, AllNonGeneratedColumnOption) +{ + ParserDeclareColumn p_column; + + String input = "col_01 VARCHAR(100) NOT NULL DEFAULT NULL AUTO_INCREMENT UNIQUE KEY PRIMARY KEY COMMENT 'column comment' COLLATE utf-8 " + "COLUMN_FORMAT FIXED STORAGE MEMORY REFERENCES tbl_name (col_01) CHECK 1"; + ASTPtr ast = parseQuery(p_column, input.data(), input.data() + input.size(), "", 0, 0); + EXPECT_EQ(ast->as()->name, "col_01"); + EXPECT_EQ(ast->as()->data_type->as()->name, "VARCHAR"); + EXPECT_EQ(ast->as()->data_type->as()->arguments->children[0]->as()->value.safeGet(), 100); + + ASTDeclareOptions * declare_options = ast->as()->column_options->as(); + EXPECT_EQ(declare_options->changes["is_null"]->as()->value.safeGet(), 0); + EXPECT_TRUE(declare_options->changes["default"]->as()->value.isNull()); + EXPECT_EQ(declare_options->changes["auto_increment"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["unique_key"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["primary_key"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["comment"]->as()->value.safeGet(), "column comment"); + EXPECT_EQ(declare_options->changes["collate"]->as()->name, "utf-8"); + EXPECT_EQ(declare_options->changes["column_format"]->as()->name, "FIXED"); + EXPECT_EQ(declare_options->changes["storage"]->as()->name, "MEMORY"); + EXPECT_TRUE(declare_options->changes["reference"]->as()); + EXPECT_TRUE(declare_options->changes["constraint"]->as()); +} + +TEST(ParserColumn, AllGeneratedColumnOption) +{ + ParserDeclareColumn p_column; + + String input = "col_01 VARCHAR(100) NULL UNIQUE KEY PRIMARY KEY COMMENT 'column comment' COLLATE utf-8 " + "REFERENCES tbl_name (col_01) CHECK 1 GENERATED ALWAYS AS (1) STORED"; + ASTPtr ast = parseQuery(p_column, input.data(), input.data() + input.size(), "", 0, 0); + EXPECT_EQ(ast->as()->name, "col_01"); + EXPECT_EQ(ast->as()->data_type->as()->name, "VARCHAR"); + EXPECT_EQ(ast->as()->data_type->as()->arguments->children[0]->as()->value.safeGet(), 100); + + ASTDeclareOptions * declare_options = ast->as()->column_options->as(); + EXPECT_EQ(declare_options->changes["is_null"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["unique_key"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["primary_key"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["comment"]->as()->value.safeGet(), "column comment"); + EXPECT_EQ(declare_options->changes["collate"]->as()->name, "utf-8"); + EXPECT_EQ(declare_options->changes["generated"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["is_stored"]->as()->value.safeGet(), 1); + EXPECT_TRUE(declare_options->changes["reference"]->as()); + EXPECT_TRUE(declare_options->changes["constraint"]->as()); +} diff --git a/src/Parsers/MySQL/tests/gtest_constraint_parser.cpp b/src/Parsers/MySQL/tests/gtest_constraint_parser.cpp new file mode 100644 index 00000000000..7f0e9a8060d --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_constraint_parser.cpp @@ -0,0 +1,57 @@ +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserConstraint, CheckConstraint) +{ + /// [CONSTRAINT [symbol]] CHECK (expr) [[NOT] ENFORCED] + ParserDeclareConstraint p_constraint; + + String constraint_01 = "CONSTRAINT symbol_name CHECK col_01 = 1"; + ASTPtr ast_constraint_01 = parseQuery(p_constraint, constraint_01.data(), constraint_01.data() + constraint_01.size(), "", 0, 0); + EXPECT_EQ(ast_constraint_01->as()->constraint_name, "symbol_name"); + auto check_expression_01 = ast_constraint_01->as()->check_expression->as(); + EXPECT_EQ(check_expression_01->name, "equals"); + EXPECT_EQ(check_expression_01->arguments->children[0]->as()->name, "col_01"); + EXPECT_EQ(check_expression_01->arguments->children[1]->as()->value.safeGet(), 1); + + String constraint_02 = "CONSTRAINT CHECK col_01 = 1"; + ASTPtr ast_constraint_02 = parseQuery(p_constraint, constraint_02.data(), constraint_02.data() + constraint_02.size(), "", 0, 0); + EXPECT_EQ(ast_constraint_02->as()->constraint_name, ""); + auto check_expression_02 = ast_constraint_02->as()->check_expression->as(); + EXPECT_EQ(check_expression_02->name, "equals"); + EXPECT_EQ(check_expression_02->arguments->children[0]->as()->name, "col_01"); + EXPECT_EQ(check_expression_02->arguments->children[1]->as()->value.safeGet(), 1); + + String constraint_03 = "CHECK col_01 = 1"; + ASTPtr ast_constraint_03 = parseQuery(p_constraint, constraint_03.data(), constraint_03.data() + constraint_03.size(), "", 0, 0); + EXPECT_EQ(ast_constraint_03->as()->constraint_name, ""); + auto check_expression_03 = ast_constraint_03->as()->check_expression->as(); + EXPECT_EQ(check_expression_03->name, "equals"); + EXPECT_EQ(check_expression_03->arguments->children[0]->as()->name, "col_01"); + EXPECT_EQ(check_expression_03->arguments->children[1]->as()->value.safeGet(), 1); + + String constraint_04 = "CONSTRAINT CHECK col_01 = 1 ENFORCED"; + ASTPtr ast_constraint_04 = parseQuery(p_constraint, constraint_04.data(), constraint_04.data() + constraint_04.size(), "", 0, 0); + EXPECT_TRUE(ast_constraint_04->as()->enforced); + EXPECT_EQ(ast_constraint_04->as()->constraint_name, ""); + auto check_expression_04 = ast_constraint_04->as()->check_expression->as(); + EXPECT_EQ(check_expression_04->name, "equals"); + EXPECT_EQ(check_expression_04->arguments->children[0]->as()->name, "col_01"); + EXPECT_EQ(check_expression_04->arguments->children[1]->as()->value.safeGet(), 1); + + String constraint_05 = "CONSTRAINT CHECK col_01 = 1 NOT ENFORCED"; + ASTPtr ast_constraint_05 = parseQuery(p_constraint, constraint_05.data(), constraint_05.data() + constraint_05.size(), "", 0, 0); + EXPECT_FALSE(ast_constraint_05->as()->enforced); + EXPECT_EQ(ast_constraint_05->as()->constraint_name, ""); + auto check_expression_05 = ast_constraint_05->as()->check_expression->as(); + EXPECT_EQ(check_expression_05->name, "equals"); + EXPECT_EQ(check_expression_05->arguments->children[0]->as()->name, "col_01"); + EXPECT_EQ(check_expression_05->arguments->children[1]->as()->value.safeGet(), 1); +} diff --git a/src/Parsers/MySQL/tests/gtest_create_parser.cpp b/src/Parsers/MySQL/tests/gtest_create_parser.cpp new file mode 100644 index 00000000000..5f752c29a7d --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_create_parser.cpp @@ -0,0 +1,33 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(CreateTableParser, LikeCreate) +{ + ParserCreateQuery p_create_query; + String like_create_01 = "CREATE TABLE IF NOT EXISTS table_name LIKE table_name_01"; + parseQuery(p_create_query, like_create_01.data(), like_create_01.data() + like_create_01.size(), "", 0, 0); + String like_create_02 = "CREATE TABLE IF NOT EXISTS table_name (LIKE table_name_01)"; + parseQuery(p_create_query, like_create_02.data(), like_create_02.data() + like_create_02.size(), "", 0, 0); +} + +TEST(CreateTableParser, SimpleCreate) +{ + ParserCreateQuery p_create_query; + String input = "CREATE TABLE IF NOT EXISTS table_name(col_01 VARCHAR(100), INDEX (col_01), CHECK 1) ENGINE INNODB PARTITION BY HASH(col_01)"; + ASTPtr ast = parseQuery(p_create_query, input.data(), input.data() + input.size(), "", 0, 0); + EXPECT_TRUE(ast->as()->if_not_exists); + EXPECT_EQ(ast->as()->columns_list->as()->columns->children.size(), 1); + EXPECT_EQ(ast->as()->columns_list->as()->indices->children.size(), 1); + EXPECT_EQ(ast->as()->columns_list->as()->constraints->children.size(), 1); + EXPECT_EQ(ast->as()->table_options->as()->changes["engine"]->as()->name, "INNODB"); + EXPECT_TRUE(ast->as()->partition_options->as()); +} diff --git a/src/Parsers/MySQL/tests/gtest_index_parser.cpp b/src/Parsers/MySQL/tests/gtest_index_parser.cpp new file mode 100644 index 00000000000..02b3b10acff --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_index_parser.cpp @@ -0,0 +1,125 @@ +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserIndex, AllIndexOptions) +{ + String input = "INDEX (col_01, col_02(100), col_03 DESC) KEY_BLOCK_SIZE 3 USING HASH WITH PARSER parser_name COMMENT 'index comment' VISIBLE"; + + ParserDeclareIndex p_index; + ASTPtr ast = parseQuery(p_index, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclareIndex * declare_index = ast->as(); + EXPECT_EQ(declare_index->index_columns->children[0]->as()->name, "col_01"); + EXPECT_EQ(declare_index->index_columns->children[1]->as()->name, "col_02"); + EXPECT_EQ(declare_index->index_columns->children[1]->as()->arguments->children[0]->as()->value.safeGet(), 100); + EXPECT_EQ(declare_index->index_columns->children[2]->as()->name, "col_03"); + ASTDeclareOptions * declare_options = declare_index->index_options->as(); + EXPECT_EQ(declare_options->changes["key_block_size"]->as()->value.safeGet(), 3); + EXPECT_EQ(declare_options->changes["index_type"]->as()->name, "HASH"); + EXPECT_EQ(declare_options->changes["comment"]->as()->value.safeGet(), "index comment"); + EXPECT_EQ(declare_options->changes["visible"]->as()->value.safeGet(), 1); +} + +TEST(ParserIndex, OptionalIndexOptions) +{ + String input = "INDEX (col_01, col_02(100), col_03 DESC) USING HASH INVISIBLE KEY_BLOCK_SIZE 3"; + + ParserDeclareIndex p_index; + ASTPtr ast = parseQuery(p_index, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclareIndex * declare_index = ast->as(); + EXPECT_EQ(declare_index->index_columns->children[0]->as()->name, "col_01"); + EXPECT_EQ(declare_index->index_columns->children[1]->as()->name, "col_02"); + EXPECT_EQ(declare_index->index_columns->children[1]->as()->arguments->children[0]->as()->value.safeGet(), 100); + EXPECT_EQ(declare_index->index_columns->children[2]->as()->name, "col_03"); + ASTDeclareOptions * declare_options = declare_index->index_options->as(); + EXPECT_EQ(declare_options->changes["index_type"]->as()->name, "HASH"); + EXPECT_EQ(declare_options->changes["visible"]->as()->value.safeGet(), 0); + EXPECT_EQ(declare_options->changes["key_block_size"]->as()->value.safeGet(), 3); +} + +TEST(ParserIndex, OrdinaryIndex) +{ + ParserDeclareIndex p_index; + String non_unique_index_01 = "KEY index_name USING HASH (col_01) INVISIBLE"; + parseQuery(p_index, non_unique_index_01.data(), non_unique_index_01.data() + non_unique_index_01.size(), "", 0, 0); + + String non_unique_index_02 = "INDEX index_name USING HASH (col_01) INVISIBLE"; + parseQuery(p_index, non_unique_index_02.data(), non_unique_index_02.data() + non_unique_index_02.size(), "", 0, 0); + + String fulltext_index_01 = "FULLTEXT index_name (col_01) INVISIBLE"; + parseQuery(p_index, fulltext_index_01.data(), fulltext_index_01.data() + fulltext_index_01.size(), "", 0, 0); + + String fulltext_index_02 = "FULLTEXT INDEX index_name (col_01) INVISIBLE"; + parseQuery(p_index, fulltext_index_02.data(), fulltext_index_02.data() + fulltext_index_02.size(), "", 0, 0); + + String fulltext_index_03 = "FULLTEXT KEY index_name (col_01) INVISIBLE"; + parseQuery(p_index, fulltext_index_03.data(), fulltext_index_03.data() + fulltext_index_03.size(), "", 0, 0); + + String spatial_index_01 = "SPATIAL index_name (col_01) INVISIBLE"; + parseQuery(p_index, spatial_index_01.data(), spatial_index_01.data() + spatial_index_01.size(), "", 0, 0); + + String spatial_index_02 = "SPATIAL INDEX index_name (col_01) INVISIBLE"; + parseQuery(p_index, spatial_index_02.data(), spatial_index_02.data() + spatial_index_02.size(), "", 0, 0); + + String spatial_index_03 = "SPATIAL KEY index_name (col_01) INVISIBLE"; + parseQuery(p_index, spatial_index_03.data(), spatial_index_03.data() + spatial_index_03.size(), "", 0, 0); +} + +TEST(ParserIndex, ConstraintIndex) +{ + ParserDeclareIndex p_index; + + String primary_key_01 = "PRIMARY KEY (col_01) INVISIBLE"; + parseQuery(p_index, primary_key_01.data(), primary_key_01.data() + primary_key_01.size(), "", 0, 0); + + String primary_key_02 = "PRIMARY KEY USING BTREE (col_01) INVISIBLE"; + parseQuery(p_index, primary_key_02.data(), primary_key_02.data() + primary_key_02.size(), "", 0, 0); + + String primary_key_03 = "CONSTRAINT PRIMARY KEY USING BTREE (col_01) INVISIBLE"; + parseQuery(p_index, primary_key_03.data(), primary_key_03.data() + primary_key_03.size(), "", 0, 0); + + String primary_key_04 = "CONSTRAINT index_name PRIMARY KEY USING BTREE (col_01) INVISIBLE"; + parseQuery(p_index, primary_key_04.data(), primary_key_04.data() + primary_key_04.size(), "", 0, 0); + + String unique_key_01 = "UNIQUE (col_01) INVISIBLE"; + parseQuery(p_index, unique_key_01.data(), unique_key_01.data() + unique_key_01.size(), "", 0, 0); + + String unique_key_02 = "UNIQUE INDEX (col_01) INVISIBLE"; + parseQuery(p_index, unique_key_02.data(), unique_key_02.data() + unique_key_02.size(), "", 0, 0); + + String unique_key_03 = "UNIQUE KEY (col_01) INVISIBLE"; + parseQuery(p_index, unique_key_03.data(), unique_key_03.data() + unique_key_03.size(), "", 0, 0); + + String unique_key_04 = "UNIQUE KEY index_name (col_01) INVISIBLE"; + parseQuery(p_index, unique_key_04.data(), unique_key_04.data() + unique_key_04.size(), "", 0, 0); + + String unique_key_05 = "UNIQUE KEY index_name USING HASH (col_01) INVISIBLE"; + parseQuery(p_index, unique_key_05.data(), unique_key_05.data() + unique_key_05.size(), "", 0, 0); + + String unique_key_06 = "CONSTRAINT UNIQUE KEY index_name USING HASH (col_01) INVISIBLE"; + parseQuery(p_index, unique_key_06.data(), unique_key_06.data() + unique_key_06.size(), "", 0, 0); + + String unique_key_07 = "CONSTRAINT index_name UNIQUE KEY index_name_1 USING HASH (col_01) INVISIBLE"; + parseQuery(p_index, unique_key_07.data(), unique_key_07.data() + unique_key_07.size(), "", 0, 0); + + String foreign_key_01 = "FOREIGN KEY (col_01) REFERENCES tbl_name (col_01)"; + parseQuery(p_index, foreign_key_01.data(), foreign_key_01.data() + foreign_key_01.size(), "", 0, 0); + + String foreign_key_02 = "FOREIGN KEY index_name (col_01) REFERENCES tbl_name (col_01)"; + parseQuery(p_index, foreign_key_02.data(), foreign_key_02.data() + foreign_key_02.size(), "", 0, 0); + + String foreign_key_03 = "CONSTRAINT FOREIGN KEY index_name (col_01) REFERENCES tbl_name (col_01)"; + parseQuery(p_index, foreign_key_03.data(), foreign_key_03.data() + foreign_key_03.size(), "", 0, 0); + + String foreign_key_04 = "CONSTRAINT index_name FOREIGN KEY index_name_01 (col_01) REFERENCES tbl_name (col_01)"; + parseQuery(p_index, foreign_key_04.data(), foreign_key_04.data() + foreign_key_04.size(), "", 0, 0); +} diff --git a/src/Parsers/MySQL/tests/gtest_partition_options_parser.cpp b/src/Parsers/MySQL/tests/gtest_partition_options_parser.cpp new file mode 100644 index 00000000000..1651efcb966 --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_partition_options_parser.cpp @@ -0,0 +1,167 @@ +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserPartitionOptions, HashPatitionOptions) +{ + String hash_partition = "PARTITION BY HASH(col_01)"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast_01 = parseQuery(p_partition_options, hash_partition.data(), hash_partition.data() + hash_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_01 = ast_01->as(); + EXPECT_EQ(declare_partition_options_01->partition_type, "hash"); + EXPECT_EQ(declare_partition_options_01->partition_expression->as()->name, "col_01"); + + String linear_hash_partition = "PARTITION BY LINEAR HASH(col_01)"; + ASTPtr ast_02 = parseQuery(p_partition_options, linear_hash_partition.data(), linear_hash_partition.data() + linear_hash_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_02 = ast_02->as(); + EXPECT_EQ(declare_partition_options_02->partition_type, "linear_hash"); + EXPECT_EQ(declare_partition_options_02->partition_expression->as()->name, "col_01"); +} + +TEST(ParserPartitionOptions, KeyPatitionOptions) +{ + String key_partition = "PARTITION BY KEY(col_01)"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast_01 = parseQuery(p_partition_options, key_partition.data(), key_partition.data() + key_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_01 = ast_01->as(); + EXPECT_EQ(declare_partition_options_01->partition_type, "key"); + EXPECT_EQ(declare_partition_options_01->partition_expression->as()->name, "col_01"); + + String linear_key_partition = "PARTITION BY LINEAR KEY(col_01, col_02)"; + ASTPtr ast_02 = parseQuery(p_partition_options, linear_key_partition.data(), linear_key_partition.data() + linear_key_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_02 = ast_02->as(); + EXPECT_EQ(declare_partition_options_02->partition_type, "linear_key"); + ASTPtr columns_list = declare_partition_options_02->partition_expression->as()->arguments; + EXPECT_EQ(columns_list->children[0]->as()->name, "col_01"); + EXPECT_EQ(columns_list->children[1]->as()->name, "col_02"); + + String key_partition_with_algorithm = "PARTITION BY KEY ALGORITHM=1 (col_01)"; + ASTPtr ast_03 = parseQuery(p_partition_options, key_partition_with_algorithm.data(), key_partition_with_algorithm.data() + key_partition_with_algorithm.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_03 = ast_03->as(); + EXPECT_EQ(declare_partition_options_03->partition_type, "key_1"); + EXPECT_EQ(declare_partition_options_03->partition_expression->as()->name, "col_01"); +} + +TEST(ParserPartitionOptions, RangePatitionOptions) +{ + String range_partition = "PARTITION BY RANGE(col_01)"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast_01 = parseQuery(p_partition_options, range_partition.data(), range_partition.data() + range_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_01 = ast_01->as(); + EXPECT_EQ(declare_partition_options_01->partition_type, "range"); + EXPECT_EQ(declare_partition_options_01->partition_expression->as()->name, "col_01"); + + String range_columns_partition = "PARTITION BY RANGE COLUMNS(col_01, col_02)"; + ASTPtr ast_02 = parseQuery(p_partition_options, range_columns_partition.data(), range_columns_partition.data() + range_columns_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_02 = ast_02->as(); + EXPECT_EQ(declare_partition_options_02->partition_type, "range"); + ASTPtr columns_list = declare_partition_options_02->partition_expression->as()->arguments; + EXPECT_EQ(columns_list->children[0]->as()->name, "col_01"); + EXPECT_EQ(columns_list->children[1]->as()->name, "col_02"); +} + +TEST(ParserPartitionOptions, ListPatitionOptions) +{ + String range_partition = "PARTITION BY LIST(col_01)"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast_01 = parseQuery(p_partition_options, range_partition.data(), range_partition.data() + range_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_01 = ast_01->as(); + EXPECT_EQ(declare_partition_options_01->partition_type, "list"); + EXPECT_EQ(declare_partition_options_01->partition_expression->as()->name, "col_01"); + + String range_columns_partition = "PARTITION BY LIST COLUMNS(col_01, col_02)"; + ASTPtr ast_02 = parseQuery(p_partition_options, range_columns_partition.data(), range_columns_partition.data() + range_columns_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options_02 = ast_02->as(); + EXPECT_EQ(declare_partition_options_02->partition_type, "list"); + ASTPtr columns_list = declare_partition_options_02->partition_expression->as()->arguments; + EXPECT_EQ(columns_list->children[0]->as()->name, "col_01"); + EXPECT_EQ(columns_list->children[1]->as()->name, "col_02"); +} + +TEST(ParserPartitionOptions, PatitionNumberOptions) +{ + String numbers_partition = "PARTITION BY KEY(col_01) PARTITIONS 2"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast = parseQuery(p_partition_options, numbers_partition.data(), numbers_partition.data() + numbers_partition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options = ast->as(); + EXPECT_EQ(declare_partition_options->partition_type, "key"); + EXPECT_EQ(declare_partition_options->partition_expression->as()->name, "col_01"); + EXPECT_EQ(declare_partition_options->partition_numbers->as()->value.safeGet(), 2); +} + +TEST(ParserPartitionOptions, PatitionWithSubpartitionOptions) +{ + String partition_with_subpartition = "PARTITION BY KEY(col_01) PARTITIONS 3 SUBPARTITION BY HASH(col_02) SUBPARTITIONS 4"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast = parseQuery(p_partition_options, partition_with_subpartition.data(), partition_with_subpartition.data() + partition_with_subpartition.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options = ast->as(); + EXPECT_EQ(declare_partition_options->partition_type, "key"); + EXPECT_EQ(declare_partition_options->partition_expression->as()->name, "col_01"); + EXPECT_EQ(declare_partition_options->partition_numbers->as()->value.safeGet(), 3); + EXPECT_EQ(declare_partition_options->subpartition_type, "hash"); + EXPECT_EQ(declare_partition_options->subpartition_expression->as()->name, "col_02"); + EXPECT_EQ(declare_partition_options->subpartition_numbers->as()->value.safeGet(), 4); +} + +TEST(ParserPartitionOptions, PatitionOptionsWithDeclarePartition) +{ + String partition_options_with_declare = "PARTITION BY KEY(col_01) PARTITIONS 3 SUBPARTITION BY HASH(col_02) SUBPARTITIONS 4 (PARTITION partition_name)"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast = parseQuery(p_partition_options, + partition_options_with_declare.data(), + partition_options_with_declare.data() + partition_options_with_declare.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options = ast->as(); + EXPECT_EQ(declare_partition_options->partition_type, "key"); + EXPECT_EQ(declare_partition_options->partition_expression->as()->name, "col_01"); + EXPECT_EQ(declare_partition_options->partition_numbers->as()->value.safeGet(), 3); + EXPECT_EQ(declare_partition_options->subpartition_type, "hash"); + EXPECT_EQ(declare_partition_options->subpartition_expression->as()->name, "col_02"); + EXPECT_EQ(declare_partition_options->subpartition_numbers->as()->value.safeGet(), 4); + EXPECT_TRUE(declare_partition_options->declare_partitions->as()->children[0]->as()); +} + +TEST(ParserPartitionOptions, PatitionOptionsWithDeclarePartitions) +{ + String partition_options_with_declare = "PARTITION BY KEY(col_01) PARTITIONS 3 SUBPARTITION BY HASH(col_02) SUBPARTITIONS 4 (PARTITION partition_01, PARTITION partition_02)"; + + ParserDeclarePartitionOptions p_partition_options; + ASTPtr ast = parseQuery(p_partition_options, + partition_options_with_declare.data(), + partition_options_with_declare.data() + partition_options_with_declare.size(), "", 0, 0); + + ASTDeclarePartitionOptions * declare_partition_options = ast->as(); + EXPECT_EQ(declare_partition_options->partition_type, "key"); + EXPECT_EQ(declare_partition_options->partition_expression->as()->name, "col_01"); + EXPECT_EQ(declare_partition_options->partition_numbers->as()->value.safeGet(), 3); + EXPECT_EQ(declare_partition_options->subpartition_type, "hash"); + EXPECT_EQ(declare_partition_options->subpartition_expression->as()->name, "col_02"); + EXPECT_EQ(declare_partition_options->subpartition_numbers->as()->value.safeGet(), 4); + EXPECT_TRUE(declare_partition_options->declare_partitions->as()->children[0]->as()); + EXPECT_TRUE(declare_partition_options->declare_partitions->as()->children[1]->as()); +} diff --git a/src/Parsers/MySQL/tests/gtest_partition_parser.cpp b/src/Parsers/MySQL/tests/gtest_partition_parser.cpp new file mode 100644 index 00000000000..48e8a9f53c6 --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_partition_parser.cpp @@ -0,0 +1,149 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserPartition, AllPatitionOptions) +{ + String input = "PARTITION partition_name ENGINE = engine_name COMMENT 'partition comment'" + " INDEX DIRECTORY 'index_directory' DATA DIRECTORY 'data_directory' max_rows 1000 MIN_ROWs 0" + " TABLESPACE table_space_name"; + + ParserDeclarePartition p_partition; + ASTPtr ast = parseQuery(p_partition, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition = ast->as(); + EXPECT_EQ(declare_partition->partition_name, "partition_name"); + ASTDeclareOptions * declare_options = declare_partition->options->as(); + EXPECT_EQ(declare_options->changes["engine"]->as()->name, "engine_name"); + EXPECT_EQ(declare_options->changes["comment"]->as()->value.safeGet(), "partition comment"); + EXPECT_EQ(declare_options->changes["data_directory"]->as()->value.safeGet(), "data_directory"); + EXPECT_EQ(declare_options->changes["index_directory"]->as()->value.safeGet(), "index_directory"); + EXPECT_EQ(declare_options->changes["min_rows"]->as()->value.safeGet(), 0); + EXPECT_EQ(declare_options->changes["max_rows"]->as()->value.safeGet(), 1000); + EXPECT_EQ(declare_options->changes["tablespace"]->as()->name, "table_space_name"); +} + +TEST(ParserPartition, OptionalPatitionOptions) +{ + String input = "PARTITION partition_name STORAGE engine = engine_name max_rows 1000 min_rows 0 tablespace table_space_name"; + ParserDeclarePartition p_partition; + ASTPtr ast = parseQuery(p_partition, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition = ast->as(); + EXPECT_EQ(declare_partition->partition_name, "partition_name"); + ASTDeclareOptions * declare_options = declare_partition->options->as(); + EXPECT_EQ(declare_options->changes["engine"]->as()->name, "engine_name"); + EXPECT_EQ(declare_options->changes["min_rows"]->as()->value.safeGet(), 0); + EXPECT_EQ(declare_options->changes["max_rows"]->as()->value.safeGet(), 1000); + EXPECT_EQ(declare_options->changes["tablespace"]->as()->name, "table_space_name"); +} + +TEST(ParserPartition, PatitionOptionsWithLessThan) +{ + ParserDeclarePartition p_partition; + String partition_01 = "PARTITION partition_01 VALUES LESS THAN (1991) STORAGE engine = engine_name"; + ASTPtr ast_partition_01 = parseQuery(p_partition, partition_01.data(), partition_01.data() + partition_01.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_01 = ast_partition_01->as(); + EXPECT_EQ(declare_partition_01->partition_name, "partition_01"); + EXPECT_EQ(declare_partition_01->less_than->as()->value.safeGet(), 1991); + ASTDeclareOptions * declare_options_01 = declare_partition_01->options->as(); + EXPECT_EQ(declare_options_01->changes["engine"]->as()->name, "engine_name"); + + String partition_02 = "PARTITION partition_02 VALUES LESS THAN MAXVALUE STORAGE engine = engine_name"; + ASTPtr ast_partition_02 = parseQuery(p_partition, partition_02.data(), partition_02.data() + partition_02.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_02 = ast_partition_02->as(); + EXPECT_EQ(declare_partition_02->partition_name, "partition_02"); + EXPECT_EQ(declare_partition_02->less_than->as()->name, "MAXVALUE"); + ASTDeclareOptions * declare_options_02 = declare_partition_02->options->as(); + EXPECT_EQ(declare_options_02->changes["engine"]->as()->name, "engine_name"); + + String partition_03 = "PARTITION partition_03 VALUES LESS THAN (50, MAXVALUE) STORAGE engine = engine_name"; + ASTPtr ast_partition_03 = parseQuery(p_partition, partition_03.data(), partition_03.data() + partition_03.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_03 = ast_partition_03->as(); + EXPECT_EQ(declare_partition_03->partition_name, "partition_03"); + ASTPtr declare_partition_03_argument = declare_partition_03->less_than->as()->arguments; + EXPECT_EQ(declare_partition_03_argument->children[0]->as()->value.safeGet(), 50); + EXPECT_EQ(declare_partition_03_argument->children[1]->as()->name, "MAXVALUE"); + ASTDeclareOptions * declare_options_03 = declare_partition_03->options->as(); + EXPECT_EQ(declare_options_03->changes["engine"]->as()->name, "engine_name"); + + String partition_04 = "PARTITION partition_04 VALUES LESS THAN (MAXVALUE, MAXVALUE) STORAGE engine = engine_name"; + ASTPtr ast_partition_04 = parseQuery(p_partition, partition_04.data(), partition_04.data() + partition_04.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_04 = ast_partition_04->as(); + EXPECT_EQ(declare_partition_04->partition_name, "partition_04"); + ASTPtr declare_partition_04_argument = declare_partition_04->less_than->as()->arguments; + EXPECT_EQ(declare_partition_04_argument->children[0]->as()->name, "MAXVALUE"); + EXPECT_EQ(declare_partition_04_argument->children[1]->as()->name, "MAXVALUE"); + ASTDeclareOptions * declare_options_04 = declare_partition_04->options->as(); + EXPECT_EQ(declare_options_04->changes["engine"]->as()->name, "engine_name"); +} + +TEST(ParserPartition, PatitionOptionsWithInExpression) +{ + ParserDeclarePartition p_partition; + String partition_01 = "PARTITION partition_01 VALUES IN (NULL, 1991, MAXVALUE) STORAGE engine = engine_name"; + ASTPtr ast_partition_01 = parseQuery(p_partition, partition_01.data(), partition_01.data() + partition_01.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_01 = ast_partition_01->as(); + EXPECT_EQ(declare_partition_01->partition_name, "partition_01"); + ASTPtr declare_partition_01_argument = declare_partition_01->in_expression->as()->arguments; + EXPECT_TRUE(declare_partition_01_argument->children[0]->as()->value.isNull()); + EXPECT_EQ(declare_partition_01_argument->children[1]->as()->value.safeGet(), 1991); + EXPECT_EQ(declare_partition_01_argument->children[2]->as()->name, "MAXVALUE"); + ASTDeclareOptions * declare_options_01 = declare_partition_01->options->as(); + EXPECT_EQ(declare_options_01->changes["engine"]->as()->name, "engine_name"); + + String partition_02 = "PARTITION partition_02 VALUES IN ((NULL, 1991), (1991, NULL), (MAXVALUE, MAXVALUE)) STORAGE engine = engine_name"; + ASTPtr ast_partition_02 = parseQuery(p_partition, partition_02.data(), partition_02.data() + partition_02.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_02 = ast_partition_02->as(); + EXPECT_EQ(declare_partition_02->partition_name, "partition_02"); + ASTPtr declare_partition_02_argument = declare_partition_02->in_expression->as()->arguments; + + ASTPtr argument_01 = declare_partition_02_argument->children[0]; + EXPECT_TRUE(argument_01->as()->value.safeGet()[0].isNull()); + EXPECT_EQ(argument_01->as()->value.safeGet()[1].safeGet(), 1991); + + ASTPtr argument_02 = declare_partition_02_argument->children[1]; + EXPECT_EQ(argument_02->as()->value.safeGet()[0].safeGet(), 1991); + EXPECT_TRUE(argument_02->as()->value.safeGet()[1].isNull()); + + ASTPtr argument_03 = declare_partition_02_argument->children[2]->as()->arguments; + EXPECT_EQ(argument_03->as()->children[0]->as()->name, "MAXVALUE"); + EXPECT_EQ(argument_03->as()->children[1]->as()->name, "MAXVALUE"); + + ASTDeclareOptions * declare_options_02 = declare_partition_02->options->as(); + EXPECT_EQ(declare_options_02->changes["engine"]->as()->name, "engine_name"); +} + +TEST(ParserPartition, PatitionOptionsWithSubpartitions) +{ + ParserDeclarePartition p_partition; + String partition_01 = "PARTITION partition_01 VALUES IN (NULL, 1991, MAXVALUE) STORAGE engine = engine_name (SUBPARTITION s_p01)"; + ASTPtr ast_partition_01 = parseQuery(p_partition, partition_01.data(), partition_01.data() + partition_01.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_01 = ast_partition_01->as(); + EXPECT_EQ(declare_partition_01->partition_name, "partition_01"); + EXPECT_TRUE(declare_partition_01->subpartitions->as()->children[0]->as()); + + String partition_02 = "PARTITION partition_02 VALUES IN (NULL, 1991, MAXVALUE) STORAGE engine = engine_name (SUBPARTITION s_p01, SUBPARTITION s_p02)"; + ASTPtr ast_partition_02 = parseQuery(p_partition, partition_02.data(), partition_02.data() + partition_02.size(), "", 0, 0); + + ASTDeclarePartition * declare_partition_02 = ast_partition_02->as(); + EXPECT_EQ(declare_partition_02->partition_name, "partition_02"); + EXPECT_TRUE(declare_partition_02->subpartitions->as()->children[0]->as()); + EXPECT_TRUE(declare_partition_02->subpartitions->as()->children[1]->as()); +} + diff --git a/src/Parsers/MySQL/tests/gtest_reference_parser.cpp b/src/Parsers/MySQL/tests/gtest_reference_parser.cpp new file mode 100644 index 00000000000..694558b9cc3 --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_reference_parser.cpp @@ -0,0 +1,92 @@ +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserReference, SimpleReference) +{ + ParserDeclareReference p_reference; + + String reference_01 = "REFERENCES table_name (ref_col_01)"; + ASTPtr ast_reference_01 = parseQuery(p_reference, reference_01.data(), reference_01.data() + reference_01.size(), "", 0, 0); + EXPECT_EQ(ast_reference_01->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_01->as()->reference_expression->as()->name, "ref_col_01"); + + String reference_02 = "REFERENCES table_name (ref_col_01, ref_col_02)"; + ASTPtr ast_reference_02 = parseQuery(p_reference, reference_02.data(), reference_02.data() + reference_02.size(), "", 0, 0); + EXPECT_EQ(ast_reference_02->as()->reference_table_name, "table_name"); + ASTPtr arguments = ast_reference_02->as()->reference_expression->as()->arguments; + EXPECT_EQ(arguments->children[0]->as()->name, "ref_col_01"); + EXPECT_EQ(arguments->children[1]->as()->name, "ref_col_02"); +} + +TEST(ParserReference, ReferenceDifferenceKind) +{ + ParserDeclareReference p_reference; + String reference_01 = "REFERENCES table_name (ref_col_01) MATCH FULL"; + ASTPtr ast_reference_01 = parseQuery(p_reference, reference_01.data(), reference_01.data() + reference_01.size(), "", 0, 0); + EXPECT_EQ(ast_reference_01->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_01->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_01->as()->kind, ASTDeclareReference::MATCH_FULL); + + String reference_02 = "REFERENCES table_name (ref_col_01) MATCH PARTIAL"; + ASTPtr ast_reference_02 = parseQuery(p_reference, reference_02.data(), reference_02.data() + reference_02.size(), "", 0, 0); + EXPECT_EQ(ast_reference_02->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_02->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_02->as()->kind, ASTDeclareReference::MATCH_PARTIAL); + + String reference_03 = "REFERENCES table_name (ref_col_01) MATCH SIMPLE"; + ASTPtr ast_reference_03 = parseQuery(p_reference, reference_03.data(), reference_03.data() + reference_03.size(), "", 0, 0); + EXPECT_EQ(ast_reference_03->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_03->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_03->as()->kind, ASTDeclareReference::MATCH_SIMPLE); +} + +TEST(ParserReference, ReferenceDifferenceOption) +{ + ParserDeclareReference p_reference; + String reference_01 = "REFERENCES table_name (ref_col_01) MATCH FULL ON DELETE RESTRICT ON UPDATE RESTRICT"; + ASTPtr ast_reference_01 = parseQuery(p_reference, reference_01.data(), reference_01.data() + reference_01.size(), "", 0, 0); + EXPECT_EQ(ast_reference_01->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_01->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_01->as()->kind, ASTDeclareReference::MATCH_FULL); + EXPECT_EQ(ast_reference_01->as()->on_delete_option, ASTDeclareReference::RESTRICT); + EXPECT_EQ(ast_reference_01->as()->on_update_option, ASTDeclareReference::RESTRICT); + + String reference_02 = "REFERENCES table_name (ref_col_01) MATCH FULL ON DELETE CASCADE ON UPDATE CASCADE"; + ASTPtr ast_reference_02 = parseQuery(p_reference, reference_02.data(), reference_02.data() + reference_02.size(), "", 0, 0); + EXPECT_EQ(ast_reference_02->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_02->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_02->as()->kind, ASTDeclareReference::MATCH_FULL); + EXPECT_EQ(ast_reference_02->as()->on_delete_option, ASTDeclareReference::CASCADE); + EXPECT_EQ(ast_reference_02->as()->on_update_option, ASTDeclareReference::CASCADE); + + String reference_03 = "REFERENCES table_name (ref_col_01) MATCH FULL ON DELETE SET NULL ON UPDATE SET NULL"; + ASTPtr ast_reference_03 = parseQuery(p_reference, reference_03.data(), reference_03.data() + reference_03.size(), "", 0, 0); + EXPECT_EQ(ast_reference_03->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_03->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_03->as()->kind, ASTDeclareReference::MATCH_FULL); + EXPECT_EQ(ast_reference_03->as()->on_delete_option, ASTDeclareReference::SET_NULL); + EXPECT_EQ(ast_reference_03->as()->on_update_option, ASTDeclareReference::SET_NULL); + + String reference_04 = "REFERENCES table_name (ref_col_01) MATCH FULL ON UPDATE NO ACTION ON DELETE NO ACTION"; + ASTPtr ast_reference_04 = parseQuery(p_reference, reference_04.data(), reference_04.data() + reference_04.size(), "", 0, 0); + EXPECT_EQ(ast_reference_04->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_04->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_04->as()->kind, ASTDeclareReference::MATCH_FULL); + EXPECT_EQ(ast_reference_04->as()->on_delete_option, ASTDeclareReference::NO_ACTION); + EXPECT_EQ(ast_reference_04->as()->on_update_option, ASTDeclareReference::NO_ACTION); + + String reference_05 = "REFERENCES table_name (ref_col_01) MATCH FULL ON UPDATE SET DEFAULT ON DELETE SET DEFAULT"; + ASTPtr ast_reference_05 = parseQuery(p_reference, reference_05.data(), reference_05.data() + reference_05.size(), "", 0, 0); + EXPECT_EQ(ast_reference_05->as()->reference_table_name, "table_name"); + EXPECT_EQ(ast_reference_05->as()->reference_expression->as()->name, "ref_col_01"); + EXPECT_EQ(ast_reference_05->as()->kind, ASTDeclareReference::MATCH_FULL); + EXPECT_EQ(ast_reference_05->as()->on_delete_option, ASTDeclareReference::SET_DEFAULT); + EXPECT_EQ(ast_reference_05->as()->on_update_option, ASTDeclareReference::SET_DEFAULT); +} + diff --git a/src/Parsers/MySQL/tests/gtest_subpartition_parser.cpp b/src/Parsers/MySQL/tests/gtest_subpartition_parser.cpp new file mode 100644 index 00000000000..5c1cf3710ab --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_subpartition_parser.cpp @@ -0,0 +1,45 @@ +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserSubpartition, AllSubpatitionOptions) +{ + String input = "SUBPARTITION subpartition_name ENGINE = engine_name COMMENT 'subpartition comment'" + " DATA DIRECTORY 'data_directory' INDEX DIRECTORY 'index_directory' max_rows 1000 MIN_ROWs 0" + " TABLESPACE table_space_name"; + MySQLParser::ParserDeclareSubPartition p_subpartition; + ASTPtr ast = parseQuery(p_subpartition, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclareSubPartition * declare_subpartition = ast->as(); + EXPECT_EQ(declare_subpartition->logical_name, "subpartition_name"); + ASTDeclareOptions * declare_options = declare_subpartition->options->as(); + EXPECT_EQ(declare_options->changes["engine"]->as()->name, "engine_name"); + EXPECT_EQ(declare_options->changes["comment"]->as()->value.safeGet(), "subpartition comment"); + EXPECT_EQ(declare_options->changes["data_directory"]->as()->value.safeGet(), "data_directory"); + EXPECT_EQ(declare_options->changes["index_directory"]->as()->value.safeGet(), "index_directory"); + EXPECT_EQ(declare_options->changes["min_rows"]->as()->value.safeGet(), 0); + EXPECT_EQ(declare_options->changes["max_rows"]->as()->value.safeGet(), 1000); + EXPECT_EQ(declare_options->changes["tablespace"]->as()->name, "table_space_name"); +} + +TEST(ParserSubpartition, OptionalSubpatitionOptions) +{ + String input = "SUBPARTITION subpartition_name STORAGE engine = engine_name max_rows 1000 min_rows 0 tablespace table_space_name"; + MySQLParser::ParserDeclareSubPartition p_subpartition; + ASTPtr ast = parseQuery(p_subpartition, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclareSubPartition * declare_subpartition = ast->as(); + EXPECT_EQ(declare_subpartition->logical_name, "subpartition_name"); + ASTDeclareOptions * declare_options = declare_subpartition->options->as(); + EXPECT_EQ(declare_options->changes["engine"]->as()->name, "engine_name"); + EXPECT_EQ(declare_options->changes["min_rows"]->as()->value.safeGet(), 0); + EXPECT_EQ(declare_options->changes["max_rows"]->as()->value.safeGet(), 1000); + EXPECT_EQ(declare_options->changes["tablespace"]->as()->name, "table_space_name"); +} + diff --git a/src/Parsers/MySQL/tests/gtest_table_options_parser.cpp b/src/Parsers/MySQL/tests/gtest_table_options_parser.cpp new file mode 100644 index 00000000000..b051f6149bb --- /dev/null +++ b/src/Parsers/MySQL/tests/gtest_table_options_parser.cpp @@ -0,0 +1,64 @@ +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; +using namespace DB::MySQLParser; + +TEST(ParserTableOptions, AllSubpatitionOptions) +{ + String input = "AUTO_INCREMENt = 1 AVG_ROW_LENGTh 3 CHARACTER SET utf-8 CHECKSUM 1 COLLATE utf8_bin" + " COMMENT 'table option comment' COMPRESSION 'LZ4' CONNECTION 'connect_string' DATA DIRECTORY 'data_directory'" + " INDEX DIRECTORY 'index_directory' DELAY_KEY_WRITE 0 ENCRYPTION 'Y' ENGINE INNODB INSERT_METHOD NO KEY_BLOCK_SIZE 3" + " MAX_ROWS 1000 MIN_ROWS 0 PACK_KEYS DEFAULT PASSWORD 'password' ROW_FORMAT DYNAMIC STATS_AUTO_RECALC DEFAULT " + " STATS_PERSISTENT DEFAULT STATS_SAMPLE_PAGES 3 TABLESPACE tablespace_name STORAGE MEMORY UNION (table_01, table_02)"; + + ParserDeclareTableOptions p_table_options; + ASTPtr ast = parseQuery(p_table_options, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclareOptions * declare_options = ast->as(); + EXPECT_EQ(declare_options->changes["auto_increment"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["avg_row_length"]->as()->value.safeGet(), 3); + EXPECT_EQ(declare_options->changes["character_set"]->as()->name, "utf-8"); + EXPECT_EQ(declare_options->changes["checksum"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["collate"]->as()->name, "utf8_bin"); + EXPECT_EQ(declare_options->changes["comment"]->as()->value.safeGet(), "table option comment"); + EXPECT_EQ(declare_options->changes["compression"]->as()->value.safeGet(), "LZ4"); + EXPECT_EQ(declare_options->changes["connection"]->as()->value.safeGet(), "connect_string"); + EXPECT_EQ(declare_options->changes["data_directory"]->as()->value.safeGet(), "data_directory"); + EXPECT_EQ(declare_options->changes["index_directory"]->as()->value.safeGet(), "index_directory"); + EXPECT_EQ(declare_options->changes["delay_key_write"]->as()->value.safeGet(), 0); + EXPECT_EQ(declare_options->changes["encryption"]->as()->value.safeGet(), "Y"); + EXPECT_EQ(declare_options->changes["engine"]->as()->name, "INNODB"); + EXPECT_EQ(declare_options->changes["insert_method"]->as()->name, "NO"); + EXPECT_EQ(declare_options->changes["key_block_size"]->as()->value.safeGet(), 3); + + EXPECT_EQ(declare_options->changes["max_rows"]->as()->value.safeGet(), 1000); + EXPECT_EQ(declare_options->changes["min_rows"]->as()->value.safeGet(), 0); + EXPECT_EQ(declare_options->changes["pack_keys"]->as()->name, "DEFAULT"); + EXPECT_EQ(declare_options->changes["password"]->as()->value.safeGet(), "password"); + EXPECT_EQ(declare_options->changes["row_format"]->as()->name, "DYNAMIC"); + EXPECT_EQ(declare_options->changes["stats_auto_recalc"]->as()->name, "DEFAULT"); + EXPECT_EQ(declare_options->changes["stats_persistent"]->as()->name, "DEFAULT"); + EXPECT_EQ(declare_options->changes["stats_sample_pages"]->as()->value.safeGet(), 3); + EXPECT_EQ(declare_options->changes["tablespace"]->as()->name, "tablespace_name"); + + ASTPtr arguments = declare_options->changes["union"]->as()->arguments; + EXPECT_EQ(arguments->children[0]->as()->name, "table_01"); + EXPECT_EQ(arguments->children[1]->as()->name, "table_02"); +} + +TEST(ParserTableOptions, OptionalTableOptions) +{ + String input = "STATS_AUTO_RECALC DEFAULT AUTO_INCREMENt = 1 "; + ParserDeclareTableOptions p_table_options; + ASTPtr ast = parseQuery(p_table_options, input.data(), input.data() + input.size(), "", 0, 0); + + ASTDeclareOptions * declare_options = ast->as(); + EXPECT_EQ(declare_options->changes["auto_increment"]->as()->value.safeGet(), 1); + EXPECT_EQ(declare_options->changes["stats_auto_recalc"]->as()->name, "DEFAULT"); +}