alter delete skeleton [#CLICKHOUSE-3688]

This commit is contained in:
Alexey Zatelepin 2018-02-02 19:02:43 +03:00 committed by alexey-milovidov
parent 195f91f191
commit 3b97d3938f
8 changed files with 91 additions and 8 deletions

View File

@ -50,7 +50,14 @@ BlockIO InterpreterAlterQuery::execute()
AlterCommands alter_commands;
PartitionCommands partition_commands;
parseAlter(alter.parameters, alter_commands, partition_commands);
MutationCommands mutation_commands;
parseAlter(alter.parameters, alter_commands, partition_commands, mutation_commands);
if (!mutation_commands.commands.empty())
{
/// TODO: validate
table->mutate(mutation_commands, context);
}
partition_commands.validate(table.get());
for (const PartitionCommand & command : partition_commands)
@ -79,18 +86,20 @@ BlockIO InterpreterAlterQuery::execute()
}
}
if (alter_commands.empty())
return {};
if (!alter_commands.empty())
{
alter_commands.validate(table.get(), context);
table->alter(alter_commands, database_name, table_name, context);
}
return {};
}
void InterpreterAlterQuery::parseAlter(
const ASTAlterQuery::ParameterContainer & params_container,
AlterCommands & out_alter_commands, PartitionCommands & out_partition_commands)
AlterCommands & out_alter_commands,
PartitionCommands & out_partition_commands,
MutationCommands & out_mutation_commands)
{
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
@ -186,6 +195,10 @@ void InterpreterAlterQuery::parseAlter(
{
out_partition_commands.emplace_back(PartitionCommand::freezePartition(params.partition, params.with_name));
}
else if (params.type == ASTAlterQuery::DELETE)
{
out_mutation_commands.commands.emplace_back(MutationCommand::delete_(params.predicate));
}
else
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);
}

View File

@ -2,6 +2,7 @@
#include <Storages/IStorage.h>
#include <Storages/AlterCommands.h>
#include <Storages/MutationCommands.h>
#include <Interpreters/Context.h>
#include <Interpreters/IInterpreter.h>
#include <Parsers/ASTAlterQuery.h>
@ -102,7 +103,9 @@ private:
const Context & context;
static void parseAlter(const ASTAlterQuery::ParameterContainer & params,
AlterCommands & out_alter_commands, PartitionCommands & out_partition_commands);
AlterCommands & out_alter_commands,
PartitionCommands & out_partition_commands,
MutationCommands & out_mutation_commands);
};
}

View File

@ -17,7 +17,9 @@ void ASTAlterQuery::Parameters::clone(Parameters & p) const
p = *this;
if (col_decl) p.col_decl = col_decl->clone();
if (column) p.column = column->clone();
if (primary_key) p.primary_key = primary_key->clone();
if (partition) p.partition = partition->clone();
if (predicate) p.predicate = predicate->clone();
}
void ASTAlterQuery::addParameters(const Parameters & params)
@ -31,6 +33,8 @@ void ASTAlterQuery::addParameters(const Parameters & params)
children.push_back(params.partition);
if (params.primary_key)
children.push_back(params.primary_key);
if (params.predicate)
children.push_back(params.predicate);
}
/** Get the text that identifies this element. */
@ -150,6 +154,11 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState
<< " " << std::quoted(p.with_name, '\'');
}
}
else if (p.type == ASTAlterQuery::DELETE)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DELETE WHERE " << (settings.hilite ? hilite_none : "");
p.predicate->formatImpl(settings, state, frame);
}
else
throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE);

View File

@ -31,6 +31,8 @@ public:
FETCH_PARTITION,
FREEZE_PARTITION,
DELETE,
NO_TYPE,
};
@ -59,6 +61,9 @@ public:
*/
ASTPtr partition;
/// For DELETE WHERE: the predicate that filters the rows to delete.
ASTPtr predicate;
bool detach = false; /// true for DETACH PARTITION
bool part = false; /// true for ATTACH PART

View File

@ -33,6 +33,8 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword s_with("WITH");
ParserKeyword s_name("NAME");
ParserKeyword s_delete_where("DELETE WHERE");
ParserToken s_dot(TokenType::Dot);
ParserToken s_comma(TokenType::Comma);
@ -41,6 +43,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserCompoundColumnDeclaration parser_col_decl;
ParserPartition parser_partition;
ParserStringLiteral parser_string_literal;
ParserExpression exp_elem;
ASTPtr table;
ASTPtr database;
@ -206,6 +209,13 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
params.type = ASTAlterQuery::MODIFY_PRIMARY_KEY;
}
else if (s_delete_where.ignore(pos, expected))
{
if (!exp_elem.parse(pos, params.predicate, expected))
return false;
params.type = ASTAlterQuery::DELETE;
}
else
return false;

View File

@ -16,6 +16,7 @@ namespace DB
* [DROP|DETACH|ATTACH PARTITION|PART partition, ...]
* [FETCH PARTITION partition FROM ...]
* [FREEZE PARTITION]
* [DELETE WHERE ...]
*/
class ParserAlterQuery : public IParserBase
{

View File

@ -40,6 +40,7 @@ using StorageWeakPtr = std::weak_ptr<IStorage>;
struct Settings;
class AlterCommands;
struct MutationCommands;
/** Does not allow changing the table description (including rename and delete the table).
@ -260,6 +261,12 @@ public:
throw Exception("Method optimize is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/// Mutate the table contents
virtual void mutate(const MutationCommands &, const Context &)
{
throw Exception("Mutations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
/** If the table have to do some complicated work on startup,
* that must be postponed after creation of table object
* (like launching some background threads),

View File

@ -0,0 +1,35 @@
#pragma once
#include <Parsers/IAST.h>
namespace DB
{
struct MutationCommand
{
enum Type
{
EMPTY, /// Not used.
DELETE,
};
Type type = EMPTY;
ASTPtr predicate;
static MutationCommand delete_(const ASTPtr & predicate)
{
MutationCommand res;
res.type = DELETE;
res.predicate = predicate;
return res;
}
};
struct MutationCommands
{
std::vector<MutationCommand> commands;
};
}