You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2021/08/11 08:37:51 UTC

[incubator-doris] branch master updated: [Alter] Support alter table and column's comment (#6387)

This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 7e93405  [Alter] Support alter table and column's comment (#6387)
7e93405 is described below

commit 7e93405df39ddc3f230ffafb09156bcb0264f972
Author: Mingyu Chen <mo...@gmail.com>
AuthorDate: Wed Aug 11 16:37:42 2021 +0800

    [Alter] Support alter table and column's comment (#6387)
    
    1. alter table tbl1 modify comment "new comment";
    2. alter table tbl1 modify column k1 comment "k1", modify column v1 comment "v1";
---
 .../sql-statements/Data Definition/ALTER TABLE.md  | 16 ++++
 .../sql-statements/Data Definition/ALTER TABLE.md  | 17 ++++
 fe/fe-core/src/main/cup/sql_parser.cup             |  8 ++
 .../main/java/org/apache/doris/alter/Alter.java    | 94 ++++++++++++++++++++-
 .../java/org/apache/doris/alter/AlterOpType.java   |  4 +
 .../doris/analysis/ModifyColumnCommentClause.java  | 75 +++++++++++++++++
 .../doris/analysis/ModifyTableCommentClause.java   | 66 +++++++++++++++
 .../org/apache/doris/journal/JournalEntity.java    |  9 +-
 .../java/org/apache/doris/persist/EditLog.java     |  9 ++
 .../doris/persist/ModifyCommentOperationLog.java   | 96 ++++++++++++++++++++++
 .../org/apache/doris/persist/OperationType.java    |  1 +
 .../java/org/apache/doris/alter/AlterTest.java     | 45 ++++++++++
 .../persist/ModifyCommentOperationLogTest.java     | 91 ++++++++++++++++++++
 .../persist/ReplaceTableOperationLogTest.java      |  4 +-
 14 files changed, 529 insertions(+), 6 deletions(-)

diff --git a/docs/en/sql-reference/sql-statements/Data Definition/ALTER TABLE.md b/docs/en/sql-reference/sql-statements/Data Definition/ALTER TABLE.md
index 044be24..e11a924 100644
--- a/docs/en/sql-reference/sql-statements/Data Definition/ALTER TABLE.md	
+++ b/docs/en/sql-reference/sql-statements/Data Definition/ALTER TABLE.md	
@@ -200,6 +200,14 @@ under the License.
         note:
             1)Only support non colocate table with RANGE partition and HASH distribution
 
+    10. Modify table comment
+        grammer:
+            MODIFY COMMENT "new table comment"
+
+    11. Modify column comment
+        grammer:
+            MODIFY COLUMN col1 COMMENT "new column comment"
+
      
     Rename supports modification of the following names:
     1. Modify the table name
@@ -379,6 +387,14 @@ under the License.
     18. Modify the default buckets number of example_db.my_table to 50
 
         ALTER TABLE example_db.my_table MODIFY DISTRIBUTION DISTRIBUTED BY HASH(k1) BUCKETS 50;
+
+    19. Modify table comment
+
+        ALTER TABLE example_db.my_table MODIFY COMMENT "new comment";
+
+    20. Modify column comment
+
+        ALTER TABLE example_db.my_table MODIFY COLUMN k1 COMMENT "k1", MODIFY COLUMN k2 COMMENT "k2";
         
     [rename]
     1. Modify the table named table1 to table2
diff --git a/docs/zh-CN/sql-reference/sql-statements/Data Definition/ALTER TABLE.md b/docs/zh-CN/sql-reference/sql-statements/Data Definition/ALTER TABLE.md
index cd0ddd1..15c74e4 100644
--- a/docs/zh-CN/sql-reference/sql-statements/Data Definition/ALTER TABLE.md	
+++ b/docs/zh-CN/sql-reference/sql-statements/Data Definition/ALTER TABLE.md	
@@ -198,6 +198,13 @@ under the License.
         注意:
             1)只能用在分区类型为RANGE,采用哈希分桶的非colocate表
 
+    10. 修改表注释
+        语法:
+            MODIFY COMMENT "new table comment"
+
+    11. 修改列注释
+        语法:
+            MODIFY COLUMN col1 COMMENT "new column comment"
 
     rename 支持对以下名称进行修改:
     1. 修改表名
@@ -211,6 +218,7 @@ under the License.
     3. 修改 partition 名称
         语法:
             RENAME PARTITION old_partition_name new_partition_name;
+
     bitmap index 支持如下几种修改方式
     1. 创建bitmap 索引
         语法:
