Merge pull request #56892 from icuken/alter_temporary_table

allow ALTER for TEMPORARY tables
This commit is contained in:
Julia Kartseva 2023-11-30 18:29:43 -08:00 committed by GitHub
commit 1f8031c6e1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 121 additions and 8 deletions

View File

@ -10,7 +10,7 @@ A set of queries that allow changing the table structure.
Syntax:
``` sql
ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|RENAME|CLEAR|COMMENT|{MODIFY|ALTER}|MATERIALIZE COLUMN ...
ALTER [TEMPORARY] TABLE [db].name [ON CLUSTER cluster] ADD|DROP|RENAME|CLEAR|COMMENT|{MODIFY|ALTER}|MATERIALIZE COLUMN ...
```
In the query, specify a list of one or more comma-separated actions.

View File

@ -11,7 +11,7 @@ sidebar_label: "Манипуляции со столбцами"
Синтаксис:
``` sql
ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|RENAME|CLEAR|COMMENT|{MODIFY|ALTER}|MATERIALIZE COLUMN ...
ALTER [TEMPORARY] TABLE [db].name [ON CLUSTER cluster] ADD|DROP|RENAME|CLEAR|COMMENT|{MODIFY|ALTER}|MATERIALIZE COLUMN ...
```
В запросе можно указать сразу несколько действий над одной таблицей через запятую.

View File

@ -75,7 +75,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
if (!UserDefinedSQLFunctionFactory::instance().empty())
UserDefinedSQLFunctionVisitor::visit(query_ptr);
auto table_id = getContext()->tryResolveStorageID(alter, Context::ResolveOrdinary);
auto table_id = getContext()->tryResolveStorageID(alter);
StoragePtr table;
if (table_id)

View File

@ -898,12 +898,13 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
node = query;
ParserKeyword s_alter_table("ALTER TABLE");
ParserKeyword s_alter_temporary_table("ALTER TEMPORARY TABLE");
ParserKeyword s_alter_live_view("ALTER LIVE VIEW");
ParserKeyword s_alter_database("ALTER DATABASE");
ASTAlterQuery::AlterObjectType alter_object_type;
if (s_alter_table.ignore(pos, expected))
if (s_alter_table.ignore(pos, expected) || s_alter_temporary_table.ignore(pos, expected))
{
alter_object_type = ASTAlterQuery::AlterObjectType::TABLE;
}

View File

@ -5,6 +5,7 @@
#include <Interpreters/MutationsInterpreter.h>
#include <Interpreters/getColumnFromBlock.h>
#include <Interpreters/inplaceBlockConversions.h>
#include <Storages/AlterCommands.h>
#include <Storages/StorageFactory.h>
#include <Storages/StorageMemory.h>
#include <Storages/MemorySettings.h>
@ -42,6 +43,7 @@ namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int CANNOT_RESTORE_TABLE;
extern const int NOT_IMPLEMENTED;
}
class MemorySink : public SinkToStorage
@ -487,6 +489,17 @@ void StorageMemory::restoreDataImpl(const BackupPtr & backup, const String & dat
total_size_rows += new_rows;
}
void StorageMemory::checkAlterIsPossible(const AlterCommands & commands, ContextPtr) const
{
for (const auto & command : commands)
{
if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN
&& command.type != AlterCommand::Type::DROP_COLUMN && command.type != AlterCommand::Type::COMMENT_COLUMN
&& command.type != AlterCommand::Type::COMMENT_TABLE && command.type != AlterCommand::Type::RENAME_COLUMN)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}",
command.type, getName());
}
}
std::optional<UInt64> StorageMemory::totalRows(const Settings &) const
{

View File

@ -76,6 +76,8 @@ public:
void backupData(BackupEntriesCollector & backup_entries_collector, const String & data_path_in_backup, const std::optional<ASTs> & partitions) override;
void restoreDataFromBackup(RestorerFromBackup & restorer, const String & data_path_in_backup, const std::optional<ASTs> & partitions) override;
void checkAlterIsPossible(const AlterCommands & commands, ContextPtr local_context) const override;
std::optional<UInt64> totalRows(const Settings &) const override;
std::optional<UInt64> totalBytes(const Settings &) const override;

View File

@ -0,0 +1,2 @@
s String
m Int8

View File

@ -5,7 +5,9 @@ CREATE TABLE defaults
s String
)ENGINE = Memory();
ALTER TABLE defaults ADD COLUMN m Int8; -- { serverError 48 }
ALTER TABLE defaults DROP COLUMN n; -- { serverError 48 }
ALTER TABLE defaults ADD COLUMN m Int8;
ALTER TABLE defaults DROP COLUMN n;
DESC TABLE defaults;
DROP TABLE defaults;

View File

