Merge pull request #60886 from ArctypeZach/master

Add support for `START TRANSACTION` syntax
This commit is contained in:
Alexey Milovidov 2024-03-25 21:09:48 +03:00 committed by GitHub
commit 1928e7d1f6
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
4 changed files with 6 additions and 3 deletions

View File

@ -127,7 +127,7 @@ See the [deployment](docs/en/deployment-guides/terminology.md) documentation for
#### Verify that experimental transactions are enabled
Issue a `BEGIN TRANSACTION` followed by a `ROLLBACK` to verify that experimental transactions are enabled, and that ClickHouse Keeper is enabled as it is used to track transactions.
Issue a `BEGIN TRANSACTION` or `START TRANSACTION` followed by a `ROLLBACK` to verify that experimental transactions are enabled, and that ClickHouse Keeper is enabled as it is used to track transactions.
```sql
BEGIN TRANSACTION

View File

@ -445,6 +445,7 @@ namespace DB
MR_MACROS(SPATIAL, "SPATIAL") \
MR_MACROS(SQL_SECURITY, "SQL SECURITY") \
MR_MACROS(SS, "SS") \
MR_MACROS(START_TRANSACTION, "START TRANSACTION") \
MR_MACROS(STATISTIC, "STATISTIC") \
MR_MACROS(STEP, "STEP") \
MR_MACROS(STORAGE, "STORAGE") \

View File

@ -14,6 +14,8 @@ bool ParserTransactionControl::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
if (ParserKeyword(Keyword::BEGIN_TRANSACTION).ignore(pos, expected))
action = ASTTransactionControl::BEGIN;
else if (ParserKeyword(Keyword::START_TRANSACTION).ignore(pos, expected))
action = ASTTransactionControl::BEGIN;
else if (ParserKeyword(Keyword::COMMIT).ignore(pos, expected))
action = ASTTransactionControl::COMMIT;
else if (ParserKeyword(Keyword::ROLLBACK).ignore(pos, expected))

View File

@ -67,8 +67,8 @@ def test_rollback_unfinished_on_restart1(start_cluster):
tx(1, "insert into mt values (5, 50)")
tx(1, "alter table mt update m = m+n in partition id '1' where 1")
# check that uncommitted insert will be rolled back on restart
tx(3, "begin transaction")
# check that uncommitted insert will be rolled back on restart (using `START TRANSACTION` syntax)
tx(3, "start transaction")
tid5 = tx(3, "select transactionID()").strip()
tx(3, "insert into mt values (6, 70)")