Skip to content

Commit 58e0bc8

Browse files
authored
Merge pull request ClickHouse#80616 from ClickHouse/revert-78739-delete_to_truncate
Revert "Replace DELETE FROM ... WHERE 1 queries to TRUNCATE"
2 parents da4459b + 730bcf3 commit 58e0bc8

9 files changed

+14
-146
lines changed

src/Interpreters/InterpreterAlterQuery.cpp

Lines changed: 8 additions & 55 deletions
Original file line numberDiff line numberDiff line change
@@ -4,11 +4,6 @@
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]);

src/Interpreters/InterpreterDeleteQuery.cpp

Lines changed: 0 additions & 35 deletions
Original file line numberDiff line numberDiff line change
@@ -14,9 +14,6 @@
1414
#include <Parsers/parseQuery.h>
1515
#include <Parsers/ParserAlterQuery.h>
1616
#include <Parsers/ASTDeleteQuery.h>
17-
#include <Parsers/ASTLiteral.h>
18-
#include <Parsers/CommonParsers.h>
19-
#include <Parsers/ParserDropQuery.h>
2017
#include <Storages/AlterCommands.h>
2118
#include <Storages/IStorage.h>
2219
#include <Storages/MutationCommands.h>
@@ -83,38 +80,6 @@ BlockIO InterpreterDeleteQuery::execute()
8380
return database->tryEnqueueReplicatedDDL(query_ptr, getContext(), {});
8481
}
8582

86-
/// DELETE FROM ... WHERE 1 should be executed as truncate
87-
if ((table->supportsDelete() || table->supportsLightweightDelete()) && isAlwaysTruePredicate(delete_query.predicate))
88-
{
89-
auto context = getContext();
90-
context->checkAccess(AccessType::TRUNCATE, table_id);
91-
table->checkTableCanBeDropped(context);
92-
93-
TableExclusiveLockHolder table_excl_lock;
94-
/// We don't need any lock for ReplicatedMergeTree and for simple MergeTree
95-
/// For the rest of tables types exclusive lock is needed
96-
if (!std::dynamic_pointer_cast<MergeTreeData>(table))
97-
table_excl_lock = table->lockExclusively(context->getCurrentQueryId(), context->getSettingsRef()[Setting::lock_acquire_timeout]);
98-
99-
String truncate_query = "TRUNCATE TABLE " + table->getStorageID().getFullTableName()
100-
+ (delete_query.cluster.empty() ? "" : " ON CLUSTER " + backQuoteIfNeed(delete_query.cluster));
101-
102-
ParserDropQuery parser;
103-
auto current_query_ptr = parseQuery(
104-
parser,
105-
truncate_query.data(),
106-
truncate_query.data() + truncate_query.size(),
107-
"ALTER query",
108-
0,
109-
DBMS_DEFAULT_MAX_PARSER_DEPTH,
110-
DBMS_DEFAULT_MAX_PARSER_BACKTRACKS);
111-
112-
auto metadata_snapshot = table->getInMemoryMetadataPtr();
113-
/// Drop table data, don't touch metadata
114-
table->truncate(current_query_ptr, metadata_snapshot, context, table_excl_lock);
115-
return {};
116-
}
117-
11883
auto table_lock = table->lockForShare(getContext()->getCurrentQueryId(), getContext()->getSettingsRef()[Setting::lock_acquire_timeout]);
11984
auto metadata_snapshot = table->getInMemoryMetadataPtr();
12085

src/Parsers/CommonParsers.cpp

Lines changed: 0 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -1,10 +1,8 @@
11
#include <algorithm>
22
#include <cctype>
3-
#include <Parsers/ASTLiteral.h>
43
#include <Parsers/CommonParsers.h>
54
#include <base/find_symbols.h>
65
#include <Common/ErrorCodes.h>
7-
86
namespace DB
97
{
108

@@ -156,20 +154,5 @@ bool ParserKeyword::parseImpl(Pos & pos, [[maybe_unused]] ASTPtr & node, Expecte
156154
return true;
157155
}
158156

159-
bool isAlwaysTruePredicate(const ASTPtr & predicate)
160-
{
161-
if (!predicate)
162-
return true;
163-
164-
const auto * ast_literal = predicate->as<ASTLiteral>();
165-
if (!ast_literal)
166-
return false;
167-
168-
UInt64 result;
169-
if (ast_literal->value.tryGet<UInt64>(result) && result == 1)
170-
return true;
171-
return false;
172-
}
173-
174157

175158
}

src/Parsers/CommonParsers.h

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -710,6 +710,4 @@ class ParserNothing : public IParserBase
710710
bool parseImpl(Pos & /*pos*/, ASTPtr & /*node*/, Expected & /*expected*/) override { return true; }
711711
};
712712

713-
bool isAlwaysTruePredicate(const ASTPtr & predicate);
714-
715713
}

tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree_compact_parts.reference

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -5,6 +5,7 @@
55
1 Compact
66
95
77
1
8+
1 Compact
89
0
910
1
1011
5 Compact
Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
1-
mutations after ALTER for data_rmt 0
2-
mutations after cleanup for data_rmt 0
3-
mutations after ALTER for data_mt 0
4-
mutations after cleanup for data_mt 0
1+
mutations after ALTER for data_rmt 1
2+
mutations after cleanup for data_rmt 1
3+
mutations after ALTER for data_mt 1
4+
mutations after cleanup for data_mt 1
Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,5 @@
11
system.distribution_queue 1
22
system.rocksdb 1
33
system.databases 1
4-
system.mutations 0
4+
system.mutations 1
55
system.replication_queue 1

tests/queries/0_stateless/03403_deletes_with_true_predicate.reference

Lines changed: 0 additions & 6 deletions
This file was deleted.

tests/queries/0_stateless/03403_deletes_with_true_predicate.sql

Lines changed: 0 additions & 26 deletions
This file was deleted.

0 commit comments

Comments
 (0)