44
55#include < Access/Common/AccessRightsElement.h>
66#include < Common/typeid_cast.h>
7- #include < Storages/TableLockHolder.h>
8- #include < Parsers/ASTLiteral.h>
9- #include < Parsers/CommonParsers.h>
10- #include < Parsers/ParserDropQuery.h>
11- #include < Parsers/parseQuery.h>
127#include < Core/Settings.h>
138#include < Core/ServerSettings.h>
149#include < Databases/DatabaseFactory.h>
@@ -142,6 +137,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
142137 checkStorageSupportsTransactionsIfNeeded (table, getContext ());
143138 if (table->isStaticStorage ())
144139 throw Exception (ErrorCodes::TABLE_IS_READ_ONLY, " Table is read-only" );
140+ auto table_lock = table->lockForShare (getContext ()->getCurrentQueryId (), getContext ()->getSettingsRef ()[Setting::lock_acquire_timeout]);
145141
146142 if (modify_query)
147143 {
@@ -157,9 +153,6 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
157153 AlterCommands alter_commands;
158154 PartitionCommands partition_commands;
159155 MutationCommands mutation_commands;
160- bool is_truncate = false ;
161- bool forbid_truncate = false ;
162-
163156 for (const auto & child : alter.command_list ->children )
164157 {
165158 auto * command_ast = child->as <ASTAlterCommand>();
@@ -173,15 +166,8 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
173166 }
174167 else if (auto mut_command = MutationCommand::parse (command_ast))
175168 {
176- // / ALTER TABLE ... DELETE WHERE 1 should be executed as truncate
177- if (mut_command->type == MutationCommand::DELETE && isAlwaysTruePredicate (mut_command->predicate ))
178- {
179- is_truncate = true ;
180- }
181- else if (mut_command->type == MutationCommand::UPDATE || mut_command->type == MutationCommand::DELETE)
169+ if (mut_command->type == MutationCommand::UPDATE || mut_command->type == MutationCommand::DELETE)
182170 {
183- forbid_truncate = true ;
184-
185171 // / TODO: add a check for result query size.
186172 auto rewritten_command_ast = replaceNonDeterministicToScalars (*command_ast, getContext ());
187173 if (rewritten_command_ast)
@@ -206,45 +192,6 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
206192 throw Exception (ErrorCodes::INCORRECT_QUERY, " Alter table with statistics is now disabled. Turn on allow_experimental_statistics" );
207193 }
208194
209- if (mutation_commands.hasNonEmptyMutationCommands () || !partition_commands.empty ())
210- {
211- if (getContext ()->getServerSettings ()[ServerSetting::disable_insertion_and_mutation])
212- throw Exception (ErrorCodes::QUERY_IS_PROHIBITED, " Mutations are prohibited" );
213- }
214-
215- if (is_truncate && !forbid_truncate)
216- {
217- auto context = getContext ();
218- context->checkAccess (AccessType::TRUNCATE, table_id);
219- table->checkTableCanBeDropped (context);
220-
221- auto metadata_snapshot = table->getInMemoryMetadataPtr ();
222-
223- TableExclusiveLockHolder table_excl_lock;
224- // / We don't need any lock for ReplicatedMergeTree and for simple MergeTree
225- // / For the rest of tables types exclusive lock is needed
226- if (!std::dynamic_pointer_cast<MergeTreeData>(table))
227- table_excl_lock = table->lockExclusively (context->getCurrentQueryId (), context->getSettingsRef ()[Setting::lock_acquire_timeout]);
228- String truncate_query = " TRUNCATE TABLE " + table->getStorageID ().getFullTableName ()
229- + (alter.cluster .empty () ? " " : " ON CLUSTER " + backQuoteIfNeed (alter.cluster ));
230-
231- ParserDropQuery parser;
232- auto current_query_ptr = parseQuery (
233- parser,
234- truncate_query.data (),
235- truncate_query.data () + truncate_query.size (),
236- " ALTER query" ,
237- 0 ,
238- DBMS_DEFAULT_MAX_PARSER_DEPTH,
239- DBMS_DEFAULT_MAX_PARSER_BACKTRACKS);
240-
241- // / Drop table data, don't touch metadata
242- table->truncate (current_query_ptr, metadata_snapshot, context, table_excl_lock);
243- return {};
244- }
245-
246- auto table_lock = table->lockForShare (getContext ()->getCurrentQueryId (), getContext ()->getSettingsRef ()[Setting::lock_acquire_timeout]);
247-
248195 if (typeid_cast<DatabaseReplicated *>(database.get ()))
249196 {
250197 int command_types_count = !mutation_commands.empty () + !partition_commands.empty () + !alter_commands.empty ();
@@ -254,6 +201,12 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
254201 " to execute ALTERs of different types (replicated and non replicated) in single query" );
255202 }
256203
204+ if (mutation_commands.hasNonEmptyMutationCommands () || !partition_commands.empty ())
205+ {
206+ if (getContext ()->getServerSettings ()[ServerSetting::disable_insertion_and_mutation])
207+ throw Exception (ErrorCodes::QUERY_IS_PROHIBITED, " Mutations are prohibited" );
208+ }
209+
257210 if (!alter_commands.empty ())
258211 {
259212 auto alter_lock = table->lockForAlter (getContext ()->getSettingsRef ()[Setting::lock_acquire_timeout]);
0 commit comments