@@ -218,6 +226,7 @@ under the License.
         注意:
             1. 目前仅支持bitmap 索引
             1. BITMAP 索引仅在单列上创建
+
     2. 删除索引
         语法:
             DROP INDEX index_name;
@@ -373,6 +382,14 @@ under the License.
     18. 将表的默认分桶数改为50
 
         ALTER TABLE example_db.my_table MODIFY DISTRIBUTION DISTRIBUTED BY HASH(k1) BUCKETS 50;
+
+    19. 修改表注释
+
+        ALTER TABLE example_db.my_table MODIFY COMMENT "new comment";
+
+    20. 修改列注释
+
+        ALTER TABLE example_db.my_table MODIFY COLUMN k1 COMMENT "k1", MODIFY COLUMN k2 COMMENT "k2";
     
     [rename]
     1. 将名为 table1 的表修改为 table2
diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index a433cf4..2981e84 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -1047,6 +1047,14 @@ alter_table_clause ::=
     {:
         RESULT = new ModifyDistributionClause(distribution);
     :}
+    | KW_MODIFY KW_COMMENT STRING_LITERAL:comment
+    {:
+        RESULT = new ModifyTableCommentClause(comment);
+    :}
+    | KW_MODIFY KW_COLUMN ident:colName KW_COMMENT STRING_LITERAL:comment
+    {:
+        RESULT = new ModifyColumnCommentClause(colName, comment);
+    :}
     ;
 
 opt_enable_feature_properties ::=
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
index a7391a9..9338552 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
@@ -26,8 +26,10 @@ import org.apache.doris.analysis.ColumnRenameClause;
 import org.apache.doris.analysis.CreateMaterializedViewStmt;
 import org.apache.doris.analysis.DropMaterializedViewStmt;
 import org.apache.doris.analysis.DropPartitionClause;
+import org.apache.doris.analysis.ModifyColumnCommentClause;
 import org.apache.doris.analysis.ModifyDistributionClause;
 import org.apache.doris.analysis.ModifyPartitionClause;
+import org.apache.doris.analysis.ModifyTableCommentClause;
 import org.apache.doris.analysis.ModifyTablePropertiesClause;
 import org.apache.doris.analysis.PartitionRenameClause;
 import org.apache.doris.analysis.ReplacePartitionClause;
@@ -56,17 +58,19 @@ import org.apache.doris.common.util.DynamicPartitionUtil;
 import org.apache.doris.common.util.PropertyAnalyzer;
 import org.apache.doris.persist.AlterViewInfo;
 import org.apache.doris.persist.BatchModifyPartitionsInfo;
+import org.apache.doris.persist.ModifyCommentOperationLog;
 import org.apache.doris.persist.ModifyPartitionInfo;
 import org.apache.doris.persist.ReplaceTableOperationLog;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.thrift.TTabletType;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
@@ -196,6 +200,12 @@ public class Alter {
             Preconditions.checkState(alterClauses.size() == 1);
             AlterClause alterClause = alterClauses.get(0);
             Catalog.getCurrentCatalog().modifyDefaultDistributionBucketNum(db, olapTable, (ModifyDistributionClause) alterClause);
+        } else if (currentAlterOps.contains(AlterOpType.MODIFY_COLUMN_COMMENT)) {
+            processModifyColumnComment(db, olapTable, alterClauses);
+        } else if (currentAlterOps.contains(AlterOpType.MODIFY_TABLE_COMMENT)) {
+            Preconditions.checkState(alterClauses.size() == 1);
+            AlterClause alterClause = alterClauses.get(0);
+            processModifyTableComment(db, olapTable, alterClause);
         } else {
             throw new DdlException("Invalid alter operations: " + currentAlterOps);
         }
@@ -203,6 +213,84 @@ public class Alter {
         return needProcessOutsideTableLock;
     }
 
