You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2020/04/25 15:14:40 UTC

[incubator-iotdb] branch master updated: [IOTDB-617] Support alter one time series's tag/attribute property (#1096)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 8545fd7  [IOTDB-617] Support alter one time series's tag/attribute property (#1096)
8545fd7 is described below

commit 8545fd7d958eaa6a1ff57f43a82bcf049a73d7b2
Author: Jackie Tien <Ja...@foxmail.com>
AuthorDate: Sat Apr 25 23:14:29 2020 +0800

    [IOTDB-617] Support alter one time series's tag/attribute property (#1096)
    
    * alter tag
    * fix MTree limit&offset concurrent bug
    * fix LocalGroupByExecutor omit read chunk bug
---
 .../1-DDL Data Definition Language.md              |  26 ++
 .../5-Operation Manual/4-SQL Reference.md          |  17 +
 .../1-DDL Data Definition Language.md              |  24 +
 .../5-Operation Manual/4-SQL Reference.md          |  17 +
 .../org/apache/iotdb/db/qp/strategy/SqlBase.g4     |  14 +
 .../org/apache/iotdb/db/metadata/MLogWriter.java   |   6 +
 .../org/apache/iotdb/db/metadata/MManager.java     | 489 ++++++++++++++++-----
 .../java/org/apache/iotdb/db/metadata/MTree.java   | 205 +++++----
 .../iotdb/db/metadata/MetadataOperationType.java   |   1 +
 .../org/apache/iotdb/db/metadata/TagLogFile.java   |  24 +-
 .../main/java/org/apache/iotdb/db/qp/Planner.java  |   6 +-
 .../apache/iotdb/db/qp/constant/SQLConstant.java   |   7 +-
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  | 238 ++++++----
 .../org/apache/iotdb/db/qp/logical/Operator.java   |   3 +-
 .../db/qp/logical/sys/AlterTimeSeriesOperator.java |  67 +++
 .../db/qp/physical/sys/AlterTimeSeriesPlan.java    |  62 +++
 .../iotdb/db/qp/strategy/LogicalGenerator.java     |  55 +++
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    | 145 +++---
 .../dataset/groupby/LocalGroupByExecutor.java      |  94 ++--
 .../iotdb/db/query/reader/series/SeriesReader.java | 135 +++---
 .../iotdb/db/integration/IoTDBTagAlterIT.java      | 365 +++++++++++++++
 .../apache/iotdb/db/integration/IoTDBTagIT.java    |  16 +
 22 files changed, 1576 insertions(+), 440 deletions(-)

diff --git a/docs/UserGuide/5-Operation Manual/1-DDL Data Definition Language.md b/docs/UserGuide/5-Operation Manual/1-DDL Data Definition Language.md
index d1b67fe..03e1efa 100644
--- a/docs/UserGuide/5-Operation Manual/1-DDL Data Definition Language.md	
+++ b/docs/UserGuide/5-Operation Manual/1-DDL Data Definition Language.md	
@@ -87,6 +87,32 @@ The `temprature` in the brackets is an alias for the sensor `s1`. So we can use
 
 The only difference between tag and attribute is that we will maintain an inverted index on the tag, so we can use tag property in the show timeseries where clause which you can see in the following `Show Timeseries` section.
 
+
+## UPDATE TAG OPERATION
+We can update the tag information after creating it as following:
+
+* Rename the tag/attribute key
+```
+ALTER timeseries root.turbine.d1.s1 RENAME tag1 TO newTag1
+```
+* reset the tag/attribute value
+```
+ALTER timeseries root.turbine.d1.s1 SET tag1=newV1, attr1=newV1
+```
+* delete the existing tag/attribute
+```
+ALTER timeseries root.turbine.d1.s1 DROP tag1, tag2
+```
+* add new tags
+```
+ALTER timeseries root.turbine.d1.s1 ADD TAGS tag3=v3, tag4=v4
+```
+* add new attributes
+```
+ALTER timeseries root.turbine.d1.s1 ADD ATTRIBUTES attr3=v3, attr4=v4
+```
+
+
 ## Show Timeseries
 
 * SHOW TIMESERIES prefixPath? showWhereClause? limitClause?
diff --git a/docs/UserGuide/5-Operation Manual/4-SQL Reference.md b/docs/UserGuide/5-Operation Manual/4-SQL Reference.md
index 8fe78c2..75c1e61 100644
--- a/docs/UserGuide/5-Operation Manual/4-SQL Reference.md	
+++ b/docs/UserGuide/5-Operation Manual/4-SQL Reference.md	
@@ -104,6 +104,23 @@ Eg: IoTDB > DELETE TIMESERIES root.ln.wf01.wt01.status, root.ln.wf01.wt01.temper
 Eg: IoTDB > DELETE TIMESERIES root.ln.wf01.wt01.*
 ```
 
+* Alter Timeseries Statement
+```
+ALTER TIMESERIES fullPath alterClause
+alterClause
+    : RENAME beforeName=ID TO currentName=ID
+    | SET property (COMMA property)*
+    | DROP ID (COMMA ID)*
+    | ADD TAGS property (COMMA property)*
+    | ADD ATTRIBUTES property (COMMA property)*
+    ;
+Eg: ALTER timeseries root.turbine.d1.s1 RENAME tag1 TO newTag1
+Eg: ALTER timeseries root.turbine.d1.s1 SET tag1=newV1, attr1=newV1
+Eg: ALTER timeseries root.turbine.d1.s1 DROP tag1, tag2
+Eg: ALTER timeseries root.turbine.d1.s1 ADD TAGS tag3=v3, tag4=v4
+Eg: ALTER timeseries root.turbine.d1.s1 ADD ATTRIBUTES attr3=v3, attr4=v4
+```
+
 * Show All Timeseries Statement
 
 ```
diff --git a/docs/zh/UserGuide/5-Operation Manual/1-DDL Data Definition Language.md b/docs/zh/UserGuide/5-Operation Manual/1-DDL Data Definition Language.md
index 95ce913..b45956f 100644
--- a/docs/zh/UserGuide/5-Operation Manual/1-DDL Data Definition Language.md	
+++ b/docs/zh/UserGuide/5-Operation Manual/1-DDL Data Definition Language.md	
@@ -86,6 +86,30 @@ create timeseries root.turbine.d1.s1(temprature) with datatype=FLOAT, encoding=R
 
 标签和属性的唯一差别在于,我们为标签信息在内存中维护了一个倒排索引,所以可以在`show timeseries`的条件语句中使用标签作为查询条件,你将会在下一节看到具体查询内容。
 
+## 标签点属性更新
+创建时间序列后,我们也可以对其原有的标签点属性进行更新,主要有以下五种更新方式:
+
+* 重命名标签或属性
+```
+ALTER timeseries root.turbine.d1.s1 RENAME tag1 TO newTag1
+```
+* 重新设置标签或属性的值
+```
+ALTER timeseries root.turbine.d1.s1 SET tag1=newV1, attr1=newV1
+```
+* 删除已经存在的标签或属性
+```
+ALTER timeseries root.turbine.d1.s1 DROP tag1, tag2
+```
+* 添加新的标签
+```
+ALTER timeseries root.turbine.d1.s1 ADD TAGS tag3=v3, tag4=v4
+```
+* 添加新的属性
+```
+ALTER timeseries root.turbine.d1.s1 ADD ATTRIBUTES attr3=v3, attr4=v4
+```
+
 ## 查看时间序列
 
 * SHOW TIMESERIES prefixPath? showWhereClause? limitClause?
diff --git a/docs/zh/UserGuide/5-Operation Manual/4-SQL Reference.md b/docs/zh/UserGuide/5-Operation Manual/4-SQL Reference.md
index 1f06137..e58ba19 100644
--- a/docs/zh/UserGuide/5-Operation Manual/4-SQL Reference.md	
+++ b/docs/zh/UserGuide/5-Operation Manual/4-SQL Reference.md	
@@ -94,6 +94,23 @@ Eg: IoTDB > DELETE TIMESERIES root.ln.wf01.wt01.status, root.ln.wf01.wt01.temper
 Eg: IoTDB > DELETE TIMESERIES root.ln.wf01.wt01.*
 ```
 
+* 修改时间序列标签属性语句
+```
+ALTER TIMESERIES fullPath alterClause
+alterClause
+    : RENAME beforeName=ID TO currentName=ID
+    | SET property (COMMA property)*
+    | DROP ID (COMMA ID)*
+    | ADD TAGS property (COMMA property)*
+    | ADD ATTRIBUTES property (COMMA property)*
+    ;
+Eg: ALTER timeseries root.turbine.d1.s1 RENAME tag1 TO newTag1
+Eg: ALTER timeseries root.turbine.d1.s1 SET tag1=newV1, attr1=newV1
+Eg: ALTER timeseries root.turbine.d1.s1 DROP tag1, tag2
+Eg: ALTER timeseries root.turbine.d1.s1 ADD TAGS tag3=v3, tag4=v4
+Eg: ALTER timeseries root.turbine.d1.s1 ADD ATTRIBUTES attr3=v3, attr4=v4
+```
+
 * 显示所有时间序列语句
 
 ```
diff --git a/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4 b/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
index 4de9b2b..be8edf1 100644
--- a/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
+++ b/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
@@ -26,6 +26,7 @@ singleStatement
 statement
     : CREATE TIMESERIES fullPath alias? WITH attributeClauses #createTimeseries
     | DELETE TIMESERIES prefixPath (COMMA prefixPath)* #deleteTimeseries
+    | ALTER TIMESERIES fullPath alterClause #alterTimeseries
     | INSERT INTO fullPath insertColumnSpec VALUES insertValuesSpec #insertStatement
     | UPDATE prefixPath setClause whereClause? #updateStatement
     | DELETE FROM prefixPath (COMMA prefixPath)* (whereClause)? #deleteStatement
@@ -120,6 +121,14 @@ alias
     : LR_BRACKET ID RR_BRACKET
     ;
 
+alterClause
+    : RENAME beforeName=ID TO currentName=ID
+    | SET property (COMMA property)*
+    | DROP ID (COMMA ID)*
+    | ADD TAGS property (COMMA property)*
+    | ADD ATTRIBUTES property (COMMA property)*
+    ;
+
 attributeClauses
     : DATATYPE OPERATOR_EQ dataType COMMA ENCODING OPERATOR_EQ encoding
     (COMMA (COMPRESSOR | COMPRESSION) OPERATOR_EQ compressor=propertyValue)?
@@ -798,6 +807,11 @@ ATTRIBUTES
 TAGS
     : T A G S
     ;
+
+RENAME
+    : R E N A M E
+    ;
+
 //============================
 // End of the keywords list
 //============================
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MLogWriter.java b/server/src/main/java/org/apache/iotdb/db/metadata/MLogWriter.java
index 2e957a8..b4a3cb8 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MLogWriter.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MLogWriter.java
@@ -115,6 +115,12 @@ public class MLogWriter {
     writer.flush();
   }
 
+  public void changeOffset(String path, long offset) throws IOException {
+    writer.write(String.format("%s,%s,%s", MetadataOperationType.CHANGE_OFFSET, path, offset));
+    writer.newLine();
+    writer.flush();
+  }
+
   public static void upgradeMLog(String schemaDir, String logFileName) throws IOException {
     File logFile = FSFactoryProducer.getFSFactory()
         .getFile(schemaDir + File.separator + logFileName);
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
index 9c66700..a4267dd 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
@@ -86,7 +86,7 @@ public class MManager {
 
   private static class MManagerHolder {
     private MManagerHolder() {
-      //allowed to do nothing
+      // allowed to do nothing
     }
 
     private static final MManager INSTANCE = new MManager();
@@ -109,27 +109,28 @@ public class MManager {
     writeToLog = false;
 
     int cacheSize = config.getmManagerCacheSize();
-    mNodeCache = new RandomDeleteCache<String, MNode>(cacheSize) {
-
-      @Override
-      public MNode loadObjectByKey(String key) throws CacheException {
-        lock.readLock().lock();
-        try {
-          return mtree.getNodeByPathWithStorageGroupCheck(key);
-        } catch (MetadataException e) {
-          throw new CacheException(e);
-        } finally {
-          lock.readLock().unlock();
-        }
-      }
-    };
+    mNodeCache =
+        new RandomDeleteCache<String, MNode>(cacheSize) {
+
+          @Override
+          public MNode loadObjectByKey(String key) throws CacheException {
+            lock.readLock().lock();
+            try {
+              return mtree.getNodeByPathWithStorageGroupCheck(key);
+            } catch (MetadataException e) {
+              throw new CacheException(e);
+            } finally {
+              lock.readLock().unlock();
+            }
+          }
+        };
   }
 
   public static MManager getInstance() {
     return MManagerHolder.INSTANCE;
   }
 
-  //Because the writer will be used later and should not be closed here.
+  // Because the writer will be used later and should not be closed here.
   @SuppressWarnings("squid:S2093")
   public synchronized void init() {
     if (initialized) {
@@ -148,12 +149,8 @@ public class MManager {
           MNode node = mtree.getNodeByPath(sg);
           seriesNumberInStorageGroups.put(sg, node.getLeafCount());
         }
-        if (seriesNumberInStorageGroups.isEmpty()) {
-          maxSeriesNumberAmongStorageGroup = 0;
-        } else {
-          maxSeriesNumberAmongStorageGroup = seriesNumberInStorageGroups.values().stream()
-              .max(Integer::compareTo).get();
-        }
+        maxSeriesNumberAmongStorageGroup =
+            seriesNumberInStorageGroups.values().stream().max(Integer::compareTo).orElse(0);
       }
 
       logWriter = new MLogWriter(config.getSchemaDir(), MetadataConstant.METADATA_LOG);
@@ -183,9 +180,7 @@ public class MManager {
     }
   }
 
-  /**
-   * function for clearing MTree
-   */
+  /** function for clearing MTree */
   public void clear() {
     lock.writeLock().lock();
     try {
@@ -211,12 +206,12 @@ public class MManager {
   }
 
   public void operation(String cmd) throws IOException, MetadataException {
-    //see createTimeseries() to get the detailed format of the cmd
+    // see createTimeseries() to get the detailed format of the cmd
     String[] args = cmd.trim().split(",", -1);
     switch (args[0]) {
       case MetadataOperationType.CREATE_TIMESERIES:
         Map<String, String> props = new HashMap<>();
-        if (!args[5].isEmpty()){
+        if (!args[5].isEmpty()) {
           String[] keyValues = args[5].split("&");
           String[] kv;
           for (String keyValue : keyValues) {
@@ -230,7 +225,7 @@ public class MManager {
           alias = args[6];
         }
         long offset = -1L;
-        Map<String, String>  tagMap = null;
+        Map<String, String> tagMap = null;
         if (!args[7].isEmpty()) {
           offset = Long.parseLong(args[7]);
           tagMap = tagLogFile.readTag(config.getTagAttributeTotalSize(), offset);
@@ -239,8 +234,8 @@ public class MManager {
         CreateTimeSeriesPlan plan = new CreateTimeSeriesPlan(new Path(args[1]),
             TSDataType.deserialize(Short.parseShort(args[2])),
             TSEncoding.deserialize(Short.parseShort(args[3])),
-            CompressionType.deserialize(Short.parseShort(args[4])),
-            props, tagMap, null, alias);
+            CompressionType.deserialize(Short.parseShort(args[4])), props, tagMap, null, alias);
+
         createTimeseries(plan, offset);
         break;
       case MetadataOperationType.DELETE_TIMESERIES:
@@ -256,13 +251,15 @@ public class MManager {
         setStorageGroup(args[1]);
         break;
       case MetadataOperationType.DELETE_STORAGE_GROUP:
-        List<String> storageGroups = new ArrayList<>();
-        storageGroups.addAll(Arrays.asList(args).subList(1, args.length));
+        List<String> storageGroups = new ArrayList<>(Arrays.asList(args).subList(1, args.length));
         deleteStorageGroups(storageGroups);
         break;
       case MetadataOperationType.SET_TTL:
         setTTL(args[1], Long.parseLong(args[2]));
         break;
+      case MetadataOperationType.CHANGE_OFFSET:
+        changeOffset(args[1], Long.parseLong(args[2]));
+        break;
       default:
         logger.error("Unrecognizable command {}", cmd);
     }
@@ -286,8 +283,8 @@ public class MManager {
         if (!config.isAutoCreateSchemaEnabled()) {
           throw e;
         }
-        storageGroupName = MetaUtils.getStorageGroupNameByLevel(path,
-            config.getDefaultStorageGroupLevel());
+        storageGroupName =
+            MetaUtils.getStorageGroupNameByLevel(path, config.getDefaultStorageGroupLevel());
         setStorageGroup(storageGroupName);
       }
 
@@ -324,8 +321,8 @@ public class MManager {
       // write log
       if (writeToLog) {
         // either tags or attributes is not empty
-        if ((plan.getTags() != null && !plan.getTags().isEmpty()) || (plan.getAttributes() != null
-            && !plan.getAttributes().isEmpty())) {
+        if ((plan.getTags() != null && !plan.getTags().isEmpty())
+            || (plan.getAttributes() != null && !plan.getAttributes().isEmpty())) {
           offset = tagLogFile.write(plan.getTags(), plan.getAttributes());
         }
         logWriter.createTimeseries(plan, offset);
@@ -347,16 +344,20 @@ public class MManager {
    * @param encoding the encoding function {@code Encoding} of the timeseries
    * @param compressor the compressor function {@code Compressor} of the time series
    * @return whether the measurement occurs for the first time in this storage group (if true, the
-   * measurement should be registered to the StorageEngine too)
+   *     measurement should be registered to the StorageEngine too)
    */
-  public void createTimeseries(String path, TSDataType dataType, TSEncoding encoding,
-      CompressionType compressor, Map<String, String> props) throws MetadataException {
+  public void createTimeseries(
+      String path,
+      TSDataType dataType,
+      TSEncoding encoding,
+      CompressionType compressor,
+      Map<String, String> props)
+      throws MetadataException {
     createTimeseries(
-        new CreateTimeSeriesPlan(new Path(path), dataType, encoding, compressor, props, null, null,
-            null));
+        new CreateTimeSeriesPlan(
+            new Path(path), dataType, encoding, compressor, props, null, null, null));
   }
 
-
   /**
    * Delete all timeseries under the given path, may cross different storage group
    *
@@ -373,7 +374,8 @@ public class MManager {
         int size = seriesNumberInStorageGroups.get(prefixPath);
         seriesNumberInStorageGroups.put(prefixPath, 0);
         if (size == maxSeriesNumberAmongStorageGroup) {
-          seriesNumberInStorageGroups.values().stream().max(Integer::compareTo)
+          seriesNumberInStorageGroups.values().stream()
+              .max(Integer::compareTo)
               .ifPresent(val -> maxSeriesNumberAmongStorageGroup = val);
         }
       }
@@ -408,6 +410,7 @@ public class MManager {
 
   /**
    * remove the node from the tag inverted index
+   *
    * @param node
    * @throws IOException
    */
@@ -415,7 +418,8 @@ public class MManager {
     if (node.getOffset() < 0) {
       return;
     }
-    Map<String, String> tagMap = tagLogFile.readTag(config.getTagAttributeTotalSize(), node.getOffset());
+    Map<String, String> tagMap =
+        tagLogFile.readTag(config.getTagAttributeTotalSize(), node.getOffset());
     if (tagMap != null) {
       for (Entry<String, String> entry : tagMap.entrySet()) {
         tagIndex.get(entry.getKey()).get(entry.getValue()).remove(node);
@@ -448,7 +452,8 @@ public class MManager {
         int size = seriesNumberInStorageGroups.get(storageGroup);
         seriesNumberInStorageGroups.put(storageGroup, size - 1);
         if (size == maxSeriesNumberAmongStorageGroup) {
-          seriesNumberInStorageGroups.values().stream().max(Integer::compareTo)
+          seriesNumberInStorageGroups.values().stream()
+              .max(Integer::compareTo)
               .ifPresent(val -> maxSeriesNumberAmongStorageGroup = val);
         }
       }
@@ -510,12 +515,8 @@ public class MManager {
           ActiveTimeSeriesCounter.getInstance().delete(storageGroup);
           seriesNumberInStorageGroups.remove(storageGroup);
           if (size == maxSeriesNumberAmongStorageGroup) {
-            if (seriesNumberInStorageGroups.isEmpty()) {
-              maxSeriesNumberAmongStorageGroup = 0;
-            } else {
-              maxSeriesNumberAmongStorageGroup = seriesNumberInStorageGroups.values().stream()
-                  .max(Integer::compareTo).get();
-            }
+            maxSeriesNumberAmongStorageGroup =
+                seriesNumberInStorageGroups.values().stream().max(Integer::compareTo).orElse(0);
           }
         }
         // if success
@@ -586,7 +587,7 @@ public class MManager {
    * Get all devices under given prefixPath.
    *
    * @param prefixPath a prefix of a full path. if the wildcard is not at the tail, then each
-   * wildcard can only match one level, otherwise it can match to the tail.
+   *     wildcard can only match one level, otherwise it can match to the tail.
    * @return A HashSet instance which stores devices names with given prefixPath.
    */
   public Set<String> getDevices(String prefixPath) throws MetadataException {
@@ -602,8 +603,8 @@ public class MManager {
    * Get all nodes from the given level
    *
    * @param prefixPath can be a prefix of a full path. Can not be a full path. can not have
-   * wildcard. But, the level of the prefixPath can be smaller than the given level, e.g.,
-   * prefixPath = root.a while the given level is 5
+   *     wildcard. But, the level of the prefixPath can be smaller than the given level, e.g.,
+   *     prefixPath = root.a while the given level is 5
    * @param nodeLevel the level can not be smaller than the level of the prefixPath
    * @return A List instance which stores all node at given level
    */
@@ -619,7 +620,7 @@ public class MManager {
   /**
    * Get storage group name by path
    *
-   * e.g., root.sg1 is a storage group and path = root.sg1.d1, return root.sg1
+   * <p>e.g., root.sg1 is a storage group and path = root.sg1.d1, return root.sg1
    *
    * @return storage group in the given path
    */
@@ -632,9 +633,7 @@ public class MManager {
     }
   }
 
-  /**
-   * Get all storage group names
-   */
+  /** Get all storage group names */
   public List<String> getAllStorageGroupNames() {
     lock.readLock().lock();
     try {
@@ -644,9 +643,7 @@ public class MManager {
     }
   }
 
-  /**
-   * Get all storage group MNodes
-   */
+  /** Get all storage group MNodes */
   public List<StorageGroupMNode> getAllStorageGroupNodes() {
     lock.readLock().lock();
     try {
@@ -661,7 +658,7 @@ public class MManager {
    * expression in this method is formed by the amalgamation of seriesPath and the character '*'.
    *
    * @param prefixPath can be a prefix or a full path. if the wildcard is not at the tail, then each
-   * wildcard can only match one level, otherwise it can match to the tail.
+   *     wildcard can only match one level, otherwise it can match to the tail.
    */
   public List<String> getAllTimeseriesName(String prefixPath) throws MetadataException {
     lock.readLock().lock();
@@ -682,6 +679,9 @@ public class MManager {
         throw new MetadataException("The key " + plan.getKey() + " is not a tag.");
       }
       Map<String, Set<LeafMNode>> value2Node = tagIndex.get(plan.getKey());
+      if (value2Node.isEmpty()) {
+        throw new MetadataException("The key " + plan.getKey() + " is not a tag.");
+      }
       Set<LeafMNode> allMatchedNodes = new TreeSet<>(Comparator.comparing(MNode::getFullPath));
       if (plan.isContains()) {
         for (Entry<String, Set<LeafMNode>> entry : value2Node.entrySet()) {
@@ -707,20 +707,20 @@ public class MManager {
       for (LeafMNode leaf : allMatchedNodes) {
         if (match(leaf.getFullPath(), prefixNodes)) {
           if (limit != 0 || offset != 0) {
-            curOffset ++;
+            curOffset++;
             if (curOffset < offset || count == limit) {
               continue;
             }
           }
           try {
             Pair<Map<String, String>, Map<String, String>> pair =
-                    tagLogFile.read(config.getTagAttributeTotalSize(), leaf.getOffset());
+                tagLogFile.read(config.getTagAttributeTotalSize(), leaf.getOffset());
             pair.left.putAll(pair.right);
             MeasurementSchema measurementSchema = leaf.getSchema();
             res.add(new ShowTimeSeriesResult(leaf.getFullPath(), leaf.getAlias(),
-                    getStorageGroupName(leaf.getFullPath()), measurementSchema.getType().toString(),
-                    measurementSchema.getEncodingType().toString(),
-                    measurementSchema.getCompressor().toString(), pair.left));
+                getStorageGroupName(leaf.getFullPath()), measurementSchema.getType().toString(),
+                measurementSchema.getEncodingType().toString(),
+                measurementSchema.getCompressor().toString(), pair.left));
             if (limit != 0 || offset != 0) {
               count++;
             }
@@ -731,14 +731,12 @@ public class MManager {
         }
       }
       return res;
-    }  finally {
+    } finally {
       lock.readLock().unlock();
     }
   }
 
-  /**
-   * whether the full path has the prefixNodes
-   */
+  /** whether the full path has the prefixNodes */
   private boolean match(String fullPath, String[] prefixNodes) {
     String[] nodes = MetaUtils.getNodeNames(fullPath);
     if (nodes.length < prefixNodes.length) {
@@ -757,7 +755,8 @@ public class MManager {
    *
    * @param plan show time series query plan
    */
-  public List<ShowTimeSeriesResult> showTimeseries(ShowTimeSeriesPlan plan) throws MetadataException {
+  public List<ShowTimeSeriesResult> showTimeseries(ShowTimeSeriesPlan plan)
+      throws MetadataException {
     lock.readLock().lock();
     try {
       List<String[]> ans = mtree.getAllMeasurementSchema(plan);
@@ -767,15 +766,15 @@ public class MManager {
         try {
           if (tagFileOffset < 0) {
             // no tags/attributes
-            res.add(new ShowTimeSeriesResult(ansString[0], ansString[1], ansString[2],
-                ansString[3], ansString[4], ansString[5], Collections.emptyMap()));
+            res.add(new ShowTimeSeriesResult(ansString[0], ansString[1], ansString[2], ansString[3],
+                ansString[4], ansString[5], Collections.emptyMap()));
           } else {
             // has tags/attributes
             Pair<Map<String, String>, Map<String, String>> pair =
                 tagLogFile.read(config.getTagAttributeTotalSize(), tagFileOffset);
             pair.left.putAll(pair.right);
-            res.add(new ShowTimeSeriesResult(ansString[0], ansString[1], ansString[2],
-                ansString[3], ansString[4], ansString[5], pair.left));
+            res.add(new ShowTimeSeriesResult(ansString[0], ansString[1], ansString[2], ansString[3],
+                ansString[4], ansString[5], pair.left));
           }
         } catch (IOException e) {
           throw new MetadataException(
@@ -783,14 +782,13 @@ public class MManager {
         }
       }
       return res;
-    }  finally {
+    } finally {
       lock.readLock().unlock();
     }
   }
 
-
-
-  public MeasurementSchema getSeriesSchema(String device, String measuremnet) throws MetadataException {
+  public MeasurementSchema getSeriesSchema(String device, String measuremnet)
+      throws MetadataException {
     lock.readLock().lock();
     try {
       InternalMNode node = (InternalMNode) mtree.getNodeByPath(device);
@@ -803,8 +801,8 @@ public class MManager {
   /**
    * Get child node path in the next level of the given path.
    *
-   * e.g., MTree has [root.sg1.d1.s1, root.sg1.d1.s2, root.sg1.d2.s1] given path = root.sg1, return
-   * [root.sg1.d1, root.sg1.d2]
+   * <p>e.g., MTree has [root.sg1.d1.s1, root.sg1.d1.s2, root.sg1.d2.s1] given path = root.sg1,
+   * return [root.sg1.d1, root.sg1.d2]
    *
    * @return All child nodes' seriesPath(s) of given seriesPath.
    */
@@ -831,9 +829,7 @@ public class MManager {
     }
   }
 
-  /**
-   * Get node by path
-   */
+  /** Get node by path */
   public MNode getNodeByPath(String path) throws MetadataException {
     lock.readLock().lock();
     try {
@@ -918,9 +914,7 @@ public class MManager {
         config.getDefaultStorageGroupLevel());
   }
 
-  /**
-   * Get metadata in string
-   */
+  /** Get metadata in string */
   public String getMetadataInString() {
     lock.readLock().lock();
     try {
@@ -952,8 +946,284 @@ public class MManager {
   }
 
   /**
-   * Check whether the given path contains a storage group
+   * change or set the new offset of a timeseries
+   * @param path timeseries
+   * @param offset offset in the tag file
+   */
+  public void changeOffset(String path, long offset) throws MetadataException {
+    lock.writeLock().lock();
+    try {
+      ((LeafMNode) mtree.getNodeByPath(path)).setOffset(offset);
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  /**
+   * add new attributes key-value for the timeseries
+   * @param attributesMap newly added attributes map
+   * @param fullPath timeseries
+   */
+  public void addAttributes(Map<String, String> attributesMap, String fullPath)
+      throws MetadataException, IOException {
+    lock.writeLock().lock();
+    try {
+      MNode mNode = mtree.getNodeByPath(fullPath);
+      if (!(mNode instanceof LeafMNode)) {
+        throw new PathNotExistException(fullPath);
+      }
+      LeafMNode leafMNode = (LeafMNode) mNode;
+      // no tag or attribute, we need to add a new record in log
+      if (leafMNode.getOffset() < 0) {
+        long offset = tagLogFile.write(Collections.emptyMap(), attributesMap);
+        logWriter.changeOffset(fullPath, offset);
+        leafMNode.setOffset(offset);
+        return;
+      }
+
+      Pair<Map<String, String>, Map<String, String>> pair =
+          tagLogFile.read(config.getTagAttributeTotalSize(), leafMNode.getOffset());
+
+      for (Entry<String, String> entry : attributesMap.entrySet()) {
+        String key = entry.getKey();
+        String value = entry.getValue();
+        if (pair.right.containsKey(key)) {
+          throw new MetadataException(
+              String.format("TimeSeries [%s] already has the attribute [%s].", fullPath, key));
+        }
+        pair.right.put(key, value);
+      }
+
+      // persist the change to disk
+      tagLogFile.write(pair.left, pair.right, leafMNode.getOffset());
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  /**
+   * add new tags key-value for the timeseries
+   * @param tagsMap newly added tags map
+   * @param fullPath timeseries
+   */
+  public void addTags(Map<String, String> tagsMap, String fullPath)
+      throws MetadataException, IOException {
+    lock.writeLock().lock();
+    try {
+      MNode mNode = mtree.getNodeByPath(fullPath);
+      if (!(mNode instanceof LeafMNode)) {
+        throw new PathNotExistException(fullPath);
+      }
+      LeafMNode leafMNode = (LeafMNode) mNode;
+      // no tag or attribute, we need to add a new record in log
+      if (leafMNode.getOffset() < 0) {
+        long offset = tagLogFile.write(tagsMap, Collections.emptyMap());
+        logWriter.changeOffset(fullPath, offset);
+        leafMNode.setOffset(offset);
+        return;
+      }
+
+      Pair<Map<String, String>, Map<String, String>> pair =
+          tagLogFile.read(config.getTagAttributeTotalSize(), leafMNode.getOffset());
+
+      for (Entry<String, String> entry : tagsMap.entrySet()) {
+        String key = entry.getKey();
+        String value = entry.getValue();
+        if (pair.left.containsKey(key)) {
+          throw new MetadataException(
+              String.format("TimeSeries [%s] already has the tag [%s].", fullPath, key));
+        }
+        pair.left.put(key, value);
+      }
+
+      // persist the change to disk
+      tagLogFile.write(pair.left, pair.right, leafMNode.getOffset());
+
+      // update tag inverted map
+      tagsMap.forEach(
+          (key, value) ->
+              tagIndex
+                  .computeIfAbsent(key, k -> new HashMap<>())
+                  .computeIfAbsent(value, v -> new HashSet<>())
+                  .add(leafMNode));
+
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  /**
+   * drop tags or attributes of the timeseries
+   * @param keySet tags key or attributes key
+   * @param fullPath timeseries path
+   */
+  public void dropTagsOrAttributes(Set<String> keySet, String fullPath)
+      throws MetadataException, IOException {
+    lock.writeLock().lock();
+    try {
+      MNode mNode = mtree.getNodeByPath(fullPath);
+      if (!(mNode instanceof LeafMNode)) {
+        throw new PathNotExistException(fullPath);
+      }
+      LeafMNode leafMNode = (LeafMNode) mNode;
+      // no tag or attribute, just do nothing.
+      if (leafMNode.getOffset() < 0) {
+        return;
+      }
+      Pair<Map<String, String>, Map<String, String>> pair =
+          tagLogFile.read(config.getTagAttributeTotalSize(), leafMNode.getOffset());
+
+      Map<String, String> deleteTag = new HashMap<>();
+      for (String key : keySet) {
+        // check tag map
+        // check attribute map
+        if (pair.left.containsKey(key)) {
+          deleteTag.put(key, pair.left.remove(key));
+        } else {
+          pair.right.remove(key);
+        }
+      }
+
+      // persist the change to disk
+      tagLogFile.write(pair.left, pair.right, leafMNode.getOffset());
+
+      for (Entry<String, String> entry : deleteTag.entrySet()) {
+        String key = entry.getKey();
+        String value = entry.getValue();
+        // change the tag inverted index map
+        tagIndex.get(key).get(value).remove(leafMNode);
+        if (tagIndex.get(key).get(value).isEmpty()) {
+          tagIndex.get(key).remove(value);
+          if (tagIndex.get(key).isEmpty()) {
+            tagIndex.remove(key);
+          }
+        }
+      }
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  /**
+   * set/change the values of tags or attributes
+   * @param alterMap the new tags or attributes key-value
+   * @param fullPath timeseries
    */
+  public void setTagsOrAttributesValue(Map<String, String> alterMap, String fullPath)
+      throws MetadataException, IOException {
+    lock.writeLock().lock();
+    try {
+      MNode mNode = mtree.getNodeByPath(fullPath);
+      if (!(mNode instanceof LeafMNode)) {
+        throw new PathNotExistException(fullPath);
+      }
+      LeafMNode leafMNode = (LeafMNode) mNode;
+      if (leafMNode.getOffset() < 0) {
+        throw new MetadataException(
+            String.format("TimeSeries [%s] does not have any tag/attribute.", fullPath));
+      }
+
+      // tags, attributes
+      Pair<Map<String, String>, Map<String, String>> pair =
+          tagLogFile.read(config.getTagAttributeTotalSize(), leafMNode.getOffset());
+      Map<String, String> oldTagValue = new HashMap<>();
+      Map<String, String> newTagValue = new HashMap<>();
+
+      for (Entry<String, String> entry : alterMap.entrySet()) {
+        String key = entry.getKey();
+        String value = entry.getValue();
+        // check tag map
+        if (pair.left.containsKey(key)) {
+          oldTagValue.put(key, pair.left.get(key));
+          newTagValue.put(key, value);
+          pair.left.put(key, value);
+        } else if (pair.right.containsKey(key)) {
+          // check attribute map
+          pair.right.put(key, value);
+        } else {
+          throw new MetadataException(
+              String.format("TimeSeries [%s] does not have tag/attribute [%s].", fullPath, key));
+        }
+      }
+
+      // persist the change to disk
+      tagLogFile.write(pair.left, pair.right, leafMNode.getOffset());
+
+      for (Entry<String, String> entry : oldTagValue.entrySet()) {
+        String key = entry.getKey();
+        String beforeValue = entry.getValue();
+        String currentValue = newTagValue.get(key);
+        // change the tag inverted index map
+        tagIndex.get(key).get(beforeValue).remove(leafMNode);
+        tagIndex
+            .computeIfAbsent(key, k -> new HashMap<>())
+            .computeIfAbsent(currentValue, k -> new HashSet<>())
+            .add(leafMNode);
+      }
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  /**
+   * rename the tag or attribute's key of the timeseries
+   * @param oldKey old key of tag or attribute
+   * @param newKey new key of tag or attribute
+   * @param fullPath timeseries
+   */
+  public void renameTagOrAttributeKey(String oldKey, String newKey, String fullPath)
+      throws MetadataException, IOException {
+    lock.writeLock().lock();
+    try {
+      MNode mNode = mtree.getNodeByPath(fullPath);
+      if (!(mNode instanceof LeafMNode)) {
+        throw new PathNotExistException(fullPath);
+      }
+      LeafMNode leafMNode = (LeafMNode) mNode;
+      if (leafMNode.getOffset() < 0) {
+        throw new MetadataException(
+            String.format(
+                "TimeSeries [%s] does not have [%s] tag/attribute.", fullPath, oldKey));
+      }
+      // tags, attributes
+      Pair<Map<String, String>, Map<String, String>> pair =
+          tagLogFile.read(config.getTagAttributeTotalSize(), leafMNode.getOffset());
+
+      // current name has existed
+      if (pair.left.containsKey(newKey) || pair.right.containsKey(newKey)) {
+        throw new MetadataException(
+            String.format(
+                "TimeSeries [%s] already has a tag/attribute named [%s].", fullPath, newKey));
+      }
+
+      // check tag map
+      if (pair.left.containsKey(oldKey)) {
+        String value = pair.left.remove(oldKey);
+        pair.left.put(newKey, value);
+        // persist the change to disk
+        tagLogFile.write(pair.left, pair.right, leafMNode.getOffset());
+        // change the tag inverted index map
+        tagIndex.get(oldKey).get(value).remove(leafMNode);
+        tagIndex
+            .computeIfAbsent(newKey, k -> new HashMap<>())
+            .computeIfAbsent(value, k -> new HashSet<>())
+            .add(leafMNode);
+      } else if (pair.right.containsKey(oldKey)) {
+        // check attribute map
+        pair.right.put(newKey, pair.right.remove(oldKey));
+        // persist the change to disk
+        tagLogFile.write(pair.left, pair.right, leafMNode.getOffset());
+      } else {
+        throw new MetadataException(
+            String.format(
+                "TimeSeries [%s] does not have tag/attribute [%s].", fullPath, oldKey));
+      }
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  /** Check whether the given path contains a storage group */
   boolean checkStorageGroupByPath(String path) {
     lock.readLock().lock();
     try {
@@ -963,7 +1233,6 @@ public class MManager {
     }
   }
 
-
   /**
    * Get all storage groups under the given path
    *
@@ -988,8 +1257,8 @@ public class MManager {
       MNode node = nodeDeque.removeFirst();
       if (node instanceof LeafMNode) {
         MeasurementSchema nodeSchema = ((LeafMNode) node).getSchema();
-        timeseriesSchemas.add(new MeasurementSchema(node.getFullPath(),
-            nodeSchema.getType(), nodeSchema.getEncodingType(), nodeSchema.getCompressor()));
+        timeseriesSchemas.add(new MeasurementSchema(node.getFullPath(), nodeSchema.getType(),
+            nodeSchema.getEncodingType(), nodeSchema.getCompressor()));
       } else if (!node.getChildren().isEmpty()) {
         nodeDeque.addAll(node.getChildren().values());
       }
@@ -1007,31 +1276,25 @@ public class MManager {
   }
 
   /**
-   * For a path, infer all storage groups it may belong to.
-   * The path can have wildcards.
+   * For a path, infer all storage groups it may belong to. The path can have wildcards.
    *
-   * Consider the path into two parts: (1) the sub path which can not contain a storage group name and
-   * (2) the sub path which is substring that begin after the storage group name.
+   * <p>Consider the path into two parts: (1) the sub path which can not contain a storage group
+   * name and (2) the sub path which is substring that begin after the storage group name.
    *
-   * (1) Suppose the part of the path can not contain a storage group name (e.g.,
-   * "root".contains("root.sg") == false), then:
-   * If the wildcard is not at the tail, then for each wildcard, only one level will be inferred
-   * and the wildcard will be removed.
-   * If the wildcard is at the tail, then the inference will go on until the storage groups are found
-   * and the wildcard will be kept.
-   * (2) Suppose the part of the path is a substring that begin after the storage group name. (e.g.,
-   *  For "root.*.sg1.a.*.b.*" and "root.x.sg1" is a storage group, then this part is "a.*.b.*").
-   *  For this part, keep what it is.
+   * <p>(1) Suppose the part of the path can not contain a storage group name (e.g.,
+   * "root".contains("root.sg") == false), then: If the wildcard is not at the tail, then for each
+   * wildcard, only one level will be inferred and the wildcard will be removed. If the wildcard is
+   * at the tail, then the inference will go on until the storage groups are found and the wildcard
+   * will be kept. (2) Suppose the part of the path is a substring that begin after the storage
+   * group name. (e.g., For "root.*.sg1.a.*.b.*" and "root.x.sg1" is a storage group, then this part
+   * is "a.*.b.*"). For this part, keep what it is.
    *
-   * Assuming we have three SGs: root.group1, root.group2, root.area1.group3
-   * Eg1:
-   *  for input "root.*", returns ("root.group1", "root.group1.*"), ("root.group2", "root.group2.*")
-   *  ("root.area1.group3", "root.area1.group3.*")
-   * Eg2:
-   *  for input "root.*.s1", returns ("root.group1", "root.group1.s1"), ("root.group2", "root.group2.s1")
+   * <p>Assuming we have three SGs: root.group1, root.group2, root.area1.group3 Eg1: for input
+   * "root.*", returns ("root.group1", "root.group1.*"), ("root.group2", "root.group2.*")
+   * ("root.area1.group3", "root.area1.group3.*") Eg2: for input "root.*.s1", returns
+   * ("root.group1", "root.group1.s1"), ("root.group2", "root.group2.s1")
    *
-   * Eg3:
-   *  for input "root.area1.*", returns ("root.area1.group3", "root.area1.group3.*")
+   * <p>Eg3: for input "root.area1.*", returns ("root.area1.group3", "root.area1.group3.*")
    *
    * @param path can be a prefix or a full path.
    * @return StorageGroupName-FullPath pairs
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
index 99744d6..3deceb3 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java
@@ -18,12 +18,33 @@
  */
 package org.apache.iotdb.db.metadata;
 
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_WILDCARD;
+
 import com.alibaba.fastjson.JSON;
 import com.alibaba.fastjson.JSONObject;
 import com.alibaba.fastjson.serializer.SerializerFeature;
+import java.io.Serializable;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.regex.Pattern;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.metadata.*;
+import org.apache.iotdb.db.exception.metadata.AliasAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.IllegalPathException;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException;
+import org.apache.iotdb.db.exception.metadata.PathNotExistException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupAlreadySetException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
 import org.apache.iotdb.db.metadata.mnode.InternalMNode;
 import org.apache.iotdb.db.metadata.mnode.LeafMNode;
 import org.apache.iotdb.db.metadata.mnode.MNode;
@@ -37,26 +58,16 @@ import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 
-import java.io.Serializable;
-import java.util.*;
-import java.util.Map.Entry;
-import java.util.regex.Pattern;
-
-import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
-import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_WILDCARD;
-
-/**
- * The hierarchical struct of the Metadata Tree is implemented in this class.
- */
+/** The hierarchical struct of the Metadata Tree is implemented in this class. */
 public class MTree implements Serializable {
 
   private static final long serialVersionUID = -4200394435237291964L;
   private MNode root;
 
-  private int limit = 0;
-  private int offset = 0;
-  private int count = 0;
-  private int curOffset = -1;
+  private transient ThreadLocal<Integer> limit = new ThreadLocal<>();
+  private transient ThreadLocal<Integer> offset = new ThreadLocal<>();
+  private transient ThreadLocal<Integer> count = new ThreadLocal<>();
+  private transient ThreadLocal<Integer> curOffset = new ThreadLocal<>();
 
   MTree() {
     this.root = new InternalMNode(null, IoTDBConstant.PATH_ROOT);
@@ -73,8 +84,14 @@ public class MTree implements Serializable {
    * @param props props
    * @param alias alias of measurement
    */
-  LeafMNode createTimeseries(String path, TSDataType dataType, TSEncoding encoding,
-      CompressionType compressor, Map<String, String> props, String alias) throws MetadataException {
+  LeafMNode createTimeseries(
+      String path,
+      TSDataType dataType,
+      TSEncoding encoding,
+      CompressionType compressor,
+      Map<String, String> props,
+      String alias)
+      throws MetadataException {
     String[] nodeNames = MetaUtils.getNodeNames(path);
     if (nodeNames.length <= 2 || !nodeNames[0].equals(root.getName())) {
       throw new IllegalPathException(path);
@@ -117,7 +134,7 @@ public class MTree implements Serializable {
   /**
    * Add an interval path to MTree. This is only used for automatically creating schema
    *
-   * e.g., get root.sg.d1, get or create all internal nodes and return the node of d1
+   * <p>e.g., get root.sg.d1, get or create all internal nodes and return the node of d1
    */
   MNode getDeviceNodeWithAutoCreating(String deviceId) throws MetadataException {
     String[] nodeNames = MetaUtils.getNodeNames(deviceId);
@@ -184,15 +201,14 @@ public class MTree implements Serializable {
       // node b has child sg
       throw new StorageGroupAlreadySetException(path);
     } else {
-      StorageGroupMNode storageGroupMNode = new StorageGroupMNode(cur, nodeNames[i], path,
-          IoTDBDescriptor.getInstance().getConfig().getDefaultTTL());
+      StorageGroupMNode storageGroupMNode =
+          new StorageGroupMNode(
+              cur, nodeNames[i], path, IoTDBDescriptor.getInstance().getConfig().getDefaultTTL());
       cur.addChild(nodeNames[i], storageGroupMNode);
     }
   }
 
-  /**
-   * Delete a storage group
-   */
+  /** Delete a storage group */
   void deleteStorageGroup(String path) throws MetadataException {
     MNode cur = getNodeByPath(path);
     if (!(cur instanceof StorageGroupMNode)) {
@@ -212,7 +228,7 @@ public class MTree implements Serializable {
   /**
    * Check whether path is storage group or not
    *
-   * e.g., path = root.a.b.sg. if nor a and b is StorageGroupMNode and sg is a StorageGroupMNode
+   * <p>e.g., path = root.a.b.sg. if nor a and b is StorageGroupMNode and sg is a StorageGroupMNode
    * path is a storage group
    *
    * @param path path
@@ -241,8 +257,8 @@ public class MTree implements Serializable {
    *
    * @param path Format: root.node(.node)+
    */
-
-  Pair<String, LeafMNode> deleteTimeseriesAndReturnEmptyStorageGroup(String path) throws MetadataException {
+  Pair<String, LeafMNode> deleteTimeseriesAndReturnEmptyStorageGroup(String path)
+      throws MetadataException {
     MNode curNode = getNodeByPath(path);
     if (!(curNode instanceof LeafMNode)) {
       throw new PathNotExistException(path);
@@ -255,7 +271,7 @@ public class MTree implements Serializable {
     curNode.getParent().deleteChild(curNode.getName());
     LeafMNode deletedNode = (LeafMNode) curNode;
     if (deletedNode.getAlias() != null) {
-      curNode.getParent().deleteAliasChild(((LeafMNode)curNode).getAlias());
+      curNode.getParent().deleteAliasChild(((LeafMNode) curNode).getAlias());
     }
     curNode = curNode.getParent();
     // delete all empty ancestors except storage group
@@ -311,9 +327,7 @@ public class MTree implements Serializable {
     return cur;
   }
 
-  /**
-   * Get storage group node, if the give path is not a storage group, throw exception
-   */
+  /** Get storage group node, if the give path is not a storage group, throw exception */
   StorageGroupMNode getStorageGroupNode(String path) throws MetadataException {
     MNode node = getNodeByPath(path);
     if (node instanceof StorageGroupMNode) {
@@ -323,9 +337,7 @@ public class MTree implements Serializable {
     }
   }
 
-  /**
-   * Get device node, if the give path is not a device, throw exception
-   */
+  /** Get device node, if the give path is not a device, throw exception */
   MNode getDeviceNode(String path) throws MetadataException {
     return getNodeByPath(path);
   }
@@ -371,8 +383,8 @@ public class MTree implements Serializable {
    *
    * @apiNote :for cluster
    */
-  private void findStorageGroup(MNode node, String[] nodes, int idx, String parent,
-      List<String> storageGroupNames) {
+  private void findStorageGroup(
+      MNode node, String[] nodes, int idx, String parent, List<String> storageGroupNames) {
     if (node instanceof StorageGroupMNode) {
       storageGroupNames.add(node.getFullPath());
       return;
@@ -380,13 +392,17 @@ public class MTree implements Serializable {
     String nodeReg = MetaUtils.getNodeRegByIdx(idx, nodes);
     if (!(PATH_WILDCARD).equals(nodeReg)) {
       if (node.hasChild(nodeReg)) {
-        findStorageGroup(node.getChild(nodeReg), nodes, idx + 1,
-            parent + node.getName() + PATH_SEPARATOR, storageGroupNames);
+        findStorageGroup(
+            node.getChild(nodeReg),
+            nodes,
+            idx + 1,
+            parent + node.getName() + PATH_SEPARATOR,
+            storageGroupNames);
       }
     } else {
       for (MNode child : node.getChildren().values()) {
-        findStorageGroup(child, nodes, idx + 1, parent + node.getName() + PATH_SEPARATOR,
-            storageGroupNames);
+        findStorageGroup(
+            child, nodes, idx + 1, parent + node.getName() + PATH_SEPARATOR, storageGroupNames);
       }
     }
   }
@@ -411,9 +427,7 @@ public class MTree implements Serializable {
     return res;
   }
 
-  /**
-   * Get all storage group MNodes
-   */
+  /** Get all storage group MNodes */
   List<StorageGroupMNode> getAllStorageGroupNodes() {
     List<StorageGroupMNode> ret = new ArrayList<>();
     Deque<MNode> nodeStack = new ArrayDeque<>();
@@ -432,7 +446,7 @@ public class MTree implements Serializable {
   /**
    * Get storage group name by path
    *
-   * e.g., root.sg1 is storage group, path is root.sg1.d1, return root.sg1
+   * <p>e.g., root.sg1 is storage group, path is root.sg1.d1, return root.sg1
    *
    * @return storage group in the given path
    */
@@ -450,9 +464,7 @@ public class MTree implements Serializable {
     throw new StorageGroupNotSetException(path);
   }
 
-  /**
-   * Check whether the given path contains a storage group
-   */
+  /** Check whether the given path contains a storage group */
   boolean checkStorageGroupByPath(String path) {
     String[] nodes = MetaUtils.getNodeNames(path);
     MNode cur = root;
@@ -485,7 +497,7 @@ public class MTree implements Serializable {
   /**
    * Get all time series schema under the given path
    *
-   * result: [name, alias, storage group, dataType, encoding, compression, offset]
+   * <p>result: [name, alias, storage group, dataType, encoding, compression, offset]
    */
   List<String[]> getAllMeasurementSchema(ShowTimeSeriesPlan plan) throws MetadataException {
     List<String[]> res;
@@ -493,17 +505,22 @@ public class MTree implements Serializable {
     if (nodes.length == 0 || !nodes[0].equals(root.getName())) {
       throw new IllegalPathException(plan.getPath().getFullPath());
     }
-    this.limit = plan.getLimit();
-    this.offset = plan.getOffset();
-    if (offset != 0 || limit != 0) {
-      res = new ArrayList<>(limit);
+    limit.set(plan.getLimit());
+    offset.set(plan.getOffset());
+    curOffset.set(-1);
+    count.set(0);
+    if (offset.get() != 0 || limit.get() != 0) {
+      res = new ArrayList<>(limit.get());
       findPath(root, nodes, 1, "", res, true);
-      curOffset = -1;
-      count = 0;
     } else {
       res = new ArrayList<>();
       findPath(root, nodes, 1, "", res, false);
     }
+    // avoid memory leaks
+    limit.remove();
+    offset.remove();
+    curOffset.remove();
+    count.remove();
     return res;
   }
 
@@ -512,13 +529,19 @@ public class MTree implements Serializable {
    *
    * @param timeseriesSchemaList List<timeseriesSchema>
    */
-  private void findPath(MNode node, String[] nodes, int idx, String parent,
-      List<String[]> timeseriesSchemaList, boolean hasLimit) throws MetadataException {
+  private void findPath(
+      MNode node,
+      String[] nodes,
+      int idx,
+      String parent,
+      List<String[]> timeseriesSchemaList,
+      boolean hasLimit)
+      throws MetadataException {
     if (node instanceof LeafMNode) {
       if (nodes.length <= idx) {
         if (hasLimit) {
-          curOffset++;
-          if (curOffset < offset || count == limit) {
+          curOffset.set(curOffset.get() + 1);
+          if (curOffset.get() < offset.get() || count == limit) {
             return;
           }
         }
@@ -541,7 +564,7 @@ public class MTree implements Serializable {
         timeseriesSchemaList.add(tsRow);
 
         if (hasLimit) {
-          count++;
+          count.set(count.get() + 1);
         }
       }
       return;
@@ -549,16 +572,26 @@ public class MTree implements Serializable {
     String nodeReg = MetaUtils.getNodeRegByIdx(idx, nodes);
     if (!nodeReg.contains(PATH_WILDCARD)) {
       if (node.hasChild(nodeReg)) {
-        findPath(node.getChild(nodeReg), nodes, idx + 1, parent + node.getName() + PATH_SEPARATOR,
-            timeseriesSchemaList, hasLimit);
+        findPath(
+            node.getChild(nodeReg),
+            nodes,
+            idx + 1,
+            parent + node.getName() + PATH_SEPARATOR,
+            timeseriesSchemaList,
+            hasLimit);
       }
     } else {
       for (MNode child : node.getChildren().values()) {
         if (!Pattern.matches(nodeReg.replace("*", ".*"), child.getName())) {
           continue;
         }
-        findPath(child, nodes, idx + 1, parent + node.getName() + PATH_SEPARATOR,
-            timeseriesSchemaList, hasLimit);
+        findPath(
+            child,
+            nodes,
+            idx + 1,
+            parent + node.getName() + PATH_SEPARATOR,
+            timeseriesSchemaList,
+            hasLimit);
       }
     }
   }
@@ -566,8 +599,8 @@ public class MTree implements Serializable {
   /**
    * Get child node path in the next level of the given path.
    *
-   * e.g., MTree has [root.sg1.d1.s1, root.sg1.d1.s2, root.sg1.d2.s1] given path = root.sg1, return
-   * [root.sg1.d1, root.sg1.d2]
+   * <p>e.g., MTree has [root.sg1.d1.s1, root.sg1.d1.s2, root.sg1.d2.s1] given path = root.sg1,
+   * return [root.sg1.d1, root.sg1.d2]
    *
    * @return All child nodes' seriesPath(s) of given seriesPath.
    */
@@ -591,15 +624,20 @@ public class MTree implements Serializable {
    * @param res store all matched device names
    * @param length expected length of path
    */
-  private void findChildNodePathInNextLevel(MNode node, String[] nodes, int idx, String parent,
-      Set<String> res, int length) {
+  private void findChildNodePathInNextLevel(
+      MNode node, String[] nodes, int idx, String parent, Set<String> res, int length) {
     String nodeReg = MetaUtils.getNodeRegByIdx(idx, nodes);
     if (!nodeReg.contains(PATH_WILDCARD)) {
       if (idx == length) {
         res.add(parent + node.getName());
       } else {
-        findChildNodePathInNextLevel(node.getChild(nodeReg), nodes, idx + 1,
-            parent + node.getName() + PATH_SEPARATOR, res, length);
+        findChildNodePathInNextLevel(
+            node.getChild(nodeReg),
+            nodes,
+            idx + 1,
+            parent + node.getName() + PATH_SEPARATOR,
+            res,
+            length);
       }
     } else {
       if (node instanceof InternalMNode && node.getChildren().size() > 0) {
@@ -610,8 +648,8 @@ public class MTree implements Serializable {
           if (idx == length) {
             res.add(parent + node.getName());
           } else {
-            findChildNodePathInNextLevel(child, nodes, idx + 1,
-                parent + node.getName() + PATH_SEPARATOR, res, length);
+            findChildNodePathInNextLevel(
+                child, nodes, idx + 1, parent + node.getName() + PATH_SEPARATOR, res, length);
           }
         }
       } else if (idx == length) {
@@ -657,8 +695,12 @@ public class MTree implements Serializable {
         if (node.getChild(nodeReg) instanceof LeafMNode) {
           res.add(parent + node.getName());
         } else {
-          findDevices(node.getChild(nodeReg), nodes, idx + 1,
-              parent + node.getName() + PATH_SEPARATOR, res);
+          findDevices(
+              node.getChild(nodeReg),
+              nodes,
+              idx + 1,
+              parent + node.getName() + PATH_SEPARATOR,
+              res);
         }
       }
     } else {
@@ -674,9 +716,7 @@ public class MTree implements Serializable {
     }
   }
 
-  /**
-   * Get all paths from root to the given level
-   */
+  /** Get all paths from root to the given level */
   List<String> getNodesList(String path, int nodeLevel) throws MetadataException {
     String[] nodes = MetaUtils.getNodeNames(path);
     if (!nodes[0].equals(root.getName())) {
@@ -741,9 +781,7 @@ public class MTree implements Serializable {
     return jsonObject;
   }
 
-  /**
-   * combine multiple metadata in string format
-   */
+  /** combine multiple metadata in string format */
   static String combineMetadataInStrings(String[] metadataStrs) {
     JSONObject[] jsonObjects = new JSONObject[metadataStrs.length];
     for (int i = 0; i < jsonObjects.length; i++) {
@@ -812,8 +850,13 @@ public class MTree implements Serializable {
    * group node, put a storageGroupName-fullPath pair into paths. Otherwise put the children that
    * match the path into the queue and discard other children.
    */
-  private void determineStorageGroup(int depth, String[] nodes, MNode mNode,
-      Map<String, String> paths, Deque<MNode> nodeStack, Deque<Integer> depthStack) {
+  private void determineStorageGroup(
+      int depth,
+      String[] nodes,
+      MNode mNode,
+      Map<String, String> paths,
+      Deque<MNode> nodeStack,
+      Deque<Integer> depthStack) {
     String currNode = depth >= nodes.length ? PATH_WILDCARD : nodes[depth];
     for (Entry<String, MNode> entry : mNode.getChildren().entrySet()) {
       if (!currNode.equals(PATH_WILDCARD) && !currNode.equals(entry.getKey())) {
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MetadataOperationType.java b/server/src/main/java/org/apache/iotdb/db/metadata/MetadataOperationType.java
index 72d34f4..0ffdcb7 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/MetadataOperationType.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/MetadataOperationType.java
@@ -29,4 +29,5 @@ public class MetadataOperationType {
   public static final String SET_STORAGE_GROUP = "2";
   public static final String SET_TTL = "10";
   public static final String DELETE_STORAGE_GROUP = "11";
+  public static final String CHANGE_OFFSET = "12";
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/TagLogFile.java b/server/src/main/java/org/apache/iotdb/db/metadata/TagLogFile.java
index aefb48d..0405459 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/TagLogFile.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/TagLogFile.java
@@ -31,6 +31,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.file.StandardOpenOption;
+import java.util.Collections;
 import java.util.Map;
 
 public class TagLogFile implements AutoCloseable {
@@ -66,6 +67,9 @@ public class TagLogFile implements AutoCloseable {
    * @return tags map, attributes map
    */
   public Pair<Map<String, String>, Map<String, String>> read(int size, long position) throws IOException {
+    if (position < 0) {
+      return new Pair<>(Collections.emptyMap(), Collections.emptyMap());
+    }
     ByteBuffer byteBuffer = ByteBuffer.allocate(size);
     fileChannel.read(byteBuffer, position);
     byteBuffer.flip();
@@ -81,20 +85,32 @@ public class TagLogFile implements AutoCloseable {
 
   public long write(Map<String, String> tagMap, Map<String, String> attributeMap) throws IOException, MetadataException {
     long offset = fileChannel.position();
+    ByteBuffer byteBuffer = convertMapToByteBuffer(tagMap, attributeMap);
+    fileChannel.write(byteBuffer);
+    return offset;
+  }
+
+  /**
+   * This method does not modify this file's current position.
+   */
+  public void write(Map<String, String> tagMap, Map<String, String> attributeMap, long position) throws IOException, MetadataException {
+    ByteBuffer byteBuffer = convertMapToByteBuffer(tagMap, attributeMap);
+    fileChannel.write(byteBuffer, position);
+  }
+
+  private ByteBuffer convertMapToByteBuffer(Map<String, String> tagMap, Map<String, String> attributeMap) throws MetadataException {
     ByteBuffer byteBuffer = ByteBuffer.allocate(MAX_LENGTH);
     int length = serializeMap(tagMap, byteBuffer, 0);
     length = serializeMap(attributeMap, byteBuffer, length);
 
     // fill the remaining space
-    for (int i = length+1; i <= MAX_LENGTH; i++) {
+    for (int i = length + 1; i <= MAX_LENGTH; i++) {
       byteBuffer.put(FILL_BYTE);
     }
 
     // persist to the disk
     byteBuffer.flip();
-    fileChannel.write(byteBuffer);
-
-    return offset;
+    return byteBuffer;
   }
 
   private int serializeMap(Map<String, String> map, ByteBuffer byteBuffer, int length) throws MetadataException {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/Planner.java b/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
index 8379de1..15a3a14 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/Planner.java
@@ -18,8 +18,6 @@
  */
 package org.apache.iotdb.db.qp;
 
-import java.time.ZoneId;
-import java.util.Set;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.query.LogicalOperatorException;
@@ -38,6 +36,9 @@ import org.apache.iotdb.db.qp.strategy.optimizer.RemoveNotOptimizer;
 import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.tsfile.read.common.Path;
 
+import java.time.ZoneId;
+import java.util.Set;
+
 /**
  * provide a integration method for other user.
  */
@@ -81,6 +82,7 @@ public class Planner {
       case DELETE_STORAGE_GROUP:
       case CREATE_TIMESERIES:
       case DELETE_TIMESERIES:
+      case ALTER_TIMESERIES:
       case LOADDATA:
       case INSERT:
       case INDEX:
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
index 4dba13c..a3771a8 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
@@ -18,11 +18,12 @@
  */
 package org.apache.iotdb.db.qp.constant;
 
-import java.util.HashMap;
-import java.util.Map;
 import org.apache.iotdb.db.qp.strategy.SqlBaseLexer;
 import org.apache.iotdb.tsfile.read.common.Path;
 
+import java.util.HashMap;
+import java.util.Map;
+
 /**
  * this class contains several constants used in SQL.
  */
@@ -138,6 +139,8 @@ public class SQLConstant {
   public static final int TOK_COUNT_NODE_TIMESERIES = 78;
   public static final int TOK_COUNT_NODES = 79;
 
+  public static final int TOK_METADATA_ALTER = 80;
+
   public static final Map<Integer, String> tokenSymbol = new HashMap<>();
   public static final Map<Integer, String> tokenNames = new HashMap<>();
   public static final Map<Integer, Integer> reverseWords = new HashMap<>();
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
index 57d1950..5615f38 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
@@ -111,7 +111,7 @@ public class PlanExecutor implements IPlanExecutor {
   @Override
   public QueryDataSet processQuery(PhysicalPlan queryPlan, QueryContext context)
       throws IOException, StorageEngineException, QueryFilterOptimizationException,
-      QueryProcessException, MetadataException {
+          QueryProcessException, MetadataException {
     if (queryPlan instanceof QueryPlan) {
       return processDataQuery((QueryPlan) queryPlan, context);
     } else if (queryPlan instanceof AuthorPlan) {
@@ -159,6 +159,8 @@ public class PlanExecutor implements IPlanExecutor {
         return deleteTimeSeries((DeleteTimeSeriesPlan) plan);
       case CREATE_TIMESERIES:
         return createTimeSeries((CreateTimeSeriesPlan) plan);
+      case ALTER_TIMESERIES:
+        return alterTimeSeries((AlterTimeSeriesPlan) plan);
       case SET_STORAGE_GROUP:
         return setStorageGroup((SetStorageGroupPlan) plan);
       case DELETE_STORAGE_GROUP:
@@ -186,7 +188,7 @@ public class PlanExecutor implements IPlanExecutor {
 
   protected QueryDataSet processDataQuery(QueryPlan queryPlan, QueryContext context)
       throws StorageEngineException, QueryFilterOptimizationException, QueryProcessException,
-      IOException {
+          IOException {
     QueryDataSet queryDataSet;
     if (queryPlan instanceof AlignByDevicePlan) {
       queryDataSet = new AlignByDeviceDataSet((AlignByDevicePlan) queryPlan, context, queryRouter);
@@ -256,9 +258,10 @@ public class PlanExecutor implements IPlanExecutor {
   private QueryDataSet processCountNodes(CountPlan countPlan) throws MetadataException {
     List<String> nodes = getNodesList(countPlan.getPath().toString(), countPlan.getLevel());
     int num = nodes.size();
-    SingleDataSet singleDataSet = new SingleDataSet(
-        Collections.singletonList(new Path(COLUMN_COUNT)),
-        Collections.singletonList(TSDataType.INT32));
+    SingleDataSet singleDataSet =
+        new SingleDataSet(
+            Collections.singletonList(new Path(COLUMN_COUNT)),
+            Collections.singletonList(TSDataType.INT32));
     Field field = new Field(TSDataType.INT32);
     field.setIntV(num);
     RowRecord record = new RowRecord(0);
@@ -269,9 +272,10 @@ public class PlanExecutor implements IPlanExecutor {
 
   private QueryDataSet processCountNodeTimeSeries(CountPlan countPlan) throws MetadataException {
     List<String> nodes = getNodesList(countPlan.getPath().toString(), countPlan.getLevel());
-    ListDataSet listDataSet = new ListDataSet(
-        Arrays.asList(new Path(COLUMN_COLUMN), new Path(COLUMN_COUNT)),
-        Arrays.asList(TSDataType.TEXT, TSDataType.TEXT));
+    ListDataSet listDataSet =
+        new ListDataSet(
+            Arrays.asList(new Path(COLUMN_COLUMN), new Path(COLUMN_COUNT)),
+            Arrays.asList(TSDataType.TEXT, TSDataType.TEXT));
     for (String columnPath : nodes) {
       RowRecord record = new RowRecord(0);
       Field field = new Field(TSDataType.TEXT);
@@ -295,9 +299,10 @@ public class PlanExecutor implements IPlanExecutor {
 
   private QueryDataSet processCountTimeSeries(CountPlan countPlan) throws MetadataException {
     int num = getPaths(countPlan.getPath().toString()).size();
-    SingleDataSet singleDataSet = new SingleDataSet(
-        Collections.singletonList(new Path(COLUMN_CHILD_PATHS)),
-        Collections.singletonList(TSDataType.INT32));
+    SingleDataSet singleDataSet =
+        new SingleDataSet(
+            Collections.singletonList(new Path(COLUMN_CHILD_PATHS)),
+            Collections.singletonList(TSDataType.INT32));
     Field field = new Field(TSDataType.INT32);
     field.setIntV(num);
     RowRecord record = new RowRecord(0);
@@ -308,8 +313,10 @@ public class PlanExecutor implements IPlanExecutor {
 
   private QueryDataSet processShowDevices(ShowDevicesPlan showDevicesPlan)
       throws MetadataException {
-    ListDataSet listDataSet = new ListDataSet(Collections.singletonList(new Path(COLUMN_DEVICES)),
-        Collections.singletonList(TSDataType.TEXT));
+    ListDataSet listDataSet =
+        new ListDataSet(
+            Collections.singletonList(new Path(COLUMN_DEVICES)),
+            Collections.singletonList(TSDataType.TEXT));
     Set<String> devices = getDevices(showDevicesPlan.getPath().toString());
     for (String s : devices) {
       RowRecord record = new RowRecord(0);
@@ -328,9 +335,10 @@ public class PlanExecutor implements IPlanExecutor {
   private QueryDataSet processShowChildPaths(ShowChildPathsPlan showChildPathsPlan)
       throws MetadataException {
     Set<String> childPathsList = getPathNextChildren(showChildPathsPlan.getPath().toString());
-    ListDataSet listDataSet = new ListDataSet(
-        Collections.singletonList(new Path(COLUMN_CHILD_PATHS)),
-        Collections.singletonList(TSDataType.TEXT));
+    ListDataSet listDataSet =
+        new ListDataSet(
+            Collections.singletonList(new Path(COLUMN_CHILD_PATHS)),
+            Collections.singletonList(TSDataType.TEXT));
     for (String s : childPathsList) {
       RowRecord record = new RowRecord(0);
       Field field = new Field(TSDataType.TEXT);
@@ -342,13 +350,14 @@ public class PlanExecutor implements IPlanExecutor {
   }
 
   protected Set<String> getPathNextChildren(String path) throws MetadataException {
-     return MManager.getInstance().getChildNodePathInNextLevel(path);
+    return MManager.getInstance().getChildNodePathInNextLevel(path);
   }
 
   private QueryDataSet processShowStorageGroup() {
-    ListDataSet listDataSet = new ListDataSet(
-        Collections.singletonList(new Path(COLUMN_STORAGE_GROUP)),
-        Collections.singletonList(TSDataType.TEXT));
+    ListDataSet listDataSet =
+        new ListDataSet(
+            Collections.singletonList(new Path(COLUMN_STORAGE_GROUP)),
+            Collections.singletonList(TSDataType.TEXT));
     List<String> storageGroupList = MManager.getInstance().getAllStorageGroupNames();
     for (String s : storageGroupList) {
       RowRecord record = new RowRecord(0);
@@ -362,15 +371,13 @@ public class PlanExecutor implements IPlanExecutor {
 
   private QueryDataSet processShowTimeseriesWithIndex(ShowTimeSeriesPlan showTimeSeriesPlan)
       throws MetadataException {
-    List<ShowTimeSeriesResult> timeseriesList = MManager.getInstance()
-        .getAllTimeseriesSchema(showTimeSeriesPlan);
+    List<ShowTimeSeriesResult> timeseriesList = mManager.getAllTimeseriesSchema(showTimeSeriesPlan);
     return getQueryDataSet(timeseriesList);
   }
 
   private QueryDataSet processShowTimeseries(ShowTimeSeriesPlan showTimeSeriesPlan)
       throws MetadataException {
-    List<ShowTimeSeriesResult> timeseriesList = MManager.getInstance()
-        .showTimeseries(showTimeSeriesPlan);
+    List<ShowTimeSeriesResult> timeseriesList = mManager.showTimeseries(showTimeSeriesPlan);
     return getQueryDataSet(timeseriesList);
   }
 
@@ -414,7 +421,8 @@ public class PlanExecutor implements IPlanExecutor {
     return listDataSet;
   }
 
-  private void updateRecord(RowRecord record, Map<String, String> tagAndAttribute, List<Path> paths) {
+  private void updateRecord(
+      RowRecord record, Map<String, String> tagAndAttribute, List<Path> paths) {
     for (int i = 6; i < paths.size(); i++) {
       updateRecord(record, tagAndAttribute.get(paths.get(i).getFullPath()));
     }
@@ -431,9 +439,10 @@ public class PlanExecutor implements IPlanExecutor {
   }
 
   private QueryDataSet processShowTTLQuery(ShowTTLPlan showTTLPlan) {
-    ListDataSet listDataSet = new ListDataSet(
-        Arrays.asList(new Path(COLUMN_STORAGE_GROUP), new Path(COLUMN_TTL))
-        , Arrays.asList(TSDataType.TEXT, TSDataType.INT64));
+    ListDataSet listDataSet =
+        new ListDataSet(
+            Arrays.asList(new Path(COLUMN_STORAGE_GROUP), new Path(COLUMN_TTL)),
+            Arrays.asList(TSDataType.TEXT, TSDataType.INT64));
     List<String> selectedSgs = showTTLPlan.getStorageGroups();
 
     List<StorageGroupMNode> storageGroups = MManager.getInstance().getAllStorageGroupNodes();
@@ -462,9 +471,10 @@ public class PlanExecutor implements IPlanExecutor {
   }
 
   private QueryDataSet processShowVersion() {
-    SingleDataSet singleDataSet = new SingleDataSet(
-        Collections.singletonList(new Path(IoTDBConstant.COLUMN_VERSION)),
-        Collections.singletonList(TSDataType.TEXT));
+    SingleDataSet singleDataSet =
+        new SingleDataSet(
+            Collections.singletonList(new Path(IoTDBConstant.COLUMN_VERSION)),
+            Collections.singletonList(TSDataType.TEXT));
     Field field = new Field(TSDataType.TEXT);
     field.setBinaryV(new Binary(IoTDBConstant.VERSION));
     RowRecord rowRecord = new RowRecord(0);
@@ -474,46 +484,77 @@ public class PlanExecutor implements IPlanExecutor {
   }
 
   private QueryDataSet processShowDynamicParameterQuery() {
-    ListDataSet listDataSet = new ListDataSet(
-        Arrays.asList(new Path(COLUMN_PARAMETER), new Path(COLUMN_VALUE)),
-        Arrays.asList(TSDataType.TEXT, TSDataType.TEXT));
+    ListDataSet listDataSet =
+        new ListDataSet(
+            Arrays.asList(new Path(COLUMN_PARAMETER), new Path(COLUMN_VALUE)),
+            Arrays.asList(TSDataType.TEXT, TSDataType.TEXT));
 
     int timestamp = 0;
-    addRowRecordForShowQuery(listDataSet, timestamp++, "memtable size threshold",
+    addRowRecordForShowQuery(
+        listDataSet,
+        timestamp++,
+        "memtable size threshold",
         IoTDBDescriptor.getInstance().getConfig().getMemtableSizeThreshold() + "B");
-    addRowRecordForShowQuery(listDataSet, timestamp++, "memtable number",
+    addRowRecordForShowQuery(
+        listDataSet,
+        timestamp++,
+        "memtable number",
         IoTDBDescriptor.getInstance().getConfig().getMaxMemtableNumber() + "B");
-    addRowRecordForShowQuery(listDataSet, timestamp++, "tsfile size threshold",
+    addRowRecordForShowQuery(
+        listDataSet,
+        timestamp++,
+        "tsfile size threshold",
         IoTDBDescriptor.getInstance().getConfig().getTsFileSizeThreshold() + "B");
-    addRowRecordForShowQuery(listDataSet, timestamp++, "compression ratio",
+    addRowRecordForShowQuery(
+        listDataSet,
+        timestamp++,
+        "compression ratio",
         Double.toString(CompressionRatio.getInstance().getRatio()));
-    addRowRecordForShowQuery(listDataSet, timestamp++, "storage group number",
+    addRowRecordForShowQuery(
+        listDataSet,
+        timestamp++,
+        "storage group number",
         Integer.toString(MManager.getInstance().getAllStorageGroupNames().size()));
-    addRowRecordForShowQuery(listDataSet, timestamp++, "timeseries number",
+    addRowRecordForShowQuery(
+        listDataSet,
+        timestamp++,
+        "timeseries number",
         Integer.toString(IoTDBConfigDynamicAdapter.getInstance().getTotalTimeseries()));
-    addRowRecordForShowQuery(listDataSet, timestamp,
+    addRowRecordForShowQuery(
+        listDataSet,
+        timestamp,
         "maximal timeseries number among storage groups",
         Long.toString(MManager.getInstance().getMaximalSeriesNumberAmongStorageGroups()));
     return listDataSet;
   }
 
   private QueryDataSet processShowFlushTaskInfo() {
-    ListDataSet listDataSet = new ListDataSet(
-        Arrays.asList(new Path(COLUMN_ITEM), new Path(COLUMN_VALUE)),
-        Arrays.asList(TSDataType.TEXT, TSDataType.TEXT));
+    ListDataSet listDataSet =
+        new ListDataSet(
+            Arrays.asList(new Path(COLUMN_ITEM), new Path(COLUMN_VALUE)),
+            Arrays.asList(TSDataType.TEXT, TSDataType.TEXT));
 
     int timestamp = 0;
-    addRowRecordForShowQuery(listDataSet, timestamp++, "total number of flush tasks",
+    addRowRecordForShowQuery(
+        listDataSet,
+        timestamp++,
+        "total number of flush tasks",
         Integer.toString(FlushTaskPoolManager.getInstance().getTotalTasks()));
-    addRowRecordForShowQuery(listDataSet, timestamp++, "number of working flush tasks",
+    addRowRecordForShowQuery(
+        listDataSet,
+        timestamp++,
+        "number of working flush tasks",
         Integer.toString(FlushTaskPoolManager.getInstance().getWorkingTasksNumber()));
-    addRowRecordForShowQuery(listDataSet, timestamp, "number of waiting flush tasks",
+    addRowRecordForShowQuery(
+        listDataSet,
+        timestamp,
+        "number of waiting flush tasks",
         Integer.toString(FlushTaskPoolManager.getInstance().getWaitingTasksNumber()));
     return listDataSet;
   }
 
-  private void addRowRecordForShowQuery(ListDataSet listDataSet, int timestamp, String item,
-      String value) {
+  private void addRowRecordForShowQuery(
+      ListDataSet listDataSet, int timestamp, String item, String value) {
     RowRecord rowRecord = new RowRecord(timestamp);
     Field itemField = new Field(TSDataType.TEXT);
     itemField.setBinaryV(new Binary(item));
@@ -532,8 +573,7 @@ public class PlanExecutor implements IPlanExecutor {
         existingPaths.addAll(getPaths(p.getFullPath()));
       }
       if (existingPaths.isEmpty()) {
-        throw new QueryProcessException(
-            "TimeSeries does not exist and its data cannot be deleted");
+        throw new QueryProcessException("TimeSeries does not exist and its data cannot be deleted");
       }
       for (String path : existingPaths) {
         delete(new Path(path), deletePlan.getDeleteTime());
@@ -579,8 +619,8 @@ public class PlanExecutor implements IPlanExecutor {
       if (restorableTsFileIOWriter.hasCrashed()) {
         restorableTsFileIOWriter.close();
         throw new QueryProcessException(
-            String.format("Cannot load file %s because the file has crashed.",
-                file.getAbsolutePath()));
+            String.format(
+                "Cannot load file %s because the file has crashed.", file.getAbsolutePath()));
       }
       Map<Path, MeasurementSchema> schemaMap = new HashMap<>();
 
@@ -599,7 +639,7 @@ public class PlanExecutor implements IPlanExecutor {
                 file.getAbsolutePath()));
       }
 
-      //create schemas if they doesn't exist
+      // create schemas if they doesn't exist
       if (plan.isAutoCreateSchema()) {
         createSchemaAutomatically(chunkGroupMetadataList, schemaMap, plan.getSgLevel());
       }
@@ -613,7 +653,8 @@ public class PlanExecutor implements IPlanExecutor {
 
   private void createSchemaAutomatically(
       List<ChunkGroupMetadata> chunkGroupMetadataList,
-      Map<Path, MeasurementSchema> knownSchemas, int sgLevel)
+      Map<Path, MeasurementSchema> knownSchemas,
+      int sgLevel)
       throws QueryProcessException, MetadataException {
     if (chunkGroupMetadataList.isEmpty()) {
       return;
@@ -675,7 +716,8 @@ public class PlanExecutor implements IPlanExecutor {
       }
     } catch (StorageEngineException | IOException e) {
       throw new QueryProcessException(
-          String.format("Cannot move file %s to target directory %s because %s",
+          String.format(
+              "Cannot move file %s to target directory %s because %s",
               plan.getFile().getPath(), plan.getTargetDir().getPath(), e.getMessage()));
     }
   }
@@ -712,7 +754,6 @@ public class PlanExecutor implements IPlanExecutor {
     }
   }
 
-
   @Override
   public void insert(InsertPlan insertPlan) throws QueryProcessException {
     MNode node = null;
@@ -748,25 +789,26 @@ public class PlanExecutor implements IPlanExecutor {
     }
   }
 
-  /**
-   * create timeseries with ignore PathAlreadyExistException
-   */
+  /** create timeseries with ignore PathAlreadyExistException */
   private void internalCreateTimeseries(String path, TSDataType dataType) throws MetadataException {
     try {
-      mManager.createTimeseries(path, dataType, getDefaultEncoding(dataType),
+      mManager.createTimeseries(
+          path,
+          dataType,
+          getDefaultEncoding(dataType),
           TSFileDescriptor.getInstance().getConfig().getCompressor(),
           Collections.emptyMap());
     } catch (PathAlreadyExistException e) {
       if (logger.isDebugEnabled()) {
-        logger.debug("Ignore PathAlreadyExistException when Concurrent inserting"
-            + " a non-exist time series {}", path);
+        logger.debug(
+            "Ignore PathAlreadyExistException when Concurrent inserting"
+                + " a non-exist time series {}",
+            path);
       }
     }
   }
 
-  /**
-   * Get default encoding by dataType
-   */
+  /** Get default encoding by dataType */
   private TSEncoding getDefaultEncoding(TSDataType dataType) {
     IoTDBConfig conf = IoTDBDescriptor.getInstance().getConfig();
     switch (dataType) {
@@ -804,7 +846,8 @@ public class PlanExecutor implements IPlanExecutor {
         if (!node.hasChild(measurementList[i])) {
           if (!conf.isAutoCreateSchemaEnabled()) {
             throw new QueryProcessException(
-                String.format("Current deviceId[%s] does not contain measurement:%s",
+                String.format(
+                    "Current deviceId[%s] does not contain measurement:%s",
                     deviceId, measurementList[i]));
           }
           Path path = new Path(deviceId, measurementList[i]);
@@ -815,9 +858,11 @@ public class PlanExecutor implements IPlanExecutor {
 
         // check data type
         if (measurementNode.getSchema().getType() != insertTabletPlan.getDataTypes()[i]) {
-          throw new QueryProcessException(String
-              .format("Datatype mismatch, Insert measurement %s type %s, metadata tree type %s",
-                  measurementList[i], insertTabletPlan.getDataTypes()[i],
+          throw new QueryProcessException(
+              String.format(
+                  "Datatype mismatch, Insert measurement %s type %s, metadata tree type %s",
+                  measurementList[i],
+                  insertTabletPlan.getDataTypes()[i],
                   measurementNode.getSchema().getType()));
         }
         schemas[i] = measurementNode.getSchema();
@@ -941,6 +986,41 @@ public class PlanExecutor implements IPlanExecutor {
     return true;
   }
 
+  private boolean alterTimeSeries(AlterTimeSeriesPlan alterTimeSeriesPlan)
+      throws QueryProcessException {
+    Path path = alterTimeSeriesPlan.getPath();
+    Map<String, String> alterMap = alterTimeSeriesPlan.getAlterMap();
+    try {
+      switch (alterTimeSeriesPlan.getAlterType()) {
+        case RENAME:
+          String beforeName = alterMap.keySet().iterator().next();
+          String currentName = alterMap.get(beforeName);
+          mManager.renameTagOrAttributeKey(beforeName, currentName, path.getFullPath());
+          break;
+        case SET:
+          mManager.setTagsOrAttributesValue(alterMap, path.getFullPath());
+          break;
+        case DROP:
+          mManager.dropTagsOrAttributes(alterMap.keySet(), path.getFullPath());
+          break;
+        case ADD_TAGS:
+          mManager.addTags(alterMap, path.getFullPath());
+          break;
+        case ADD_ATTRIBUTES:
+          mManager.addAttributes(alterMap, path.getFullPath());
+          break;
+      }
+    } catch (MetadataException e) {
+      throw new QueryProcessException(e);
+    } catch (IOException e) {
+      throw new QueryProcessException(
+          String.format(
+              "Something went wrong while read/write the [%s]'s tag/attribute info.",
+              path.getFullPath()));
+    }
+    return true;
+  }
+
   public boolean setStorageGroup(SetStorageGroupPlan setStorageGroupPlan)
       throws QueryProcessException {
     Path path = setStorageGroupPlan.getPath();
@@ -981,8 +1061,7 @@ public class PlanExecutor implements IPlanExecutor {
     }
   }
 
-  private QueryDataSet processAuthorQuery(AuthorPlan plan)
-      throws QueryProcessException {
+  private QueryDataSet processAuthorQuery(AuthorPlan plan) throws QueryProcessException {
     AuthorType authorType = plan.getAuthorType();
     String userName = plan.getUserName();
     String roleName = plan.getRoleName();
@@ -1055,8 +1134,7 @@ public class PlanExecutor implements IPlanExecutor {
     return dataSet;
   }
 
-  private ListDataSet executeListRoleUsers(String roleName)
-      throws AuthException {
+  private ListDataSet executeListRoleUsers(String roleName) throws AuthException {
     Role role = authorizer.getRole(roleName);
     if (role == null) {
       throw new AuthException("No such role : " + roleName);
@@ -1081,8 +1159,7 @@ public class PlanExecutor implements IPlanExecutor {
     return dataSet;
   }
 
-  private ListDataSet executeListUserRoles(String userName)
-      throws AuthException {
+  private ListDataSet executeListUserRoles(String userName) throws AuthException {
     User user = authorizer.getUser(userName);
     if (user != null) {
       List<Path> headerList = new ArrayList<>();
@@ -1104,8 +1181,7 @@ public class PlanExecutor implements IPlanExecutor {
     }
   }
 
-  private ListDataSet executeListRolePrivileges(String roleName, Path path)
-      throws AuthException {
+  private ListDataSet executeListRolePrivileges(String roleName, Path path) throws AuthException {
     Role role = authorizer.getRole(roleName);
     if (role != null) {
       List<Path> headerList = new ArrayList<>();
@@ -1115,8 +1191,7 @@ public class PlanExecutor implements IPlanExecutor {
       ListDataSet dataSet = new ListDataSet(headerList, typeList);
       int index = 0;
       for (PathPrivilege pathPrivilege : role.getPrivilegeList()) {
-        if (path == null || AuthUtils
-            .pathBelongsTo(path.getFullPath(), pathPrivilege.getPath())) {
+        if (path == null || AuthUtils.pathBelongsTo(path.getFullPath(), pathPrivilege.getPath())) {
           RowRecord record = new RowRecord(index++);
           Field field = new Field(TSDataType.TEXT);
           field.setBinaryV(new Binary(pathPrivilege.toString()));
@@ -1130,8 +1205,7 @@ public class PlanExecutor implements IPlanExecutor {
     }
   }
 
-  private ListDataSet executeListUserPrivileges(String userName, Path path)
-      throws AuthException {
+  private ListDataSet executeListUserPrivileges(String userName, Path path) throws AuthException {
     User user = authorizer.getUser(userName);
     if (user == null) {
       throw new AuthException("No such user : " + userName);
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
index 47af477..37c1d10 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
@@ -74,6 +74,7 @@ public abstract class Operator {
     DELETE_ROLE, GRANT_ROLE_PRIVILEGE, REVOKE_ROLE_PRIVILEGE, LIST_USER, LIST_ROLE,
     LIST_USER_PRIVILEGE, LIST_ROLE_PRIVILEGE, LIST_USER_ROLES, LIST_ROLE_USERS,
     GRANT_WATERMARK_EMBEDDING, REVOKE_WATERMARK_EMBEDDING,
-    TTL, DELETE_STORAGE_GROUP, LOAD_CONFIGURATION, SHOW, LOAD_FILES, REMOVE_FILE, MOVE_FILE, LAST, GROUP_BY_FILL
+    TTL, DELETE_STORAGE_GROUP, LOAD_CONFIGURATION, SHOW, LOAD_FILES, REMOVE_FILE, MOVE_FILE, LAST, GROUP_BY_FILL,
+    ALTER_TIMESERIES
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/AlterTimeSeriesOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/AlterTimeSeriesOperator.java
new file mode 100644
index 0000000..41d15ac
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/AlterTimeSeriesOperator.java
@@ -0,0 +1,67 @@
+/*
+ * 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.iotdb.db.qp.logical.sys;
+
+import org.apache.iotdb.db.qp.logical.RootOperator;
+import org.apache.iotdb.tsfile.read.common.Path;
+
+import java.util.Map;
+
+public class AlterTimeSeriesOperator extends RootOperator {
+
+  private Path path;
+
+  private AlterType alterType;
+
+  private Map<String, String> alterMap;
+
+  public AlterTimeSeriesOperator(int tokenIntType) {
+    super(tokenIntType);
+    operatorType = OperatorType.ALTER_TIMESERIES;
+  }
+
+  public Path getPath() {
+    return path;
+  }
+
+  public void setPath(Path path) {
+    this.path = path;
+  }
+
+  public AlterType getAlterType() {
+    return alterType;
+  }
+
+  public void setAlterType(AlterType alterType) {
+    this.alterType = alterType;
+  }
+
+  public Map<String, String> getAlterMap() {
+    return alterMap;
+  }
+
+  public void setAlterMap(Map<String, String> alterMap) {
+    this.alterMap = alterMap;
+  }
+
+  public enum AlterType {
+    RENAME, SET, DROP, ADD_TAGS, ADD_ATTRIBUTES
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/AlterTimeSeriesPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/AlterTimeSeriesPlan.java
new file mode 100644
index 0000000..7967c96
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/AlterTimeSeriesPlan.java
@@ -0,0 +1,62 @@
+/*
+ * 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.iotdb.db.qp.physical.sys;
+
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.qp.logical.sys.AlterTimeSeriesOperator;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.tsfile.read.common.Path;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class AlterTimeSeriesPlan extends PhysicalPlan {
+
+  private Path path;
+
+  private AlterTimeSeriesOperator.AlterType alterType;
+
+  private Map<String, String> alterMap;
+
+  public AlterTimeSeriesPlan(Path path, AlterTimeSeriesOperator.AlterType alterType, Map<String, String> alterMap) {
+    super(false, Operator.OperatorType.ALTER_TIMESERIES);
+    this.path = path;
+    this.alterType = alterType;
+    this.alterMap = alterMap;
+  }
+
+  public Path getPath() {
+    return path;
+  }
+
+  public AlterTimeSeriesOperator.AlterType getAlterType() {
+    return alterType;
+  }
+
+  public Map<String, String> getAlterMap() {
+    return alterMap;
+  }
+
+  @Override
+  public List<Path> getPaths() {
+    return Collections.singletonList(path);
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
index 5df1f3a..a9b069b 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
@@ -52,6 +52,7 @@ public class LogicalGenerator extends SqlBaseBaseListener {
   private ZoneId zoneId;
   private int operatorType;
   private CreateTimeSeriesOperator createTimeSeriesOperator;
+  private AlterTimeSeriesOperator alterTimeSeriesOperator;
   private InsertOperator insertOp;
   private SelectOperator selectOp;
   private UpdateOperator updateOp;
@@ -194,6 +195,60 @@ public class LogicalGenerator extends SqlBaseBaseListener {
   }
 
   @Override
+  public void enterAlterTimeseries(SqlBaseParser.AlterTimeseriesContext ctx) {
+    super.enterAlterTimeseries(ctx);
+    alterTimeSeriesOperator = new AlterTimeSeriesOperator(SQLConstant.TOK_METADATA_ALTER);
+    operatorType = SQLConstant.TOK_METADATA_ALTER;
+    alterTimeSeriesOperator.setPath(parseFullPath(ctx.fullPath()));
+  }
+
+  @Override
+  public void enterAlterClause(SqlBaseParser.AlterClauseContext ctx) {
+    super.enterAlterClause(ctx);
+    Map<String, String> alterMap = new HashMap<>();
+    // rename
+    if (ctx.RENAME() != null) {
+      alterTimeSeriesOperator.setAlterType(AlterTimeSeriesOperator.AlterType.RENAME);
+      alterMap.put(ctx.beforeName.getText(), ctx.currentName.getText());
+    } else if (ctx.SET() != null) {
+      // set
+      alterTimeSeriesOperator.setAlterType(AlterTimeSeriesOperator.AlterType.SET);
+      setMap(ctx, alterMap);
+    } else if (ctx.DROP() != null) {
+      // drop
+      alterTimeSeriesOperator.setAlterType(AlterTimeSeriesOperator.AlterType.DROP);
+      for (TerminalNode dropId : ctx.ID()) {
+        alterMap.put(dropId.getText(), null);
+      }
+    } else if (ctx.TAGS() != null) {
+      // add tag
+      alterTimeSeriesOperator.setAlterType(AlterTimeSeriesOperator.AlterType.ADD_TAGS);
+      setMap(ctx, alterMap);
+    } else {
+      // add attribute
+      alterTimeSeriesOperator.setAlterType(AlterTimeSeriesOperator.AlterType.ADD_ATTRIBUTES);
+      setMap(ctx, alterMap);
+    }
+    alterTimeSeriesOperator.setAlterMap(alterMap);
+    initializedOperator = alterTimeSeriesOperator;
+  }
+
+  private void setMap(SqlBaseParser.AlterClauseContext ctx, Map<String, String> alterMap) {
+    List<PropertyContext> tagsList = ctx.property();
+    if (ctx.property(0) != null) {
+      for (PropertyContext property : tagsList) {
+        String value;
+        if(property.propertyValue().STRING_LITERAL() != null) {
+          value = removeStringQuote(property.propertyValue().getText());
+        } else {
+          value = property.propertyValue().getText();
+        }
+        alterMap.put(property.ID().getText(), value);
+      }
+    }
+  }
+
+  @Override
   public void enterAlias(AliasContext ctx) {
     super.enterAlias(ctx);
     createTimeSeriesOperator.setAlias(ctx.ID().getText());
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
index ff80539..8f27c54 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
@@ -42,20 +42,22 @@ import org.apache.iotdb.tsfile.utils.Pair;
 
 import java.util.*;
 
-/**
- * Used to convert logical operator to physical plan
- */
+/** Used to convert logical operator to physical plan */
 public class PhysicalGenerator {
 
-  public PhysicalPlan transformToPhysicalPlan(Operator operator)
-      throws QueryProcessException {
+  public PhysicalPlan transformToPhysicalPlan(Operator operator) throws QueryProcessException {
     List<Path> paths;
     switch (operator.getType()) {
       case AUTHOR:
         AuthorOperator author = (AuthorOperator) operator;
         try {
-          return new AuthorPlan(author.getAuthorType(), author.getUserName(), author.getRoleName(),
-              author.getPassWord(), author.getNewPassword(), author.getPrivilegeList(),
+          return new AuthorPlan(
+              author.getAuthorType(),
+              author.getUserName(),
+              author.getRoleName(),
+              author.getPassWord(),
+              author.getNewPassword(),
+              author.getPrivilegeList(),
               author.getNodeName());
         } catch (AuthException e) {
           throw new QueryProcessException(e.getMessage());
@@ -75,13 +77,37 @@ public class PhysicalGenerator {
         DeleteStorageGroupOperator deleteStorageGroup = (DeleteStorageGroupOperator) operator;
         return new DeleteStorageGroupPlan(deleteStorageGroup.getDeletePathList());
       case CREATE_TIMESERIES:
-        CreateTimeSeriesOperator addPath = (CreateTimeSeriesOperator) operator;
-        return new CreateTimeSeriesPlan(addPath.getPath(), addPath.getDataType(),
-            addPath.getEncoding(), addPath.getCompressor(), addPath.getProps(),
-            addPath.getTags(), addPath.getAttributes(), addPath.getAlias());
+        CreateTimeSeriesOperator createOperator = (CreateTimeSeriesOperator) operator;
+        if (createOperator.getTags() != null
+            && !createOperator.getTags().isEmpty()
+            && createOperator.getAttributes() != null
+            && !createOperator.getAttributes().isEmpty()) {
+          for (String tagKey : createOperator.getTags().keySet()) {
+            if (createOperator.getAttributes().containsKey(tagKey)) {
+              throw new QueryProcessException(
+                  String.format(
+                      "Tag and attribute shouldn't have the same property key [%s]", tagKey));
+            }
+          }
+        }
+        return new CreateTimeSeriesPlan(
+            createOperator.getPath(),
+            createOperator.getDataType(),
+            createOperator.getEncoding(),
+            createOperator.getCompressor(),
+            createOperator.getProps(),
+            createOperator.getTags(),
+            createOperator.getAttributes(),
+            createOperator.getAlias());
       case DELETE_TIMESERIES:
         DeleteTimeSeriesOperator deletePath = (DeleteTimeSeriesOperator) operator;
         return new DeleteTimeSeriesPlan(deletePath.getDeletePathList());
+      case ALTER_TIMESERIES:
+        AlterTimeSeriesOperator alterTimeSeriesOperator = (AlterTimeSeriesOperator) operator;
+        return new AlterTimeSeriesPlan(
+            alterTimeSeriesOperator.getPath(),
+            alterTimeSeriesOperator.getAlterType(),
+            alterTimeSeriesOperator.getAlterMap());
       case DELETE:
         DeleteDataOperator delete = (DeleteDataOperator) operator;
         paths = delete.getSelectedPaths();
@@ -93,7 +119,9 @@ public class PhysicalGenerator {
           throw new LogicalOperatorException(
               "For Insert command, cannot specified more than one seriesPath: " + paths);
         }
-        return new InsertPlan(paths.get(0).getFullPath(), insert.getTime(),
+        return new InsertPlan(
+            paths.get(0).getFullPath(),
+            insert.getTime(),
             insert.getMeasurementList(),
             insert.getValueList());
       case QUERY:
@@ -111,8 +139,9 @@ public class PhysicalGenerator {
             ShowTTLOperator showTTLOperator = (ShowTTLOperator) operator;
             return new ShowTTLPlan(showTTLOperator.getStorageGroups());
           default:
-            throw new LogicalOperatorException(String
-                .format("not supported operator type %s in ttl operation.", operator.getType()));
+            throw new LogicalOperatorException(
+                String.format(
+                    "not supported operator type %s in ttl operation.", operator.getType()));
         }
       case LOAD_CONFIGURATION:
         return new LoadConfigurationPlan();
@@ -133,44 +162,53 @@ public class PhysicalGenerator {
           case SQLConstant.TOK_STORAGE_GROUP:
             return new ShowPlan(ShowContentType.STORAGE_GROUP);
           case SQLConstant.TOK_DEVICES:
-            return new ShowDevicesPlan(ShowContentType.DEVICES,
-                ((ShowDevicesOperator) operator).getPath());
+            return new ShowDevicesPlan(
+                ShowContentType.DEVICES, ((ShowDevicesOperator) operator).getPath());
           case SQLConstant.TOK_COUNT_NODE_TIMESERIES:
-            return new CountPlan(ShowContentType.COUNT_NODE_TIMESERIES,
-                ((CountOperator) operator).getPath(), ((CountOperator) operator).getLevel());
+            return new CountPlan(
+                ShowContentType.COUNT_NODE_TIMESERIES,
+                ((CountOperator) operator).getPath(),
+                ((CountOperator) operator).getLevel());
           case SQLConstant.TOK_COUNT_NODES:
-            return new CountPlan(ShowContentType.COUNT_NODES,
-                ((CountOperator) operator).getPath(), ((CountOperator) operator).getLevel());
+            return new CountPlan(
+                ShowContentType.COUNT_NODES,
+                ((CountOperator) operator).getPath(),
+                ((CountOperator) operator).getLevel());
           case SQLConstant.TOK_COUNT_TIMESERIES:
-            return new CountPlan(ShowContentType.COUNT_TIMESERIES,
-                ((CountOperator) operator).getPath());
+            return new CountPlan(
+                ShowContentType.COUNT_TIMESERIES, ((CountOperator) operator).getPath());
           case SQLConstant.TOK_CHILD_PATHS:
-            return new ShowChildPathsPlan(ShowContentType.CHILD_PATH,
-                ((ShowChildPathsOperator) operator).getPath());
+            return new ShowChildPathsPlan(
+                ShowContentType.CHILD_PATH, ((ShowChildPathsOperator) operator).getPath());
           default:
-            throw new LogicalOperatorException(String
-                .format("not supported operator type %s in show operation.", operator.getType()));
+            throw new LogicalOperatorException(
+                String.format(
+                    "not supported operator type %s in show operation.", operator.getType()));
         }
       case LOAD_FILES:
         if (((LoadFilesOperator) operator).isInvalid()) {
           throw new LogicalOperatorException(((LoadFilesOperator) operator).getErrMsg());
         }
-        return new OperateFilePlan(((LoadFilesOperator) operator).getFile(),
-            OperatorType.LOAD_FILES, ((LoadFilesOperator) operator).isAutoCreateSchema(),
+        return new OperateFilePlan(
+            ((LoadFilesOperator) operator).getFile(),
+            OperatorType.LOAD_FILES,
+            ((LoadFilesOperator) operator).isAutoCreateSchema(),
             ((LoadFilesOperator) operator).getSgLevel());
       case REMOVE_FILE:
-        return new OperateFilePlan(((RemoveFileOperator) operator).getFile(),
-            OperatorType.REMOVE_FILE);
+        return new OperateFilePlan(
+            ((RemoveFileOperator) operator).getFile(), OperatorType.REMOVE_FILE);
       case MOVE_FILE:
-        return new OperateFilePlan(((MoveFileOperator) operator).getFile(),
-            ((MoveFileOperator) operator).getTargetDir(), OperatorType.MOVE_FILE);
+        return new OperateFilePlan(
+            ((MoveFileOperator) operator).getFile(),
+            ((MoveFileOperator) operator).getTargetDir(),
+            OperatorType.MOVE_FILE);
       default:
         throw new LogicalOperatorException(operator.getType().toString(), "");
     }
   }
 
-  protected List<TSDataType> getSeriesTypes(List<String> paths,
-      String aggregation) throws MetadataException {
+  protected List<TSDataType> getSeriesTypes(List<String> paths, String aggregation)
+      throws MetadataException {
     return SchemaUtils.getSeriesTypesByString(paths, aggregation);
   }
 
@@ -178,8 +216,7 @@ public class PhysicalGenerator {
     return SchemaUtils.getSeriesTypesByPath(paths);
   }
 
-  private PhysicalPlan transformQuery(QueryOperator queryOperator)
-      throws QueryProcessException {
+  private PhysicalPlan transformQuery(QueryOperator queryOperator) throws QueryProcessException {
     QueryPlan queryPlan;
 
     if (queryOperator.isGroupBy() && queryOperator.isFill()) {
@@ -188,14 +225,14 @@ public class PhysicalGenerator {
       ((GroupByFillPlan) queryPlan).setSlidingStep(queryOperator.getSlidingStep());
       ((GroupByFillPlan) queryPlan).setLeftCRightO(queryOperator.isLeftCRightO());
       if (!queryOperator.isLeftCRightO()) {
-        ((GroupByPlan) queryPlan).setStartTime(queryOperator.getStartTime()+1);
-        ((GroupByPlan) queryPlan).setEndTime(queryOperator.getEndTime()+1);
+        ((GroupByPlan) queryPlan).setStartTime(queryOperator.getStartTime() + 1);
+        ((GroupByPlan) queryPlan).setEndTime(queryOperator.getEndTime() + 1);
       } else {
         ((GroupByPlan) queryPlan).setStartTime(queryOperator.getStartTime());
         ((GroupByPlan) queryPlan).setEndTime(queryOperator.getEndTime());
       }
       ((GroupByFillPlan) queryPlan)
-              .setAggregations(queryOperator.getSelectOperator().getAggregations());
+          .setAggregations(queryOperator.getSelectOperator().getAggregations());
       for (String aggregation : queryPlan.getAggregations()) {
         if (!SQLConstant.LAST_VALUE.equals(aggregation)) {
           throw new QueryProcessException("Group By Fill only support last_value function");
@@ -208,8 +245,8 @@ public class PhysicalGenerator {
       ((GroupByPlan) queryPlan).setSlidingStep(queryOperator.getSlidingStep());
       ((GroupByPlan) queryPlan).setLeftCRightO(queryOperator.isLeftCRightO());
       if (!queryOperator.isLeftCRightO()) {
-        ((GroupByPlan) queryPlan).setStartTime(queryOperator.getStartTime()+1);
-        ((GroupByPlan) queryPlan).setEndTime(queryOperator.getEndTime()+1);
+        ((GroupByPlan) queryPlan).setStartTime(queryOperator.getStartTime() + 1);
+        ((GroupByPlan) queryPlan).setEndTime(queryOperator.getEndTime() + 1);
       } else {
         ((GroupByPlan) queryPlan).setStartTime(queryOperator.getStartTime());
         ((GroupByPlan) queryPlan).setEndTime(queryOperator.getEndTime());
@@ -292,14 +329,17 @@ public class PhysicalGenerator {
               }
             }
 
-            String aggregation = originAggregations != null && !originAggregations.isEmpty() ?
-                originAggregations.get(i) : null;
+            String aggregation =
+                originAggregations != null && !originAggregations.isEmpty()
+                    ? originAggregations.get(i)
+                    : null;
             List<TSDataType> dataTypes = getSeriesTypes(actualPaths, aggregation);
             for (int pathIdx = 0; pathIdx < actualPaths.size(); pathIdx++) {
               Path path = new Path(actualPaths.get(pathIdx));
 
               // check datatype consistency
-              // a example of inconsistency: select s0 from root.sg1.d1, root.sg2.d3 align by device,
+              // a example of inconsistency: select s0 from root.sg1.d1, root.sg2.d3 align by
+              // device,
               // while root.sg1.d1.s0 is INT32 and root.sg2.d3.s0 is FLOAT.
               String measurementChecked;
               if (originAggregations != null && !originAggregations.isEmpty()) {
@@ -330,8 +370,9 @@ public class PhysicalGenerator {
 
           } catch (MetadataException e) {
             throw new LogicalOptimizeException(
-                String.format("Error when getting all paths of a full path: %s",
-                    fullPath.getFullPath()) + e.getMessage());
+                String.format(
+                        "Error when getting all paths of a full path: %s", fullPath.getFullPath())
+                    + e.getMessage());
           }
         }
 
@@ -404,9 +445,8 @@ public class PhysicalGenerator {
   // e.g. translate "select * from root.ln.d1, root.ln.d2 where s1 < 20 AND s2 > 10" to
   // [root.ln.d1 -> root.ln.d1.s1 < 20 AND root.ln.d1.s2 > 10,
   //  root.ln.d2 -> root.ln.d2.s1 < 20 AND root.ln.d2.s2 > 10)]
-  private Map<String, IExpression> concatFilterByDevice(List<String> devices,
-      FilterOperator operator)
-      throws QueryProcessException {
+  private Map<String, IExpression> concatFilterByDevice(
+      List<String> devices, FilterOperator operator) throws QueryProcessException {
     Map<String, IExpression> deviceToFilterMap = new HashMap<>();
     Set<Path> filterPaths = new HashSet<>();
     for (String device : devices) {
@@ -446,8 +486,7 @@ public class PhysicalGenerator {
     return retDevices;
   }
 
-  private void concatFilterPath(String prefix, FilterOperator operator,
-      Set<Path> filterPaths) {
+  private void concatFilterPath(String prefix, FilterOperator operator, Set<Path> filterPaths) {
     if (!operator.isLeaf()) {
       for (FilterOperator child : operator.getChildren()) {
         concatFilterPath(prefix, child, filterPaths);
@@ -490,7 +529,8 @@ public class PhysicalGenerator {
       Set<String> columnSet = new HashSet<>();
       int index = 0;
       for (Pair<Path, Integer> indexedPath : indexedPaths) {
-        String column = aggregations.get(indexedPath.right) + "(" + indexedPath.left.toString() + ")";
+        String column =
+            aggregations.get(indexedPath.right) + "(" + indexedPath.left.toString() + ")";
         if (!columnSet.contains(column)) {
           aggregationPlan.addDeduplicatedPaths(indexedPath.left);
           TSDataType seriesType = dataTypes.get(indexedPath.right);
@@ -558,4 +598,3 @@ public class PhysicalGenerator {
     return MManager.getInstance().getDevices(path);
   }
 }
-
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/LocalGroupByExecutor.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/LocalGroupByExecutor.java
index a78250b..a6735b6 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/LocalGroupByExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/LocalGroupByExecutor.java
@@ -49,15 +49,21 @@ public class LocalGroupByExecutor implements GroupByExecutor {
   private List<AggregateResult> results = new ArrayList<>();
   private TimeRange timeRange;
 
-  public LocalGroupByExecutor(Path path, Set<String> allSensors, TSDataType dataType, QueryContext context, Filter timeFilter,
-                              TsFileFilter fileFilter)
+  public LocalGroupByExecutor(
+      Path path,
+      Set<String> allSensors,
+      TSDataType dataType,
+      QueryContext context,
+      Filter timeFilter,
+      TsFileFilter fileFilter)
       throws StorageEngineException, QueryProcessException {
-    QueryDataSource queryDataSource = QueryResourceManager.getInstance()
-        .getQueryDataSource(path, context, timeFilter);
+    QueryDataSource queryDataSource =
+        QueryResourceManager.getInstance().getQueryDataSource(path, context, timeFilter);
     // update filter by TTL
     timeFilter = queryDataSource.updateFilterUsingTTL(timeFilter);
-    this.reader = new SeriesAggregateReader(path, allSensors, dataType, context, queryDataSource, timeFilter,
-        null, fileFilter);
+    this.reader =
+        new SeriesAggregateReader(
+            path, allSensors, dataType, context, queryDataSource, timeFilter, null, fileFilter);
     this.preCachedData = null;
     timeRange = new TimeRange(Long.MIN_VALUE, Long.MAX_VALUE);
   }
@@ -79,11 +85,11 @@ public class LocalGroupByExecutor implements GroupByExecutor {
   private boolean calcFromCacheData(long curStartTime, long curEndTime) throws IOException {
     calcFromBatch(preCachedData, curStartTime, curEndTime);
     // The result is calculated from the cache
-    return (preCachedData != null && preCachedData.getMaxTimestamp() >= curEndTime)
-        || isEndCalc();
+    return (preCachedData != null && preCachedData.getMaxTimestamp() >= curEndTime) || isEndCalc();
   }
 
-  private void calcFromBatch(BatchData batchData, long curStartTime, long curEndTime) throws IOException {
+  private void calcFromBatch(BatchData batchData, long curStartTime, long curEndTime)
+      throws IOException {
     // is error data
     if (batchData == null
         || !batchData.hasCurrent()
@@ -93,13 +99,13 @@ public class LocalGroupByExecutor implements GroupByExecutor {
     }
 
     for (AggregateResult result : results) {
-      //current agg method has been calculated
+      // current agg method has been calculated
       if (result.isCalculatedAggregationResult()) {
         continue;
       }
-      //lazy reset batch data for calculation
+      // lazy reset batch data for calculation
       batchData.resetBatchData();
-      //skip points that cannot be calculated
+      // skip points that cannot be calculated
       while (batchData.currentTime() < curStartTime && batchData.hasCurrent()) {
         batchData.next();
       }
@@ -107,16 +113,15 @@ public class LocalGroupByExecutor implements GroupByExecutor {
         result.updateResultFromPageData(batchData, curEndTime);
       }
     }
-    //can calc for next interval
+    // can calc for next interval
     if (batchData.getMaxTimestamp() >= curEndTime) {
       preCachedData = batchData;
     }
   }
 
-  private void calcFromStatistics(Statistics pageStatistics)
-      throws QueryProcessException {
+  private void calcFromStatistics(Statistics pageStatistics) throws QueryProcessException {
     for (AggregateResult result : results) {
-      //cacl is compile
+      // cacl is compile
       if (result.isCalculatedAggregationResult()) {
         continue;
       }
@@ -138,11 +143,16 @@ public class LocalGroupByExecutor implements GroupByExecutor {
       return results;
     }
 
-    //read page data firstly
+    // read page data firstly
     if (readAndCalcFromPage(curStartTime, curEndTime)) {
       return results;
     }
 
+    // read chunk data secondly
+    if (readAndCalcFromChunk(curStartTime, curEndTime)) {
+      return results;
+    }
+
     // read from file first
     while (reader.hasNextFile()) {
       Statistics fileStatistics = reader.currentFileStatistics();
@@ -157,39 +167,47 @@ public class LocalGroupByExecutor implements GroupByExecutor {
         continue;
       }
 
-      //read chunk
-      while (reader.hasNextChunk()) {
-        Statistics chunkStatistics = reader.currentChunkStatistics();
-        if (chunkStatistics.getStartTime() >= curEndTime) {
-          return results;
-        }
-        //calc from chunkMetaData
-        if (reader.canUseCurrentChunkStatistics()
-                && timeRange.contains(chunkStatistics.getStartTime(), chunkStatistics.getEndTime())) {
-          calcFromStatistics(chunkStatistics);
-          reader.skipCurrentChunk();
-          continue;
-        }
-        if (readAndCalcFromPage(curStartTime, curEndTime)) {
-          return results;
-        }
+      // read chunk
+      if (readAndCalcFromChunk(curStartTime, curEndTime)) {
+        return results;
       }
     }
 
     return results;
   }
 
-  private boolean readAndCalcFromPage(long curStartTime, long curEndTime) throws IOException,
-      QueryProcessException {
+  private boolean readAndCalcFromChunk(long curStartTime, long curEndTime)
+      throws IOException, QueryProcessException {
+    while (reader.hasNextChunk()) {
+      Statistics chunkStatistics = reader.currentChunkStatistics();
+      if (chunkStatistics.getStartTime() >= curEndTime) {
+        return true;
+      }
+      // calc from chunkMetaData
+      if (reader.canUseCurrentChunkStatistics()
+          && timeRange.contains(chunkStatistics.getStartTime(), chunkStatistics.getEndTime())) {
+        calcFromStatistics(chunkStatistics);
+        reader.skipCurrentChunk();
+        continue;
+      }
+      if (readAndCalcFromPage(curStartTime, curEndTime)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private boolean readAndCalcFromPage(long curStartTime, long curEndTime)
+      throws IOException, QueryProcessException {
     while (reader.hasNextPage()) {
       Statistics pageStatistics = reader.currentPageStatistics();
-      //must be non overlapped page
+      // must be non overlapped page
       if (pageStatistics != null) {
-        //current page max than time range
+        // current page max than time range
         if (pageStatistics.getStartTime() >= curEndTime) {
           return true;
         }
-        //can use pageHeader
+        // can use pageHeader
         if (reader.canUseCurrentPageStatistics()
             && timeRange.contains(pageStatistics.getStartTime(), pageStatistics.getEndTime())) {
           calcFromStatistics(pageStatistics);
diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
index 0988cd9..0e04e23 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReader.java
@@ -66,14 +66,14 @@ class SeriesReader {
   private final List<TsFileResource> seqFileResource;
   private final List<TsFileResource> unseqFileResource;
 
-
   /*
    * TimeSeriesMetadata cache
    */
   private TimeseriesMetadata firstTimeSeriesMetadata;
   private final List<TimeseriesMetadata> seqTimeSeriesMetadata = new LinkedList<>();
   private final PriorityQueue<TimeseriesMetadata> unSeqTimeSeriesMetadata =
-          new PriorityQueue<>(Comparator.comparingLong(timeSeriesMetadata -> timeSeriesMetadata.getStatistics().getStartTime()));
+      new PriorityQueue<>(Comparator.comparingLong(
+              timeSeriesMetadata -> timeSeriesMetadata.getStatistics().getStartTime()));
 
   /*
    * chunk cache
@@ -100,16 +100,8 @@ class SeriesReader {
   private boolean hasCachedNextOverlappedPage;
   private BatchData cachedBatchData;
 
-
-  SeriesReader(
-      Path seriesPath,
-      Set<String> allSensors,
-      TSDataType dataType,
-      QueryContext context,
-      QueryDataSource dataSource,
-      Filter timeFilter,
-      Filter valueFilter,
-      TsFileFilter fileFilter) {
+  SeriesReader(Path seriesPath, Set<String> allSensors, TSDataType dataType, QueryContext context,
+      QueryDataSource dataSource, Filter timeFilter, Filter valueFilter, TsFileFilter fileFilter) {
     this.seriesPath = seriesPath;
     this.allSensors = allSensors;
     this.dataType = dataType;
@@ -122,15 +114,9 @@ class SeriesReader {
   }
 
   @TestOnly
-  SeriesReader(
-      Path seriesPath,
-      Set<String> allSensors,
-      TSDataType dataType,
-      QueryContext context,
-      List<TsFileResource> seqFileResource,
-      List<TsFileResource> unseqFileResource,
-      Filter timeFilter,
-      Filter valueFilter) {
+  SeriesReader(Path seriesPath, Set<String> allSensors, TSDataType dataType, QueryContext context,
+      List<TsFileResource> seqFileResource, List<TsFileResource> unseqFileResource,
+      Filter timeFilter, Filter valueFilter) {
     this.seriesPath = seriesPath;
     this.allSensors = allSensors;
     this.dataType = dataType;
@@ -144,17 +130,18 @@ class SeriesReader {
   boolean hasNextFile() throws IOException {
 
     if (!cachedPageReaders.isEmpty()
-            || firstPageReader != null
+        || firstPageReader != null
         || mergeReader.hasNextTimeValuePair()) {
       throw new IOException(
           "all cached pages should be consumed first cachedPageReaders.isEmpty() is "
-              + cachedPageReaders.isEmpty() + " firstPageReader != null is " + (firstPageReader
-              != null) + " mergeReader.hasNextTimeValuePair() = " + mergeReader
-              .hasNextTimeValuePair());
+              + cachedPageReaders.isEmpty()
+              + " firstPageReader != null is "
+              + (firstPageReader != null)
+              + " mergeReader.hasNextTimeValuePair() = "
+              + mergeReader.hasNextTimeValuePair());
     }
 
-    if (firstChunkMetadata != null
-            || !cachedChunkMetadata.isEmpty()) {
+    if (firstChunkMetadata != null || !cachedChunkMetadata.isEmpty()) {
       throw new IOException("all cached chunks should be consumed first");
     }
 
@@ -175,9 +162,11 @@ class SeriesReader {
 
     Statistics fileStatistics = firstTimeSeriesMetadata.getStatistics();
     return !seqTimeSeriesMetadata.isEmpty()
-            && fileStatistics.getEndTime() >= seqTimeSeriesMetadata.get(0).getStatistics().getStartTime()
-            || !unSeqTimeSeriesMetadata.isEmpty()
-            && fileStatistics.getEndTime() >= unSeqTimeSeriesMetadata.peek().getStatistics().getStartTime();
+            && fileStatistics.getEndTime()
+                >= seqTimeSeriesMetadata.get(0).getStatistics().getStartTime()
+        || !unSeqTimeSeriesMetadata.isEmpty()
+            && fileStatistics.getEndTime()
+                >= unSeqTimeSeriesMetadata.peek().getStatistics().getStartTime();
   }
 
   Statistics currentFileStatistics() {
@@ -205,9 +194,11 @@ class SeriesReader {
         || mergeReader.hasNextTimeValuePair()) {
       throw new IOException(
           "all cached pages should be consumed first cachedPageReaders.isEmpty() is "
-              + cachedPageReaders.isEmpty() + " firstPageReader != null is " + (firstPageReader
-              != null) + " mergeReader.hasNextTimeValuePair() = " + mergeReader
-              .hasNextTimeValuePair());
+              + cachedPageReaders.isEmpty()
+              + " firstPageReader != null is "
+              + (firstPageReader != null)
+              + " mergeReader.hasNextTimeValuePair() = "
+              + mergeReader.hasNextTimeValuePair());
     }
 
     if (firstChunkMetadata != null) {
@@ -221,8 +212,10 @@ class SeriesReader {
       /*
        * try to unpack all overlapped TimeSeriesMetadata to cachedChunkMetadata
        */
-      unpackAllOverlappedTsFilesToTimeSeriesMetadata(firstTimeSeriesMetadata.getStatistics().getEndTime());
-      unpackAllOverlappedTimeSeriesMetadataToCachedChunkMetadata(firstTimeSeriesMetadata.getStatistics().getEndTime(), true);
+      unpackAllOverlappedTsFilesToTimeSeriesMetadata(
+          firstTimeSeriesMetadata.getStatistics().getEndTime());
+      unpackAllOverlappedTimeSeriesMetadataToCachedChunkMetadata(
+          firstTimeSeriesMetadata.getStatistics().getEndTime(), true);
     } else {
       /*
        * first time series metadata is already unpacked, consume cached ChunkMetadata
@@ -230,23 +223,27 @@ class SeriesReader {
       if (!cachedChunkMetadata.isEmpty()) {
         firstChunkMetadata = cachedChunkMetadata.poll();
         unpackAllOverlappedTsFilesToTimeSeriesMetadata(firstChunkMetadata.getEndTime());
-        unpackAllOverlappedTimeSeriesMetadataToCachedChunkMetadata(firstChunkMetadata.getEndTime(), false);
+        unpackAllOverlappedTimeSeriesMetadataToCachedChunkMetadata(
+            firstChunkMetadata.getEndTime(), false);
       }
-
     }
 
     return firstChunkMetadata != null;
   }
 
-  private void unpackAllOverlappedTimeSeriesMetadataToCachedChunkMetadata(long endTime, boolean init) throws IOException {
-    while (!seqTimeSeriesMetadata.isEmpty() && endTime >= seqTimeSeriesMetadata.get(0).getStatistics().getStartTime()) {
+  private void unpackAllOverlappedTimeSeriesMetadataToCachedChunkMetadata(
+      long endTime, boolean init) throws IOException {
+    while (!seqTimeSeriesMetadata.isEmpty()
+        && endTime >= seqTimeSeriesMetadata.get(0).getStatistics().getStartTime()) {
       unpackOneTimeSeriesMetadata(seqTimeSeriesMetadata.remove(0));
     }
-    while (!unSeqTimeSeriesMetadata.isEmpty() && endTime >= unSeqTimeSeriesMetadata.peek().getStatistics().getStartTime()) {
+    while (!unSeqTimeSeriesMetadata.isEmpty()
+        && endTime >= unSeqTimeSeriesMetadata.peek().getStatistics().getStartTime()) {
       unpackOneTimeSeriesMetadata(unSeqTimeSeriesMetadata.poll());
     }
 
-    if (firstTimeSeriesMetadata != null && endTime >= firstTimeSeriesMetadata.getStatistics().getStartTime()) {
+    if (firstTimeSeriesMetadata != null
+        && endTime >= firstTimeSeriesMetadata.getStatistics().getStartTime()) {
       unpackOneTimeSeriesMetadata(firstTimeSeriesMetadata);
       firstTimeSeriesMetadata = null;
     }
@@ -255,7 +252,8 @@ class SeriesReader {
     }
   }
 
-  private void unpackOneTimeSeriesMetadata(TimeseriesMetadata timeSeriesMetadata) throws IOException {
+  private void unpackOneTimeSeriesMetadata(TimeseriesMetadata timeSeriesMetadata)
+      throws IOException {
     cachedChunkMetadata.addAll(FileLoaderUtils.loadChunkMetadataList(timeSeriesMetadata));
   }
 
@@ -266,7 +264,7 @@ class SeriesReader {
 
     Statistics chunkStatistics = firstChunkMetadata.getStatistics();
     return !cachedChunkMetadata.isEmpty()
-            && chunkStatistics.getEndTime() >= cachedChunkMetadata.peek().getStartTime();
+        && chunkStatistics.getEndTime() >= cachedChunkMetadata.peek().getStartTime();
   }
 
   Statistics currentChunkStatistics() {
@@ -309,7 +307,6 @@ class SeriesReader {
       return true;
     }
 
-
     /*
      * construct first page reader
      */
@@ -346,7 +343,7 @@ class SeriesReader {
     while (firstPageReader == null && !cachedPageReaders.isEmpty()) {
       firstPageReader = cachedPageReaders.poll();
       if (!cachedPageReaders.isEmpty()
-              && firstPageReader.getEndTime() >= cachedPageReaders.peek().getStartTime()) {
+          && firstPageReader.getEndTime() >= cachedPageReaders.peek().getStartTime()) {
         /*
          * next page is overlapped, read overlapped data and cache it
          */
@@ -378,7 +375,10 @@ class SeriesReader {
 
   private void unpackOneChunkMetaData(ChunkMetadata chunkMetaData) throws IOException {
     FileLoaderUtils.loadPageReaderList(chunkMetaData, timeFilter)
-            .forEach(pageReader -> cachedPageReaders.add(new VersionPageReader(chunkMetaData.getVersion(), pageReader)));
+        .forEach(
+            pageReader ->
+                cachedPageReaders.add(
+                    new VersionPageReader(chunkMetaData.getVersion(), pageReader)));
   }
 
   /**
@@ -483,7 +483,8 @@ class SeriesReader {
           }
 
           unpackAllOverlappedTsFilesToTimeSeriesMetadata(timeValuePair.getTimestamp());
-          unpackAllOverlappedTimeSeriesMetadataToCachedChunkMetadata(timeValuePair.getTimestamp(), false);
+          unpackAllOverlappedTimeSeriesMetadataToCachedChunkMetadata(
+              timeValuePair.getTimestamp(), false);
           unpackAllOverlappedChunkMetadataToCachedPageReaders(timeValuePair.getTimestamp(), false);
           unpackAllOverlappedCachedPageReadersToMergeReader(timeValuePair.getTimestamp());
 
@@ -568,15 +569,14 @@ class SeriesReader {
     throw new IOException("No more batch data");
   }
 
-  private LinkedList<TsFileResource> sortUnSeqFileResources(
-      List<TsFileResource> tsFileResources) {
+  private LinkedList<TsFileResource> sortUnSeqFileResources(List<TsFileResource> tsFileResources) {
     return tsFileResources.stream()
-            .sorted(Comparator.comparingLong(tsFileResource -> tsFileResource.getStartTimeMap().get(seriesPath.getDevice())))
-            .collect(Collectors.toCollection(LinkedList::new));
-
+        .sorted(
+            Comparator.comparingLong(
+                tsFileResource -> tsFileResource.getStartTimeMap().get(seriesPath.getDevice())))
+        .collect(Collectors.toCollection(LinkedList::new));
   }
 
-
   /**
    * unpack all overlapped seq/unseq files and find the first TimeSeriesMetadata
    *
@@ -589,9 +589,9 @@ class SeriesReader {
      * Fill sequence TimeSeriesMetadata List until it is not empty
      */
     while (seqTimeSeriesMetadata.isEmpty() && !seqFileResource.isEmpty()) {
-      TimeseriesMetadata timeseriesMetadata = FileLoaderUtils
-          .loadTimeSeriesMetadata(seqFileResource.remove(0), seriesPath, context, getAnyFilter(),
-              allSensors);
+      TimeseriesMetadata timeseriesMetadata =
+          FileLoaderUtils.loadTimeSeriesMetadata(
+              seqFileResource.remove(0), seriesPath, context, getAnyFilter(), allSensors);
       if (timeseriesMetadata != null) {
         seqTimeSeriesMetadata.add(timeseriesMetadata);
       }
@@ -601,9 +601,9 @@ class SeriesReader {
      * Fill unSequence TimeSeriesMetadata Priority Queue until it is not empty
      */
     while (unSeqTimeSeriesMetadata.isEmpty() && !unseqFileResource.isEmpty()) {
-      TimeseriesMetadata timeseriesMetadata = FileLoaderUtils
-          .loadTimeSeriesMetadata(unseqFileResource.remove(0), seriesPath, context, getAnyFilter(),
-              allSensors);
+      TimeseriesMetadata timeseriesMetadata =
+          FileLoaderUtils.loadTimeSeriesMetadata(
+              unseqFileResource.remove(0), seriesPath, context, getAnyFilter(), allSensors);
       if (timeseriesMetadata != null) {
         timeseriesMetadata.setModified(true);
         unSeqTimeSeriesMetadata.add(timeseriesMetadata);
@@ -649,7 +649,8 @@ class SeriesReader {
     } else if (!seqTimeSeriesMetadata.isEmpty()) {
       // has seq and unseq
       // has seq and unseq
-      if (seqTimeSeriesMetadata.get(0).getStatistics().getStartTime() <= unSeqTimeSeriesMetadata.peek().getStatistics().getStartTime()) {
+      if (seqTimeSeriesMetadata.get(0).getStatistics().getStartTime()
+          <= unSeqTimeSeriesMetadata.peek().getStatistics().getStartTime()) {
         firstTimeSeriesMetadata = seqTimeSeriesMetadata.remove(0);
       } else {
         firstTimeSeriesMetadata = unSeqTimeSeriesMetadata.poll();
@@ -658,15 +659,21 @@ class SeriesReader {
   }
 
   private void unpackAllOverlappedTsFilesToTimeSeriesMetadata(long endTime) throws IOException {
-    while (!unseqFileResource.isEmpty() && endTime >= unseqFileResource.get(0).getStartTimeMap().get(seriesPath.getDevice())) {
-      TimeseriesMetadata timeseriesMetadata = FileLoaderUtils.loadTimeSeriesMetadata(unseqFileResource.remove(0), seriesPath, context, getAnyFilter(), allSensors);
+    while (!unseqFileResource.isEmpty()
+        && endTime >= unseqFileResource.get(0).getStartTimeMap().get(seriesPath.getDevice())) {
+      TimeseriesMetadata timeseriesMetadata =
+          FileLoaderUtils.loadTimeSeriesMetadata(
+              unseqFileResource.remove(0), seriesPath, context, getAnyFilter(), allSensors);
       if (timeseriesMetadata != null) {
         timeseriesMetadata.setModified(true);
         unSeqTimeSeriesMetadata.add(timeseriesMetadata);
       }
     }
-    while (!seqFileResource.isEmpty() && endTime >= seqFileResource.get(0).getStartTimeMap().get(seriesPath.getDevice())) {
-      TimeseriesMetadata timeseriesMetadata = FileLoaderUtils.loadTimeSeriesMetadata(seqFileResource.remove(0), seriesPath, context, getAnyFilter(), allSensors);
+    while (!seqFileResource.isEmpty()
+        && endTime >= seqFileResource.get(0).getStartTimeMap().get(seriesPath.getDevice())) {
+      TimeseriesMetadata timeseriesMetadata =
+          FileLoaderUtils.loadTimeSeriesMetadata(
+              seqFileResource.remove(0), seriesPath, context, getAnyFilter(), allSensors);
       if (timeseriesMetadata != null) {
         seqTimeSeriesMetadata.add(timeseriesMetadata);
       }
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagAlterIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagAlterIT.java
new file mode 100644
index 0000000..4ec5888
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagAlterIT.java
@@ -0,0 +1,365 @@
+/*
+ * 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.iotdb.db.integration;
+
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+import static org.junit.Assert.*;
+
+public class IoTDBTagAlterIT {
+
+  @Before
+  public void setUp() throws Exception {
+    EnvironmentUtils.closeStatMonitor();
+    EnvironmentUtils.envSetUp();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void renameTest() throws ClassNotFoundException {
+    String[] ret = {"root.turbine.d1.s1,temperature,root.turbine,FLOAT,RLE,SNAPPY,v1,v2,v1,v2"};
+    String sql = "create timeseries root.turbine.d1.s1(temperature) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+            "tags(tag1=v1, tag2=v2) attributes(attr1=v1, attr2=v2)";
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    try (Connection connection = DriverManager
+            .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      statement.execute(sql);
+      boolean hasResult = statement.execute("show timeseries");
+      assertTrue(hasResult);
+      ResultSet resultSet = statement.getResultSet();
+      int count = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString("timeseries")
+                + "," + resultSet.getString("alias")
+                + "," + resultSet.getString("storage group")
+                + "," + resultSet.getString("dataType")
+                + "," + resultSet.getString("encoding")
+                + "," + resultSet.getString("compression")
+                + "," + resultSet.getString("attr1")
+                + "," + resultSet.getString("attr2")
+                + "," + resultSet.getString("tag1")
+                + "," + resultSet.getString("tag2");
+        assertEquals(ret[count], ans);
+        count++;
+      }
+      assertEquals(ret.length, count);
+
+      try {
+        statement.execute("ALTER timeseries root.turbine.d1.s1 RENAME tag3 TO tagNew3");
+        fail();
+      }  catch (Exception e) {
+        assertTrue(e.getMessage().contains("TimeSeries [root.turbine.d1.s1] does not have tag/attribute [tag3]."));
+      }
+
+      try {
+        statement.execute("ALTER timeseries root.turbine.d1.s1 RENAME tag1 TO tag2");
+        fail();
+      }  catch (Exception e) {
+        assertTrue(e.getMessage().contains("TimeSeries [root.turbine.d1.s1] already has a tag/attribute named [tag2]."));
+      }
+
+      statement.execute("ALTER timeseries root.turbine.d1.s1 RENAME tag1 TO tagNew1");
+      hasResult = statement.execute("show timeseries");
+      assertTrue(hasResult);
+      resultSet = statement.getResultSet();
+      count = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString("timeseries")
+                + "," + resultSet.getString("alias")
+                + "," + resultSet.getString("storage group")
+                + "," + resultSet.getString("dataType")
+                + "," + resultSet.getString("encoding")
+                + "," + resultSet.getString("compression")
+                + "," + resultSet.getString("attr1")
+                + "," + resultSet.getString("attr2")
+                + "," + resultSet.getString("tagNew1")
+                + "," + resultSet.getString("tag2");
+        assertEquals(ret[count], ans);
+        count++;
+      }
+      assertEquals(ret.length, count);
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail();
+    }
+  }
+
+  @Test
+  public void setTest() throws ClassNotFoundException {
+    String[] ret = {"root.turbine.d1.s1,temperature,root.turbine,FLOAT,RLE,SNAPPY,v1,v2,v1,v2"};
+    String[] ret2 = {"root.turbine.d1.s1,temperature,root.turbine,FLOAT,RLE,SNAPPY,v1,newV2,newV1,v2"};
+
+    String sql = "create timeseries root.turbine.d1.s1(temperature) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+            "tags(tag1=v1, tag2=v2) attributes(attr1=v1, attr2=v2)";
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    try (Connection connection = DriverManager
+            .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      statement.execute(sql);
+      boolean hasResult = statement.execute("show timeseries");
+      assertTrue(hasResult);
+      ResultSet resultSet = statement.getResultSet();
+      int count = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString("timeseries")
+                + "," + resultSet.getString("alias")
+                + "," + resultSet.getString("storage group")
+                + "," + resultSet.getString("dataType")
+                + "," + resultSet.getString("encoding")
+                + "," + resultSet.getString("compression")
+                + "," + resultSet.getString("attr1")
+                + "," + resultSet.getString("attr2")
+                + "," + resultSet.getString("tag1")
+                + "," + resultSet.getString("tag2");
+        assertEquals(ret[count], ans);
+        count++;
+      }
+      assertEquals(ret.length, count);
+
+      try {
+        statement.execute("ALTER timeseries root.turbine.d1.s1 SET tag3=v3");
+        fail();
+      }  catch (Exception e) {
+        assertTrue(e.getMessage().contains("TimeSeries [root.turbine.d1.s1] does not have tag/attribute [tag3]."));
+      }
+
+      statement.execute("ALTER timeseries root.turbine.d1.s1 SET tag1=newV1, attr2=newV2");
+      hasResult = statement.execute("show timeseries");
+      assertTrue(hasResult);
+      resultSet = statement.getResultSet();
+      count = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString("timeseries")
+                + "," + resultSet.getString("alias")
+                + "," + resultSet.getString("storage group")
+                + "," + resultSet.getString("dataType")
+                + "," + resultSet.getString("encoding")
+                + "," + resultSet.getString("compression")
+                + "," + resultSet.getString("attr1")
+                + "," + resultSet.getString("attr2")
+                + "," + resultSet.getString("tag1")
+                + "," + resultSet.getString("tag2");
+        assertEquals(ret2[count], ans);
+        count++;
+      }
+      assertEquals(ret2.length, count);
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail();
+    }
+  }
+
+  @Test
+  public void dropTest() throws ClassNotFoundException {
+    String[] ret = {"root.turbine.d1.s1,temperature,root.turbine,FLOAT,RLE,SNAPPY,v1,v2,v1,v2"};
+    String[] ret2 = {"root.turbine.d1.s1,temperature,root.turbine,FLOAT,RLE,SNAPPY,v2,v2"};
+
+    String sql = "create timeseries root.turbine.d1.s1(temperature) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+            "tags(tag1=v1, tag2=v2) attributes(attr1=v1, attr2=v2)";
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    try (Connection connection = DriverManager
+            .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      statement.execute(sql);
+      boolean hasResult = statement.execute("show timeseries");
+      assertTrue(hasResult);
+      ResultSet resultSet = statement.getResultSet();
+      int count = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString("timeseries")
+                + "," + resultSet.getString("alias")
+                + "," + resultSet.getString("storage group")
+                + "," + resultSet.getString("dataType")
+                + "," + resultSet.getString("encoding")
+                + "," + resultSet.getString("compression")
+                + "," + resultSet.getString("attr1")
+                + "," + resultSet.getString("attr2")
+                + "," + resultSet.getString("tag1")
+                + "," + resultSet.getString("tag2");
+        assertEquals(ret[count], ans);
+        count++;
+      }
+      assertEquals(ret.length, count);
+
+      statement.execute("ALTER timeseries root.turbine.d1.s1 DROP attr1,tag1");
+      hasResult = statement.execute("show timeseries");
+      assertTrue(hasResult);
+      resultSet = statement.getResultSet();
+      count = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString("timeseries")
+                + "," + resultSet.getString("alias")
+                + "," + resultSet.getString("storage group")
+                + "," + resultSet.getString("dataType")
+                + "," + resultSet.getString("encoding")
+                + "," + resultSet.getString("compression")
+                + "," + resultSet.getString("attr2")
+                + "," + resultSet.getString("tag2");
+        assertEquals(ret2[count], ans);
+        count++;
+      }
+      assertEquals(ret2.length, count);
+
+      try {
+        statement.execute("show timeseries where tag1=v1");
+        fail();
+      } catch (Exception e) {
+        assertTrue(e.getMessage().contains("The key tag1 is not a tag"));
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail();
+    }
+  }
+
+  @Test
+  public void addTagTest() throws ClassNotFoundException {
+    String[] ret = {"root.turbine.d1.s1,temperature,root.turbine,FLOAT,RLE,SNAPPY,v1,v2,v1,v2"};
+    String[] ret2 = {"root.turbine.d1.s1,temperature,root.turbine,FLOAT,RLE,SNAPPY,v1,v2,v1,v2,v3,v4"};
+
+    String sql = "create timeseries root.turbine.d1.s1(temperature) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+            "tags(tag1=v1, tag2=v2) attributes(attr1=v1, attr2=v2)";
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    try (Connection connection = DriverManager
+            .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      statement.execute(sql);
+      boolean hasResult = statement.execute("show timeseries");
+      assertTrue(hasResult);
+      ResultSet resultSet = statement.getResultSet();
+      int count = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString("timeseries")
+                + "," + resultSet.getString("alias")
+                + "," + resultSet.getString("storage group")
+                + "," + resultSet.getString("dataType")
+                + "," + resultSet.getString("encoding")
+                + "," + resultSet.getString("compression")
+                + "," + resultSet.getString("attr1")
+                + "," + resultSet.getString("attr2")
+                + "," + resultSet.getString("tag1")
+                + "," + resultSet.getString("tag2");
+        assertEquals(ret[count], ans);
+        count++;
+      }
+      assertEquals(ret.length, count);
+
+      statement.execute("ALTER timeseries root.turbine.d1.s1 ADD TAGS tag3=v3, tag4=v4");
+      hasResult = statement.execute("show timeseries where tag3=v3");
+      assertTrue(hasResult);
+      resultSet = statement.getResultSet();
+      count = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString("timeseries")
+                + "," + resultSet.getString("alias")
+                + "," + resultSet.getString("storage group")
+                + "," + resultSet.getString("dataType")
+                + "," + resultSet.getString("encoding")
+                + "," + resultSet.getString("compression")
+                + "," + resultSet.getString("attr1")
+                + "," + resultSet.getString("attr2")
+                + "," + resultSet.getString("tag1")
+                + "," + resultSet.getString("tag2")
+                + "," + resultSet.getString("tag3")
+                + "," + resultSet.getString("tag4");
+        assertEquals(ret2[count], ans);
+        count++;
+      }
+      assertEquals(ret2.length, count);
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail();
+    }
+  }
+
+  @Test
+  public void addAttributeTest() throws ClassNotFoundException {
+    String[] ret = {"root.turbine.d1.s1,temperature,root.turbine,FLOAT,RLE,SNAPPY,v1,v2,v1,v2"};
+    String[] ret2 = {"root.turbine.d1.s1,temperature,root.turbine,FLOAT,RLE,SNAPPY,v1,v2,v3,v4,v1,v2"};
+
+    String sql = "create timeseries root.turbine.d1.s1(temperature) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+            "tags(tag1=v1, tag2=v2) attributes(attr1=v1, attr2=v2)";
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    try (Connection connection = DriverManager
+            .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      statement.execute(sql);
+      boolean hasResult = statement.execute("show timeseries");
+      assertTrue(hasResult);
+      ResultSet resultSet = statement.getResultSet();
+      int count = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString("timeseries")
+                + "," + resultSet.getString("alias")
+                + "," + resultSet.getString("storage group")
+                + "," + resultSet.getString("dataType")
+                + "," + resultSet.getString("encoding")
+                + "," + resultSet.getString("compression")
+                + "," + resultSet.getString("attr1")
+                + "," + resultSet.getString("attr2")
+                + "," + resultSet.getString("tag1")
+                + "," + resultSet.getString("tag2");
+        assertEquals(ret[count], ans);
+        count++;
+      }
+      assertEquals(ret.length, count);
+
+      statement.execute("ALTER timeseries root.turbine.d1.s1 ADD ATTRIBUTES attr3=v3, attr4=v4");
+      hasResult = statement.execute("show timeseries");
+      assertTrue(hasResult);
+      resultSet = statement.getResultSet();
+      count = 0;
+      while (resultSet.next()) {
+        String ans = resultSet.getString("timeseries")
+                + "," + resultSet.getString("alias")
+                + "," + resultSet.getString("storage group")
+                + "," + resultSet.getString("dataType")
+                + "," + resultSet.getString("encoding")
+                + "," + resultSet.getString("compression")
+                + "," + resultSet.getString("attr1")
+                + "," + resultSet.getString("attr2")
+                + "," + resultSet.getString("attr3")
+                + "," + resultSet.getString("attr4")
+                + "," + resultSet.getString("tag1")
+                + "," + resultSet.getString("tag2");
+        assertEquals(ret2[count], ans);
+        count++;
+      }
+      assertEquals(ret2.length, count);
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail();
+    }
+  }
+
+}
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagIT.java
index ccd8e14..51d0080 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTagIT.java
@@ -832,4 +832,20 @@ public class IoTDBTagIT {
       assertTrue(e.getMessage().contains("The key H_Alarm is not a tag"));
     }
   }
+
+  @Test
+  public void sameNameTest() throws ClassNotFoundException {
+    String[] ret = {"root.turbine.d1.s1,temperature,root.turbine,FLOAT,RLE,SNAPPY,v1,v2,v1,v2"};
+    String sql = "create timeseries root.turbine.d1.s1(temperature) with datatype=FLOAT, encoding=RLE, compression=SNAPPY " +
+            "tags(tag1=v1, tag2=v2) attributes(tag1=v1, attr2=v2)";
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    try (Connection connection = DriverManager
+            .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      statement.execute(sql);
+      fail();
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains("Tag and attribute shouldn't have the same property key"));
+    }
+  }
 }