@ -16,6 +16,7 @@ select * from t_row_exists;
drop table t_row_exists;
create table t_row_exists(a int, b int) engine=Memory;
alter table t_row_exists add column _row_exists int; --{serverError NOT_IMPLEMENTED}
alter table t_row_exists rename column b to _row_exists; --{serverError NOT_IMPLEMENTED}
alter table t_row_exists add column _row_exists int;
alter table t_row_exists drop column _row_exists;
alter table t_row_exists rename column b to _row_exists;
drop table t_row_exists;

View File

@ -0,0 +1,27 @@
CounterID UInt32
StartDate Date
UserID UInt32
VisitID UInt32
NestedColumn.A Array(UInt8)
NestedColumn.S Array(String)
ToDrop UInt32
Added0 UInt32
Added1 UInt32
Added2 UInt32
AddedNested1.A Array(UInt32)
AddedNested1.B Array(UInt64)
AddedNested1.C Array(String)
AddedNested2.A Array(UInt32)
AddedNested2.B Array(UInt64)
CounterID UInt32
StartDate Date
UserID UInt32
VisitID UInt32
RenamedColumn String
Added1 UInt32
Added2 UInt32
AddedNested1.A Array(UInt32)
AddedNested1.C Array(String)
AddedNested2.A Array(UInt32)
AddedNested2.B Array(UInt64)
1 2014-01-01 2 3 0 0 [] [] [] []

View File

@ -0,0 +1,38 @@
DROP TABLE IF EXISTS alter_test;
CREATE TEMPORARY TABLE alter_test (CounterID UInt32, StartDate Date, UserID UInt32, VisitID UInt32, NestedColumn Nested(A UInt8, S String), ToDrop UInt32);
INSERT INTO alter_test VALUES (1, '2014-01-01', 2, 3, [1,2,3], ['a','b','c'], 4);
ALTER TABLE alter_test ADD COLUMN Added0 UInt32;
ALTER TEMPORARY TABLE alter_test ADD COLUMN Added2 UInt32;
ALTER TABLE alter_test ADD COLUMN Added1 UInt32 AFTER Added0;
ALTER TABLE alter_test ADD COLUMN AddedNested1 Nested(A UInt32, B UInt64) AFTER Added2;
ALTER TABLE alter_test ADD COLUMN AddedNested1.C Array(String) AFTER AddedNested1.B;
ALTER TABLE alter_test ADD COLUMN AddedNested2 Nested(A UInt32, B UInt64) AFTER AddedNested1;
DESC TABLE alter_test;
ALTER TABLE alter_test DROP COLUMN ToDrop;
ALTER TABLE alter_test MODIFY COLUMN Added0 String;
ALTER TABLE alter_test DROP COLUMN NestedColumn.A;
ALTER TABLE alter_test DROP COLUMN NestedColumn.S;
ALTER TABLE alter_test DROP COLUMN AddedNested1.B;
ALTER TABLE alter_test ADD COLUMN IF NOT EXISTS Added0 UInt32;
ALTER TABLE alter_test ADD COLUMN IF NOT EXISTS AddedNested1 Nested(A UInt32, B UInt64);
ALTER TABLE alter_test ADD COLUMN IF NOT EXISTS AddedNested1.C Array(String);
ALTER TABLE alter_test MODIFY COLUMN IF EXISTS ToDrop UInt64;
ALTER TABLE alter_test DROP COLUMN IF EXISTS ToDrop;
ALTER TABLE alter_test COMMENT COLUMN IF EXISTS ToDrop 'new comment';
ALTER TABLE alter_test RENAME COLUMN Added0 to RenamedColumn;
DESC TABLE alter_test;
SELECT * FROM alter_test;
DROP TABLE alter_test;

View File

@ -0,0 +1,6 @@
b UInt8
a UInt32
a UInt32
b UInt8 this is comment for log engine
b UInt8
a UInt32

View File

@ -0,0 +1,21 @@
DROP TABLE IF EXISTS alter_test;
CREATE TEMPORARY TABLE alter_test (a UInt32, b UInt8) ENGINE=MergeTree ORDER BY a;
INSERT INTO alter_test VALUES (1, 2);
ALTER TEMPORARY TABLE alter_test MODIFY COLUMN b UInt8 FIRST;
DESC TABLE alter_test;
DROP TABLE IF EXISTS alter_test;
CREATE TEMPORARY TABLE alter_test (a UInt32, b UInt8) ENGINE=Log;
INSERT INTO alter_test VALUES (1, 2);
ALTER TEMPORARY TABLE alter_test COMMENT COLUMN b 'this is comment for log engine';
DESC TABLE alter_test;
DROP TABLE IF EXISTS alter_test;
CREATE TEMPORARY TABLE alter_test (a UInt32, b UInt8) ENGINE=Null;
INSERT INTO alter_test VALUES (1, 2);
ALTER TEMPORARY TABLE alter_test MODIFY COLUMN b UInt8 FIRST;
DESC TABLE alter_test;