+    private void processModifyTableComment(Database db, OlapTable tbl, AlterClause alterClause)
+            throws DdlException {
+        tbl.writeLock();
+        try {
+            ModifyTableCommentClause clause = (ModifyTableCommentClause) alterClause;
+            tbl.setComment(clause.getComment());
+            // log
+            ModifyCommentOperationLog op = ModifyCommentOperationLog.forTable(db.getId(), tbl.getId(), clause.getComment());
+            Catalog.getCurrentCatalog().getEditLog().logModifyComment(op);
+        } finally {
+            tbl.writeUnlock();
+        }
+    }
+
+    private void processModifyColumnComment(Database db, OlapTable tbl, List<AlterClause> alterClauses)
+            throws DdlException {
+        tbl.writeLock();
+        try {
+            // check first
+            Map<String, String> colToComment = Maps.newHashMap();
+            for (AlterClause alterClause : alterClauses) {
+                Preconditions.checkState(alterClause instanceof ModifyColumnCommentClause);
+                ModifyColumnCommentClause clause = (ModifyColumnCommentClause) alterClause;
+                String colName = clause.getColName();
+                if (tbl.getColumn(colName) == null) {
+                    throw new DdlException("Unknown column: " + colName);
+                }
+                if (colToComment.containsKey(colName)) {
+                    throw new DdlException("Duplicate column: " + colName);
+                }
+                colToComment.put(colName, clause.getComment());
+            }
+
+            // modify comment
+            for (Map.Entry<String, String> entry : colToComment.entrySet()) {
+                Column col = tbl.getColumn(entry.getKey());
+                col.setComment(entry.getValue());
+            }
+
+            // log
+            ModifyCommentOperationLog op = ModifyCommentOperationLog.forColumn(db.getId(), tbl.getId(), colToComment);
+            Catalog.getCurrentCatalog().getEditLog().logModifyComment(op);
+        } finally {
+            tbl.writeUnlock();
+        }
+    }
+
+    public void replayModifyComment(ModifyCommentOperationLog operation) {
+        long dbId = operation.getDbId();
+        long tblId = operation.getTblId();
+        Database db = Catalog.getCurrentCatalog().getDb(dbId);
+        if (db == null) {
+            return;
+        }
+        Table tbl = db.getTable(tblId);
+        if (tbl == null) {
+            return;
+        }
+        tbl.writeLock();
+        try {
+            ModifyCommentOperationLog.Type type = operation.getType();
+            switch (type) {
+                case TABLE:
+                    tbl.setComment(operation.getTblComment());
+                    break;
+                case COLUMN:
+                    for (Map.Entry<String, String> entry : operation.getColToComment().entrySet()) {
+                        tbl.getColumn(entry.getKey()).setComment(entry.getValue());
+                    }
+                    break;
+                default:
+                    break;
+            }
+        } finally {
+            tbl.writeUnlock();
+        }
+    }
+
     private void processAlterExternalTable(AlterTableStmt stmt, Table externalTable, Database db) throws UserException {
         stmt.rewriteAlterClause(externalTable);
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOpType.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOpType.java
index 8ceb918..ff0710b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOpType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterOpType.java
@@ -38,6 +38,8 @@ public enum AlterOpType {
     ENABLE_FEATURE,
     REPLACE_TABLE,
     MODIFY_DISTRIBUTION,
+    MODIFY_TABLE_COMMENT,
+    MODIFY_COLUMN_COMMENT,
     INVALID_OP; // INVALID_OP must be the last one
 
     // true means 2 operations have no conflict.
@@ -55,6 +57,8 @@ public enum AlterOpType {
         COMPATIBILITY_MATRIX[DROP_ROLLUP.ordinal()][DROP_ROLLUP.ordinal()] = true;
         // schema change, such as add/modify/drop columns can be processed in batch
         COMPATIBILITY_MATRIX[SCHEMA_CHANGE.ordinal()][SCHEMA_CHANGE.ordinal()] = true;
+        // can modify multi column comments at same time
+        COMPATIBILITY_MATRIX[MODIFY_COLUMN_COMMENT.ordinal()][MODIFY_COLUMN_COMMENT.ordinal()] = true;
     }
 
     public boolean needCheckCapacity() {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyColumnCommentClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyColumnCommentClause.java
new file mode 100644
index 0000000..ecbe676
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyColumnCommentClause.java
@@ -0,0 +1,75 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.analysis;
+
+import org.apache.doris.alter.AlterOpType;
+import org.apache.doris.common.AnalysisException;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Maps;
+
+import java.util.Map;
+
+// MODIFY COLUMN k1 COMMENT "new comment";
+public class ModifyColumnCommentClause extends AlterTableClause {
+    private static final Logger LOG = LogManager.getLogger(ModifyColumnCommentClause.class);
+    private String colName;
+    private String comment;
+
+    public ModifyColumnCommentClause(String colName, String comment) {
+        super(AlterOpType.MODIFY_COLUMN_COMMENT);
+        this.colName = colName;
+        this.comment = Strings.nullToEmpty(comment);
+    }
+
+    public String getColName() {
+        return colName;
+    }
+
+    public String getComment() {
+        return comment;
+    }
+
+    @Override
+    public Map<String, String> getProperties() {
+        return Maps.newHashMap();
+    }
+
+    @Override
+    public void analyze(Analyzer analyzer) throws AnalysisException {
+        if (Strings.isNullOrEmpty(colName)) {
+            throw new AnalysisException("Empty column name");
+        }
+    }
+
+    @Override
+    public String toSql() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("MODIFY COLUMN COMMENT ").append(colName);
+        sb.append(" '").append(comment).append("'");
+        return sb.toString();
+    }
+
+    @Override
+    public String toString() {
+        return toSql();
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTableCommentClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTableCommentClause.java
new file mode 100644
index 0000000..1e6780b
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTableCommentClause.java
@@ -0,0 +1,66 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.analysis;
+
+import org.apache.doris.alter.AlterOpType;
+import org.apache.doris.common.AnalysisException;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Maps;
+
+import java.util.Map;
+
+// MODIFY COMMENT "new tbl comment";
+public class ModifyTableCommentClause extends AlterTableClause {
+    private static final Logger LOG = LogManager.getLogger(ModifyTableCommentClause.class);
+    private String comment;
+
+    public ModifyTableCommentClause(String comment) {
+        super(AlterOpType.MODIFY_TABLE_COMMENT);
+        this.comment = Strings.nullToEmpty(comment);
+    }
+
+    public String getComment() {
+        return comment;
+    }
+
+    @Override
+    public void analyze(Analyzer analyzer) throws AnalysisException {
+    }
+
+    @Override
+    public Map<String, String> getProperties() {
+        return Maps.newHashMap();
+    }
+
+    @Override
+    public String toSql() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("MODIFY COMMENT ");
+        sb.append("'").append(comment).append("'");
+        return sb.toString();
+    }
+
+    @Override
+    public String toString() {
+        return toSql();
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java
index 10bb4e9..9f9b3b9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java
@@ -68,13 +68,14 @@ import org.apache.doris.persist.DropPartitionInfo;
 import org.apache.doris.persist.DropResourceOperationLog;
 import org.apache.doris.persist.GlobalVarPersistInfo;
 import org.apache.doris.persist.HbPackage;
+import org.apache.doris.persist.LdapInfo;
+import org.apache.doris.persist.ModifyCommentOperationLog;
 import org.apache.doris.persist.ModifyPartitionInfo;
 import org.apache.doris.persist.ModifyTableDefaultDistributionBucketNumOperationLog;
 import org.apache.doris.persist.ModifyTablePropertyOperationLog;
 import org.apache.doris.persist.OperationType;
 import org.apache.doris.persist.PartitionPersistInfo;
 import org.apache.doris.persist.PrivInfo;
-import org.apache.doris.persist.LdapInfo;
 import org.apache.doris.persist.RecoverInfo;
 import org.apache.doris.persist.RefreshExternalTableInfo;
 import org.apache.doris.persist.RemoveAlterJobV2OperationLog;
@@ -91,6 +92,7 @@ import org.apache.doris.qe.SessionVariable;
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.Frontend;
 import org.apache.doris.transaction.TransactionState;
+
 import com.google.common.base.Preconditions;
 
 import org.apache.logging.log4j.LogManager;
@@ -616,6 +618,11 @@ public class JournalEntity implements Writable {
                 isRead = true;
                 break;
             }
+            case OperationType.OP_MODIFY_COMMENT: {
+                data = ModifyCommentOperationLog.read(in);
+                isRead = true;
+                break;
+            }
             case OperationType.OP_ALTER_ROUTINE_LOAD_JOB: {
                 data = AlterRoutineLoadJobOperationLog.read(in);
                 isRead = true;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java
index 7fbcd47..dd029f2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java
@@ -823,6 +823,11 @@ public class EditLog {
                     }
                     break;
                 }
+                case OperationType.OP_MODIFY_COMMENT: {
+                    ModifyCommentOperationLog operation = (ModifyCommentOperationLog) journal.getData();
+                    catalog.getAlterInstance().replayModifyComment(operation);
+                    break;
+                }
                 case OperationType.OP_ALTER_ROUTINE_LOAD_JOB: {
                     AlterRoutineLoadJobOperationLog log = (AlterRoutineLoadJobOperationLog) journal.getData();
                     catalog.getRoutineLoadManager().replayAlterRoutineLoadJob(log);
@@ -1437,4 +1442,8 @@ public class EditLog {
     public void logBatchRemoveTransactions(BatchRemoveTransactionsOperation op) {
         logEdit(OperationType.OP_BATCH_REMOVE_TXNS, op);
     }
+
+    public void logModifyComment(ModifyCommentOperationLog op) {
+        logEdit(OperationType.OP_MODIFY_COMMENT, op);
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/ModifyCommentOperationLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/ModifyCommentOperationLog.java
new file mode 100644
index 0000000..ed3d737
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/ModifyCommentOperationLog.java
@@ -0,0 +1,96 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.persist;
+
+import org.apache.doris.common.io.Text;
+import org.apache.doris.common.io.Writable;
+import org.apache.doris.persist.gson.GsonUtils;
+
+import com.google.gson.annotations.SerializedName;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map;
+
+// Persist the info when removing batch of expired txns
+public class ModifyCommentOperationLog implements Writable {
+
+    public enum Type {
+        COLUMN, TABLE
+    }
+
+    @SerializedName(value = "type")
+    private Type type;
+    @SerializedName(value = "dbId")
+    private long dbId;
+    @SerializedName(value = "tblId")
+    private long tblId;
+    @SerializedName(value = "colToComment")
+    // col name to comment
+    private Map<String, String> colToComment;
+    @SerializedName(value = "tblComment")
+    private String tblComment;
+
+    private ModifyCommentOperationLog(Type type, long dbId, long tblId, Map<String, String> colToComment, String tblComment) {
+        this.type = type;
+        this.dbId = dbId;
+        this.tblId = tblId;
+        this.colToComment = colToComment;
+        this.tblComment = tblComment;
+    }
+
+    public static ModifyCommentOperationLog forColumn(long dbId, long tblId, Map<String, String> colToComment) {
+        return new ModifyCommentOperationLog(Type.COLUMN, dbId, tblId, colToComment, null);
+    }
+
+    public static ModifyCommentOperationLog forTable(long dbId, long tblId, String comment) {
+        return new ModifyCommentOperationLog(Type.TABLE, dbId, tblId, null, comment);
+    }
+
+    public Type getType() {
+        return type;
+    }
+
+    public long getDbId() {
+        return dbId;
+    }
+
+    public long getTblId() {
+        return tblId;
+    }
+
+    public Map<String, String> getColToComment() {
+        return colToComment;
+    }
+
+    public String getTblComment() {
+        return tblComment;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        String json = GsonUtils.GSON.toJson(this);
+        Text.writeString(out, json);
+    }
+
+    public static ModifyCommentOperationLog read(DataInput in) throws IOException {
+        String json = Text.readString(in);
+        return GsonUtils.GSON.fromJson(json, ModifyCommentOperationLog.class);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java
index 9ac9aa5..6d19457 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java
@@ -66,6 +66,7 @@ public class OperationType {
     public static final short OP_BATCH_ADD_ROLLUP = 123;
     public static final short OP_BATCH_DROP_ROLLUP = 124;
     public static final short OP_REMOVE_ALTER_JOB_V2 = 125;
+    public static final short OP_MODIFY_COMMENT = 126;
 
     // 30~39 130~139 230~239 ...
     // load job for only hadoop load
diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
index 62b1561..a206165 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
@@ -211,6 +211,51 @@ public class AlterTest {
     }
 
     @Test
+    public void alterTableModifyComment() throws Exception {
+        Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
+        Table tbl = db.getTable("tbl5");
+
+        // table comment
+        String stmt = "alter table test.tbl5 modify comment 'comment1'";
+        alterTable(stmt, false);
+        Assert.assertEquals("comment1", tbl.getComment());
+
+        // column comment
+        stmt = "alter table test.tbl5 modify column k1 comment 'k1'";
+        alterTable(stmt, false);
+        Assert.assertEquals("k1", tbl.getColumn("k1").getComment());
+
+        // columns comment
+        stmt = "alter table test.tbl5 modify column k1 comment 'k11', modify column v1 comment 'v11'";
+        alterTable(stmt, false);
+        Assert.assertEquals("k11", tbl.getColumn("k1").getComment());
+        Assert.assertEquals("v11", tbl.getColumn("v1").getComment());
+
+        // empty comment
+        stmt = "alter table test.tbl5 modify comment ''";
+        alterTable(stmt, false);
+        Assert.assertEquals("OLAP", tbl.getComment());
+
+        // empty column comment
+        stmt = "alter table test.tbl5 modify column k1 comment '', modify column v1 comment 'v111'";
+        alterTable(stmt, false);
+        Assert.assertEquals("", tbl.getColumn("k1").getComment());
+        Assert.assertEquals("v111", tbl.getColumn("v1").getComment());
+
+        // unknown column
+        stmt = "alter table test.tbl5 modify column x comment '', modify column v1 comment 'v111'";
+        alterTable(stmt, true);
+        Assert.assertEquals("", tbl.getColumn("k1").getComment());
+        Assert.assertEquals("v111", tbl.getColumn("v1").getComment());
+
+        // duplicate column
+        stmt = "alter table test.tbl5 modify column k1 comment '', modify column k1 comment 'v111'";
+        alterTable(stmt, true);
+        Assert.assertEquals("", tbl.getColumn("k1").getComment());
+        Assert.assertEquals("v111", tbl.getColumn("v1").getComment());
+    }
+
+    @Test
     public void testConflictAlterOperations() throws Exception {
         String stmt = "alter table test.tbl1 add partition p3 values less than('2020-04-01'), add partition p4 values less than('2020-05-01')";
         alterTable(stmt, true);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/ModifyCommentOperationLogTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/ModifyCommentOperationLogTest.java
new file mode 100644
index 0000000..51a547a
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/persist/ModifyCommentOperationLogTest.java
@@ -0,0 +1,91 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+package org.apache.doris.persist;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.util.Map;
+
+public class ModifyCommentOperationLogTest {
+    @Test
+    public void testColCommentSerialization() throws Exception {
+        // 1. Write objects to file
+        File file = new File("./ModifyColumnCommentOperationLogTest");
+        file.createNewFile();
+        DataOutputStream dos = new DataOutputStream(new FileOutputStream(file));
+
+        Map<String, String> colToComment = Maps.newHashMap();
+        colToComment.put("k1", "comment1");
+        colToComment.put("k2", "comment2");
+        ModifyCommentOperationLog log = ModifyCommentOperationLog.forColumn(1L, 2L, colToComment);
+        log.write(dos);
+
+        dos.flush();
+        dos.close();
+
+        // 2. Read objects from file
+        DataInputStream dis = new DataInputStream(new FileInputStream(file));
+
+        ModifyCommentOperationLog readLog = ModifyCommentOperationLog.read(dis);
+        Assert.assertTrue(readLog.getType() == ModifyCommentOperationLog.Type.COLUMN);
+        Assert.assertTrue(readLog.getDbId() == log.getDbId());
+        Assert.assertTrue(readLog.getTblId() == log.getTblId());
+        Assert.assertTrue(readLog.getTblComment() == null);
+        Assert.assertTrue(readLog.getColToComment().size() == 2);
+
+        // 3. delete files
+        dis.close();
+        file.delete();
+    }
+
+    @Test
+    public void testTableCommentSerialization() throws Exception {
+        // 1. Write objects to file
+        File file = new File("./ModifyTableCommentOperationLogTest");
+        file.createNewFile();
+        DataOutputStream dos = new DataOutputStream(new FileOutputStream(file));
+
+        ModifyCommentOperationLog log = ModifyCommentOperationLog.forTable(1L, 2L, "comment");
+        log.write(dos);
+
+        dos.flush();
+        dos.close();
+
+        // 2. Read objects from file
+        DataInputStream dis = new DataInputStream(new FileInputStream(file));
+
+        ModifyCommentOperationLog readLog = ModifyCommentOperationLog.read(dis);
+        Assert.assertTrue(readLog.getType() == ModifyCommentOperationLog.Type.TABLE);
+        Assert.assertTrue(readLog.getDbId() == log.getDbId());
+        Assert.assertTrue(readLog.getTblId() == log.getTblId());
+        Assert.assertEquals("comment", readLog.getTblComment());
+        Assert.assertTrue(readLog.getColToComment() == null);
+
+        // 3. delete files
+        dis.close();
+        file.delete();
+    }
+}
+
diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/ReplaceTableOperationLogTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/ReplaceTableOperationLogTest.java
index 48bb81f..b7af609 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/persist/ReplaceTableOperationLogTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/persist/ReplaceTableOperationLogTest.java
@@ -1,5 +1,3 @@
-package org.apache.doris.persist;
-
 // Licensed to the Apache Software Foundation (ASF) under one
 // or more contributor license agreements.  See the NOTICE file
 // distributed with this work for additional information
@@ -17,6 +15,8 @@ package org.apache.doris.persist;
 // specific language governing permissions and limitations
 // under the License.
 
+package org.apache.doris.persist;
+
 import org.junit.Assert;
 import org.junit.Test;
 

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org