2018-09-03 13:36:58 +00:00
|
|
|
#pragma once
|
|
|
|
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <DataStreams/IBlockInputStream.h>
|
2018-09-03 13:36:58 +00:00
|
|
|
#include <Interpreters/ExpressionActions.h>
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <Interpreters/ExpressionAnalyzer.h>
|
2018-09-03 13:36:58 +00:00
|
|
|
#include <Interpreters/InterpreterSelectQuery.h>
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <Storages/IStorage_fwd.h>
|
2018-09-03 13:36:58 +00:00
|
|
|
#include <Storages/MutationCommands.h>
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
class Context;
|
|
|
|
|
2019-11-07 08:40:36 +00:00
|
|
|
/// Return false if the data isn't going to be changed by mutations.
|
|
|
|
bool isStorageTouchedByMutations(StoragePtr storage, const std::vector<MutationCommand> & commands, Context context_copy);
|
|
|
|
|
2019-04-10 17:44:39 +00:00
|
|
|
/// Create an input stream that will read data from storage and apply mutation commands (UPDATEs, DELETEs, MATERIALIZEs)
|
2018-09-03 13:36:58 +00:00
|
|
|
/// to this data.
|
|
|
|
class MutationsInterpreter
|
|
|
|
{
|
|
|
|
public:
|
2019-11-06 17:28:03 +00:00
|
|
|
/// Storage to mutate, array of mutations commands and context. If you really want to execute mutation
|
|
|
|
/// use can_execute = true, in other cases (validation, amount of commands) it can be false
|
2019-12-19 13:10:57 +00:00
|
|
|
MutationsInterpreter(StoragePtr storage_, MutationCommands commands_, const Context & context_, bool can_execute_);
|
2018-09-03 13:36:58 +00:00
|
|
|
|
2019-08-19 19:02:20 +00:00
|
|
|
void validate(TableStructureReadLockHolder & table_lock_holder);
|
2018-09-04 13:45:39 +00:00
|
|
|
|
2019-08-05 08:36:41 +00:00
|
|
|
size_t evaluateCommandsSize();
|
2019-08-05 03:24:59 +00:00
|
|
|
|
2019-08-05 18:06:05 +00:00
|
|
|
/// The resulting stream will return blocks containing only changed columns and columns, that we need to recalculate indices.
|
2019-08-19 19:02:20 +00:00
|
|
|
BlockInputStreamPtr execute(TableStructureReadLockHolder & table_lock_holder);
|
2018-09-03 13:36:58 +00:00
|
|
|
|
2019-08-05 18:06:05 +00:00
|
|
|
/// Only changed columns.
|
|
|
|
const Block & getUpdatedHeader() const;
|
|
|
|
|
2018-09-03 13:36:58 +00:00
|
|
|
private:
|
2019-08-05 03:24:59 +00:00
|
|
|
ASTPtr prepare(bool dry_run);
|
2018-09-04 13:45:39 +00:00
|
|
|
|
2019-08-05 18:06:05 +00:00
|
|
|
struct Stage;
|
|
|
|
|
2019-08-05 03:24:59 +00:00
|
|
|
ASTPtr prepareInterpreterSelectQuery(std::vector<Stage> &prepared_stages, bool dry_run);
|
2019-08-05 18:22:44 +00:00
|
|
|
BlockInputStreamPtr addStreamsForLaterStages(const std::vector<Stage> & prepared_stages, BlockInputStreamPtr in) const;
|
2018-09-03 13:36:58 +00:00
|
|
|
|
2020-03-26 15:59:09 +00:00
|
|
|
std::optional<SortDescription> getStorageSortDescriptionIfPossible(const Block & header) const;
|
|
|
|
|
2018-09-03 13:36:58 +00:00
|
|
|
StoragePtr storage;
|
2019-12-18 13:09:58 +00:00
|
|
|
MutationCommands commands;
|
2020-03-27 15:10:27 +00:00
|
|
|
Context context;
|
2019-11-06 17:28:03 +00:00
|
|
|
bool can_execute;
|
|
|
|
|
|
|
|
ASTPtr mutation_ast;
|
|
|
|
|
|
|
|
/// We have to store interpreter because it use own copy of context
|
|
|
|
/// and some streams from execute method may use it.
|
|
|
|
std::unique_ptr<InterpreterSelectQuery> select_interpreter;
|
2018-09-03 13:36:58 +00:00
|
|
|
|
|
|
|
/// A sequence of mutation commands is executed as a sequence of stages. Each stage consists of several
|
2018-09-07 15:44:51 +00:00
|
|
|
/// filters, followed by updating values of some columns. Commands can reuse expressions calculated by the
|
|
|
|
/// previous commands in the same stage, but at the end of each stage intermediate columns are thrown away
|
|
|
|
/// (they may contain wrong values because the column values have been updated).
|
|
|
|
///
|
|
|
|
/// If an UPDATE command changes some columns that some MATERIALIZED columns depend on, a stage to
|
|
|
|
/// recalculate these columns is added.
|
2018-09-03 13:36:58 +00:00
|
|
|
///
|
|
|
|
/// Each stage has output_columns that contain columns that are changed at the end of that stage
|
|
|
|
/// plus columns needed for the next mutations.
|
|
|
|
///
|
2019-12-25 12:34:04 +00:00
|
|
|
/// First stage is special: it can contain only filters and is executed using InterpreterSelectQuery
|
|
|
|
/// to take advantage of table indexes (if there are any). It's necessary because all mutations have
|
|
|
|
/// `WHERE clause` part.
|
2018-09-07 15:44:51 +00:00
|
|
|
|
2018-09-03 13:36:58 +00:00
|
|
|
struct Stage
|
|
|
|
{
|
2019-08-03 11:02:40 +00:00
|
|
|
Stage(const Context & context_) : expressions_chain(context_) {}
|
2018-09-07 19:14:05 +00:00
|
|
|
|
2018-09-07 15:44:51 +00:00
|
|
|
ASTs filters;
|
|
|
|
std::unordered_map<String, ASTPtr> column_to_updated;
|
2018-09-03 13:36:58 +00:00
|
|
|
|
|
|
|
/// Contains columns that are changed by this stage,
|
|
|
|
/// columns changed by the previous stages and also columns needed by the next stages.
|
|
|
|
NameSet output_columns;
|
|
|
|
|
2018-09-04 11:38:41 +00:00
|
|
|
std::unique_ptr<ExpressionAnalyzer> analyzer;
|
|
|
|
|
2018-09-03 13:36:58 +00:00
|
|
|
/// A chain of actions needed to execute this stage.
|
2018-09-07 15:44:51 +00:00
|
|
|
/// First steps calculate filter columns for DELETEs (in the same order as in `filter_column_names`),
|
2019-12-25 12:34:04 +00:00
|
|
|
/// then there is (possibly) an UPDATE step, and finally a projection step.
|
2018-09-03 13:36:58 +00:00
|
|
|
ExpressionActionsChain expressions_chain;
|
2018-09-07 15:44:51 +00:00
|
|
|
Names filter_column_names;
|
2018-09-03 13:36:58 +00:00
|
|
|
};
|
|
|
|
|
2019-08-05 18:06:05 +00:00
|
|
|
std::unique_ptr<Block> updated_header;
|
2018-09-03 13:36:58 +00:00
|
|
|
std::vector<Stage> stages;
|
|
|
|
bool is_prepared = false; /// Has the sequence of stages been prepared.
|
|
|
|
};
|
|
|
|
|
|
|
|
}
|