mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
Merge pull request #35166 from zhangjmruc/sql_compatible_create_drop_index
Support SQL compatible create/drop index syntax
This commit is contained in:
commit
23e09b5e9b
77
src/Interpreters/InterpreterCreateIndexQuery.cpp
Normal file
77
src/Interpreters/InterpreterCreateIndexQuery.cpp
Normal file
@ -0,0 +1,77 @@
|
||||
#include <Interpreters/InterpreterCreateIndexQuery.h>
|
||||
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Databases/DatabaseReplicated.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <Parsers/ASTCreateIndexQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TABLE_IS_READ_ONLY;
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterCreateIndexQuery::execute()
|
||||
{
|
||||
const auto & create_index = query_ptr->as<ASTCreateIndexQuery &>();
|
||||
|
||||
AccessRightsElements required_access;
|
||||
required_access.emplace_back(AccessType::ALTER_ADD_INDEX, create_index.getDatabase(), create_index.getTable());
|
||||
|
||||
if (!create_index.cluster.empty())
|
||||
{
|
||||
DDLQueryOnClusterParams params;
|
||||
params.access_to_check = std::move(required_access);
|
||||
return executeDDLQueryOnCluster(query_ptr, getContext(), params);
|
||||
}
|
||||
|
||||
getContext()->checkAccess(required_access);
|
||||
auto table_id = getContext()->resolveStorageID(create_index, Context::ResolveOrdinary);
|
||||
query_ptr->as<ASTCreateIndexQuery &>().setDatabase(table_id.database_name);
|
||||
|
||||
DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name);
|
||||
if (typeid_cast<DatabaseReplicated *>(database.get())
|
||||
&& !getContext()->getClientInfo().is_replicated_database_internal)
|
||||
{
|
||||
auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name);
|
||||
guard->releaseTableLock();
|
||||
return typeid_cast<DatabaseReplicated *>(database.get())->tryEnqueueReplicatedDDL(query_ptr, getContext());
|
||||
}
|
||||
|
||||
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext());
|
||||
if (table->isStaticStorage())
|
||||
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only");
|
||||
|
||||
/// Convert ASTCreateIndexQuery to AlterCommand.
|
||||
AlterCommands alter_commands;
|
||||
|
||||
AlterCommand command;
|
||||
command.index_decl = create_index.index_decl;
|
||||
command.type = AlterCommand::ADD_INDEX;
|
||||
command.index_name = create_index.index_name->as<ASTIdentifier &>().name();
|
||||
command.if_not_exists = create_index.if_not_exists;
|
||||
|
||||
/// Fill name in ASTIndexDeclaration
|
||||
auto & ast_index_decl = command.index_decl->as<ASTIndexDeclaration &>();
|
||||
ast_index_decl.name = command.index_name;
|
||||
|
||||
alter_commands.emplace_back(std::move(command));
|
||||
|
||||
auto alter_lock = table->lockForAlter(getContext()->getSettingsRef().lock_acquire_timeout);
|
||||
StorageInMemoryMetadata metadata = table->getInMemoryMetadata();
|
||||
alter_commands.validate(table, getContext());
|
||||
alter_commands.prepare(metadata);
|
||||
table->checkAlterIsPossible(alter_commands, getContext());
|
||||
table->alter(alter_commands, getContext(), alter_lock);
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
}
|
24
src/Interpreters/InterpreterCreateIndexQuery.h
Normal file
24
src/Interpreters/InterpreterCreateIndexQuery.h
Normal file
@ -0,0 +1,24 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
class InterpreterCreateIndexQuery : public IInterpreter, WithContext
|
||||
{
|
||||
public:
|
||||
InterpreterCreateIndexQuery(const ASTPtr & query_ptr_, ContextPtr context_)
|
||||
: WithContext(context_)
|
||||
, query_ptr(query_ptr_) {}
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
};
|
||||
|
||||
}
|
71
src/Interpreters/InterpreterDropIndexQuery.cpp
Normal file
71
src/Interpreters/InterpreterDropIndexQuery.cpp
Normal file
@ -0,0 +1,71 @@
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Databases/DatabaseReplicated.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/executeDDLQueryOnCluster.h>
|
||||
#include <Interpreters/InterpreterDropIndexQuery.h>
|
||||
#include <Parsers/ASTDropIndexQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Storages/AlterCommands.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TABLE_IS_READ_ONLY;
|
||||
}
|
||||
|
||||
|
||||
BlockIO InterpreterDropIndexQuery::execute()
|
||||
{
|
||||
const auto & drop_index = query_ptr->as<ASTDropIndexQuery &>();
|
||||
|
||||
AccessRightsElements required_access;
|
||||
required_access.emplace_back(AccessType::ALTER_DROP_INDEX, drop_index.getDatabase(), drop_index.getTable());
|
||||
|
||||
if (!drop_index.cluster.empty())
|
||||
{
|
||||
DDLQueryOnClusterParams params;
|
||||
params.access_to_check = std::move(required_access);
|
||||
return executeDDLQueryOnCluster(query_ptr, getContext(), params);
|
||||
}
|
||||
|
||||
getContext()->checkAccess(required_access);
|
||||
auto table_id = getContext()->resolveStorageID(drop_index, Context::ResolveOrdinary);
|
||||
query_ptr->as<ASTDropIndexQuery &>().setDatabase(table_id.database_name);
|
||||
|
||||
DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name);
|
||||
if (typeid_cast<DatabaseReplicated *>(database.get())
|
||||
&& !getContext()->getClientInfo().is_replicated_database_internal)
|
||||
{
|
||||
auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name);
|
||||
guard->releaseTableLock();
|
||||
return typeid_cast<DatabaseReplicated *>(database.get())->tryEnqueueReplicatedDDL(query_ptr, getContext());
|
||||
}
|
||||
|
||||
StoragePtr table = DatabaseCatalog::instance().getTable(table_id, getContext());
|
||||
if (table->isStaticStorage())
|
||||
throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only");
|
||||
|
||||
/// Convert ASTDropIndexQuery to AlterCommand.
|
||||
AlterCommands alter_commands;
|
||||
|
||||
AlterCommand command;
|
||||
command.ast = drop_index.convertToASTAlterCommand();
|
||||
command.type = AlterCommand::DROP_INDEX;
|
||||
command.index_name = drop_index.index_name->as<ASTIdentifier &>().name();
|
||||
command.if_exists = drop_index.if_exists;
|
||||
|
||||
alter_commands.emplace_back(std::move(command));
|
||||
|
||||
auto alter_lock = table->lockForAlter(getContext()->getSettingsRef().lock_acquire_timeout);
|
||||
StorageInMemoryMetadata metadata = table->getInMemoryMetadata();
|
||||
alter_commands.validate(table, getContext());
|
||||
alter_commands.prepare(metadata);
|
||||
table->checkAlterIsPossible(alter_commands, getContext());
|
||||
table->alter(alter_commands, getContext(), alter_lock);
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
}
|
24
src/Interpreters/InterpreterDropIndexQuery.h
Normal file
24
src/Interpreters/InterpreterDropIndexQuery.h
Normal file
@ -0,0 +1,24 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
class InterpreterDropIndexQuery : public IInterpreter, WithContext
|
||||
{
|
||||
public:
|
||||
InterpreterDropIndexQuery(const ASTPtr & query_ptr_, ContextPtr context_)
|
||||
: WithContext(context_)
|
||||
, query_ptr(query_ptr_) {}
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
};
|
||||
|
||||
}
|
@ -3,7 +3,9 @@
|
||||
#include <Parsers/ASTCheckQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTCreateFunctionQuery.h>
|
||||
#include <Parsers/ASTCreateIndexQuery.h>
|
||||
#include <Parsers/ASTDropFunctionQuery.h>
|
||||
#include <Parsers/ASTDropIndexQuery.h>
|
||||
#include <Parsers/ASTDropQuery.h>
|
||||
#include <Parsers/ASTExplainQuery.h>
|
||||
#include <Parsers/ASTInsertQuery.h>
|
||||
@ -42,10 +44,12 @@
|
||||
#include <Interpreters/InterpreterBackupQuery.h>
|
||||
#include <Interpreters/InterpreterCheckQuery.h>
|
||||
#include <Interpreters/InterpreterCreateFunctionQuery.h>
|
||||
#include <Interpreters/InterpreterCreateIndexQuery.h>
|
||||
#include <Interpreters/InterpreterCreateQuery.h>
|
||||
#include <Interpreters/InterpreterDescribeQuery.h>
|
||||
#include <Interpreters/InterpreterDescribeCacheQuery.h>
|
||||
#include <Interpreters/InterpreterDropFunctionQuery.h>
|
||||
#include <Interpreters/InterpreterDropIndexQuery.h>
|
||||
#include <Interpreters/InterpreterDropQuery.h>
|
||||
#include <Interpreters/InterpreterExistsQuery.h>
|
||||
#include <Interpreters/InterpreterExplainQuery.h>
|
||||
@ -298,6 +302,14 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, ContextMut
|
||||
{
|
||||
return std::make_unique<InterpreterDropFunctionQuery>(query, context);
|
||||
}
|
||||
else if (query->as<ASTCreateIndexQuery>())
|
||||
{
|
||||
return std::make_unique<InterpreterCreateIndexQuery>(query, context);
|
||||
}
|
||||
else if (query->as<ASTDropIndexQuery>())
|
||||
{
|
||||
return std::make_unique<InterpreterDropIndexQuery>(query, context);
|
||||
}
|
||||
else if (query->as<ASTBackupQuery>())
|
||||
{
|
||||
return std::make_unique<InterpreterBackupQuery>(query, context);
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
#include <iomanip>
|
||||
#include <IO/Operators.h>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
@ -556,6 +557,7 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState
|
||||
frame.need_parens = false;
|
||||
|
||||
std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' ');
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str;
|
||||
|
||||
switch (alter_object)
|
||||
|
61
src/Parsers/ASTCreateIndexQuery.cpp
Normal file
61
src/Parsers/ASTCreateIndexQuery.cpp
Normal file
@ -0,0 +1,61 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Parsers/ASTCreateIndexQuery.h>
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
String ASTCreateIndexQuery::getID(char delim) const
|
||||
{
|
||||
return "CreateIndexQuery" + (delim + getDatabase()) + delim + getTable();
|
||||
}
|
||||
|
||||
ASTPtr ASTCreateIndexQuery::clone() const
|
||||
{
|
||||
auto res = std::make_shared<ASTCreateIndexQuery>(*this);
|
||||
res->children.clear();
|
||||
|
||||
res->index_name = index_name->clone();
|
||||
res->children.push_back(res->index_name);
|
||||
|
||||
res->index_decl = index_decl->clone();
|
||||
res->children.push_back(res->index_decl);
|
||||
return res;
|
||||
}
|
||||
|
||||
void ASTCreateIndexQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
frame.need_parens = false;
|
||||
|
||||
std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' ');
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str;
|
||||
|
||||
settings.ostr << "CREATE INDEX " << (if_not_exists ? "IF NOT EXISTS " : "");
|
||||
index_name->formatImpl(settings, state, frame);
|
||||
settings.ostr << " ON ";
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_none : "");
|
||||
|
||||
if (table)
|
||||
{
|
||||
if (database)
|
||||
{
|
||||
settings.ostr << indent_str << backQuoteIfNeed(getDatabase());
|
||||
settings.ostr << ".";
|
||||
}
|
||||
settings.ostr << indent_str << backQuoteIfNeed(getTable());
|
||||
}
|
||||
|
||||
formatOnCluster(settings);
|
||||
|
||||
if (!cluster.empty())
|
||||
settings.ostr << " ";
|
||||
|
||||
index_decl->formatImpl(settings, state, frame);
|
||||
}
|
||||
|
||||
}
|
39
src/Parsers/ASTCreateIndexQuery.h
Normal file
39
src/Parsers/ASTCreateIndexQuery.h
Normal file
@ -0,0 +1,39 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/ASTQueryWithOnCluster.h>
|
||||
#include <Parsers/ASTQueryWithTableAndOutput.h>
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** CREATE INDEX [IF NOT EXISTS] name ON [db].name (expression) TYPE type GRANULARITY value
|
||||
*/
|
||||
|
||||
class ASTCreateIndexQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster
|
||||
{
|
||||
public:
|
||||
bool if_not_exists{false};
|
||||
|
||||
ASTPtr index_name;
|
||||
|
||||
/// Stores the IndexDeclaration here.
|
||||
ASTPtr index_decl;
|
||||
|
||||
String getID(char delim) const override;
|
||||
|
||||
ASTPtr clone() const override;
|
||||
|
||||
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams & params) const override
|
||||
{
|
||||
return removeOnCluster<ASTCreateIndexQuery>(clone(), params.default_database);
|
||||
}
|
||||
|
||||
virtual QueryKind getQueryKind() const override { return QueryKind::Create; }
|
||||
|
||||
protected:
|
||||
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
};
|
||||
|
||||
}
|
63
src/Parsers/ASTDropIndexQuery.cpp
Normal file
63
src/Parsers/ASTDropIndexQuery.cpp
Normal file
@ -0,0 +1,63 @@
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Parsers/ASTDropIndexQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
String ASTDropIndexQuery::getID(char delim) const
|
||||
{
|
||||
return "CreateIndexQuery" + (delim + getDatabase()) + delim + getTable();
|
||||
}
|
||||
|
||||
ASTPtr ASTDropIndexQuery::clone() const
|
||||
{
|
||||
auto res = std::make_shared<ASTDropIndexQuery>(*this);
|
||||
res->children.clear();
|
||||
|
||||
res->index_name = index_name->clone();
|
||||
res->children.push_back(res->index_name);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void ASTDropIndexQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
frame.need_parens = false;
|
||||
|
||||
std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' ');
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str;
|
||||
|
||||
settings.ostr << "DROP INDEX " << (if_exists ? "IF EXISTS " : "");
|
||||
index_name->formatImpl(settings, state, frame);
|
||||
settings.ostr << " ON ";
|
||||
|
||||
settings.ostr << (settings.hilite ? hilite_none : "");
|
||||
|
||||
if (table)
|
||||
{
|
||||
if (database)
|
||||
{
|
||||
settings.ostr << indent_str << backQuoteIfNeed(getDatabase());
|
||||
settings.ostr << ".";
|
||||
}
|
||||
settings.ostr << indent_str << backQuoteIfNeed(getTable());
|
||||
}
|
||||
|
||||
formatOnCluster(settings);
|
||||
}
|
||||
|
||||
ASTPtr ASTDropIndexQuery::convertToASTAlterCommand() const
|
||||
{
|
||||
auto command = std::make_shared<ASTAlterCommand>();
|
||||
command->index = index_name->clone();
|
||||
command->if_exists = if_exists;
|
||||
command->type = ASTAlterCommand::DROP_INDEX;
|
||||
|
||||
return command;
|
||||
}
|
||||
|
||||
}
|
42
src/Parsers/ASTDropIndexQuery.h
Normal file
42
src/Parsers/ASTDropIndexQuery.h
Normal file
@ -0,0 +1,42 @@
|
||||
#pragma once
|
||||
|
||||
#include <optional>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <Parsers/ASTQueryWithOnCluster.h>
|
||||
#include <Parsers/ASTQueryWithTableAndOutput.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/IParserBase.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** DROP INDEX [IF EXISTS] name on [db].name
|
||||
*/
|
||||
|
||||
class ASTDropIndexQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster
|
||||
{
|
||||
public:
|
||||
bool if_exists{false};
|
||||
|
||||
ASTPtr index_name;
|
||||
|
||||
String getID(char delim) const override;
|
||||
|
||||
ASTPtr clone() const override;
|
||||
|
||||
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams & params) const override
|
||||
{
|
||||
return removeOnCluster<ASTDropIndexQuery>(clone(), params.default_database);
|
||||
}
|
||||
|
||||
virtual QueryKind getQueryKind() const override { return QueryKind::Drop; }
|
||||
|
||||
/// Convert ASTDropIndexQuery to ASTAlterCommand.
|
||||
ASTPtr convertToASTAlterCommand() const;
|
||||
|
||||
protected:
|
||||
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
};
|
||||
|
||||
}
|
@ -24,10 +24,20 @@ ASTPtr ASTIndexDeclaration::clone() const
|
||||
|
||||
|
||||
void ASTIndexDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
if (from_create_index)
|
||||
{
|
||||
s.ostr << "(";
|
||||
expr->formatImpl(s, state, frame);
|
||||
s.ostr << ")";
|
||||
}
|
||||
else
|
||||
{
|
||||
s.ostr << backQuoteIfNeed(name);
|
||||
s.ostr << " ";
|
||||
expr->formatImpl(s, state, frame);
|
||||
}
|
||||
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << " TYPE " << (s.hilite ? hilite_none : "");
|
||||
type->formatImpl(s, state, frame);
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << " GRANULARITY " << (s.hilite ? hilite_none : "");
|
||||
|
@ -16,6 +16,7 @@ public:
|
||||
IAST * expr;
|
||||
ASTFunction * type;
|
||||
UInt64 granularity;
|
||||
bool from_create_index = false;
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
String getID(char) const override { return "Index"; }
|
||||
|
@ -840,7 +840,6 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
auto query = std::make_shared<ASTAlterQuery>();
|
||||
|
120
src/Parsers/ParserCreateIndexQuery.cpp
Normal file
120
src/Parsers/ParserCreateIndexQuery.cpp
Normal file
@ -0,0 +1,120 @@
|
||||
#include <Parsers/ParserCreateIndexQuery.h>
|
||||
|
||||
#include <Parsers/ASTCreateIndexQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTIndexDeclaration.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Parsers/ParserDataType.h>
|
||||
#include <Parsers/parseDatabaseAndTableName.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserCreateIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKeyword s_type("TYPE");
|
||||
ParserKeyword s_granularity("GRANULARITY");
|
||||
|
||||
ParserDataType data_type_p;
|
||||
ParserExpression expression_p;
|
||||
ParserUnsignedInteger granularity_p;
|
||||
|
||||
ASTPtr expr;
|
||||
ASTPtr type;
|
||||
ASTPtr granularity;
|
||||
|
||||
/// Skip name parser for SQL-standard CREATE INDEX
|
||||
if (!expression_p.parse(pos, expr, expected))
|
||||
return false;
|
||||
|
||||
if (!s_type.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!data_type_p.parse(pos, type, expected))
|
||||
return false;
|
||||
|
||||
if (!s_granularity.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!granularity_p.parse(pos, granularity, expected))
|
||||
return false;
|
||||
|
||||
auto index = std::make_shared<ASTIndexDeclaration>();
|
||||
index->from_create_index = true;
|
||||
index->granularity = granularity->as<ASTLiteral &>().value.safeGet<UInt64>();
|
||||
index->set(index->expr, expr);
|
||||
index->set(index->type, type);
|
||||
node = index;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ParserCreateIndexQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
auto query = std::make_shared<ASTCreateIndexQuery>();
|
||||
node = query;
|
||||
|
||||
ParserKeyword s_create("CREATE");
|
||||
ParserKeyword s_index("INDEX");
|
||||
ParserKeyword s_if_not_exists("IF NOT EXISTS");
|
||||
ParserKeyword s_on("ON");
|
||||
ParserIdentifier index_name_p;
|
||||
ParserCreateIndexDeclaration parser_create_idx_decl;
|
||||
|
||||
ASTPtr index_name;
|
||||
ASTPtr index_decl;
|
||||
|
||||
String cluster_str;
|
||||
bool if_not_exists = false;
|
||||
|
||||
if (!s_create.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!s_index.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
if_not_exists = true;
|
||||
|
||||
if (!index_name_p.parse(pos, index_name, expected))
|
||||
return false;
|
||||
|
||||
/// ON [db.] table_name
|
||||
if (!s_on.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!parseDatabaseAndTableAsAST(pos, expected, query->database, query->table))
|
||||
return false;
|
||||
|
||||
/// [ON cluster_name]
|
||||
if (s_on.ignore(pos, expected))
|
||||
{
|
||||
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!parser_create_idx_decl.parse(pos, index_decl, expected))
|
||||
return false;
|
||||
|
||||
query->index_name = index_name;
|
||||
query->children.push_back(index_name);
|
||||
|
||||
query->index_decl = index_decl;
|
||||
query->children.push_back(index_decl);
|
||||
|
||||
query->if_not_exists = if_not_exists;
|
||||
query->cluster = cluster_str;
|
||||
|
||||
if (query->database)
|
||||
query->children.push_back(query->database);
|
||||
|
||||
if (query->table)
|
||||
query->children.push_back(query->table);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
31
src/Parsers/ParserCreateIndexQuery.h
Normal file
31
src/Parsers/ParserCreateIndexQuery.h
Normal file
@ -0,0 +1,31 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Query like this:
|
||||
* CREATE INDEX [IF NOT EXISTS] name ON [db].name (expression) TYPE type GRANULARITY value
|
||||
*/
|
||||
|
||||
class ParserCreateIndexQuery : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override{ return "CREATE INDEX query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
|
||||
/** Parser for index declaration in create index, where name is ignored. */
|
||||
class ParserCreateIndexDeclaration : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserCreateIndexDeclaration() {}
|
||||
|
||||
protected:
|
||||
const char * getName() const override { return "index declaration in create index"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
67
src/Parsers/ParserDropIndexQuery.cpp
Normal file
67
src/Parsers/ParserDropIndexQuery.cpp
Normal file
@ -0,0 +1,67 @@
|
||||
#include <Parsers/ASTDropIndexQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/CommonParsers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/ParserDropIndexQuery.h>
|
||||
#include <Parsers/parseDatabaseAndTableName.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool ParserDropIndexQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
auto query = std::make_shared<ASTDropIndexQuery>();
|
||||
node = query;
|
||||
|
||||
ParserKeyword s_drop("DROP");
|
||||
ParserKeyword s_index("INDEX");
|
||||
ParserKeyword s_on("ON");
|
||||
ParserKeyword s_if_exists("IF EXISTS");
|
||||
ParserIdentifier index_name_p;
|
||||
|
||||
String cluster_str;
|
||||
bool if_exists = false;
|
||||
|
||||
if (!s_drop.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!s_index.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
if_exists = true;
|
||||
|
||||
if (!index_name_p.parse(pos, query->index_name, expected))
|
||||
return false;
|
||||
|
||||
/// ON [db.] table_name
|
||||
if (!s_on.ignore(pos, expected))
|
||||
return false;
|
||||
|
||||
if (!parseDatabaseAndTableAsAST(pos, expected, query->database, query->table))
|
||||
return false;
|
||||
|
||||
/// [ON cluster_name]
|
||||
if (s_on.ignore(pos, expected))
|
||||
{
|
||||
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
|
||||
return false;
|
||||
|
||||
query->cluster = std::move(cluster_str);
|
||||
}
|
||||
|
||||
if (query->index_name)
|
||||
query->children.push_back(query->index_name);
|
||||
|
||||
query->if_exists = if_exists;
|
||||
|
||||
if (query->database)
|
||||
query->children.push_back(query->database);
|
||||
|
||||
if (query->table)
|
||||
query->children.push_back(query->table);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
19
src/Parsers/ParserDropIndexQuery.h
Normal file
19
src/Parsers/ParserDropIndexQuery.h
Normal file
@ -0,0 +1,19 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IParserBase.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Query like this:
|
||||
* DROP INDEX [IF EXISTS] name ON [db].name
|
||||
*/
|
||||
|
||||
class ParserDropIndexQuery : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const override{ return "DROP INDEX query"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
}
|
@ -2,7 +2,9 @@
|
||||
#include <Parsers/ParserCreateFunctionQuery.h>
|
||||
#include <Parsers/ParserBackupQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/ParserCreateIndexQuery.h>
|
||||
#include <Parsers/ParserDropFunctionQuery.h>
|
||||
#include <Parsers/ParserDropIndexQuery.h>
|
||||
#include <Parsers/ParserDropQuery.h>
|
||||
#include <Parsers/ParserInsertQuery.h>
|
||||
#include <Parsers/ParserOptimizeQuery.h>
|
||||
@ -43,6 +45,8 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ParserCreateSettingsProfileQuery create_settings_profile_p;
|
||||
ParserCreateFunctionQuery create_function_p;
|
||||
ParserDropFunctionQuery drop_function_p;
|
||||
ParserCreateIndexQuery create_index_p;
|
||||
ParserDropIndexQuery drop_index_p;
|
||||
ParserDropAccessEntityQuery drop_access_entity_p;
|
||||
ParserGrantQuery grant_p;
|
||||
ParserSetRoleQuery set_role_p;
|
||||
@ -63,6 +67,8 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|| create_settings_profile_p.parse(pos, node, expected)
|
||||
|| create_function_p.parse(pos, node, expected)
|
||||
|| drop_function_p.parse(pos, node, expected)
|
||||
|| create_index_p.parse(pos, node, expected)
|
||||
|| drop_index_p.parse(pos, node, expected)
|
||||
|| drop_access_entity_p.parse(pos, node, expected)
|
||||
|| grant_p.parse(pos, node, expected)
|
||||
|| external_ddl_p.parse(pos, node, expected)
|
||||
|
@ -0,0 +1,4 @@
|
||||
CREATE TABLE default.t_index\n(\n `a` Int32,\n `b` String,\n INDEX i_a a TYPE minmax GRANULARITY 4,\n INDEX i_b b TYPE bloom_filter GRANULARITY 2\n)\nENGINE = MergeTree\nORDER BY a\nSETTINGS index_granularity = 8192
|
||||
t_index i_a minmax a 4
|
||||
t_index i_b bloom_filter b 2
|
||||
t_index i_b bloom_filter b 2
|
@ -0,0 +1,17 @@
|
||||
drop table if exists t_index;
|
||||
create table t_index(a int, b String) engine=MergeTree() order by a;
|
||||
|
||||
create index i_a on t_index(a) TYPE minmax GRANULARITY 4;
|
||||
create index if not exists i_a on t_index(a) TYPE minmax GRANULARITY 2;
|
||||
|
||||
create index i_b on t_index(b) TYPE bloom_filter GRANULARITY 2;
|
||||
|
||||
show create table t_index;
|
||||
select table, name, type, expr, granularity from system.data_skipping_indices where database = currentDatabase() and table = 't_index';
|
||||
|
||||
drop index i_a on t_index;
|
||||
drop index if exists i_a on t_index;
|
||||
|
||||
select table, name, type, expr, granularity from system.data_skipping_indices where database = currentDatabase() and table = 't_index';
|
||||
|
||||
drop table t_index;
|
Loading…
Reference in New Issue
Block a user