You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ea...@apache.org on 2019/04/24 06:40:44 UTC

[incubator-iotdb] branch master updated: [IOTDB-76] Reformat MManager.getMetadataInString() in JSON format (#132)

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

east 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 bb9b958  [IOTDB-76] Reformat MManager.getMetadataInString() in JSON format (#132)
bb9b958 is described below

commit bb9b958f7d4a637997a95a5441d0bd946eb6bf6d
Author: DongFang Mao <95...@qq.com>
AuthorDate: Wed Apr 24 14:40:40 2019 +0800

    [IOTDB-76] Reformat MManager.getMetadataInString() in JSON format (#132)
    
    * reimplement MTree.toString() by JSON
    add MManager.combineMetadataInStrings(String[] metadatas) for cluster module
    
    * move add TIMESERIES HEADER from MGraph to TSServiceImpl
    fix bug when MTree combine JSONObjects
    
    * update timeseries metadata format in docs
---
 docs/Documentation/QuickStart.md                   |  74 +++++-----
 .../java/org/apache/iotdb/db/metadata/MGraph.java  |  28 +++-
 .../org/apache/iotdb/db/metadata/MManager.java     |  23 ++-
 .../java/org/apache/iotdb/db/metadata/MTree.java   | 164 +++++++++++++--------
 .../org/apache/iotdb/db/monitor/StatMonitor.java   |   2 +-
 .../iotdb/db/qp/executor/OverflowQPExecutor.java   |   2 +-
 .../org/apache/iotdb/db/service/TSServiceImpl.java |   1 +
 .../iotdb/db/integration/IoTDBCompleteIT.java      | 154 ++++++++++---------
 .../iotdb/db/integration/IoTDBMetadataFetchIT.java |  45 +++---
 .../org/apache/iotdb/db/metadata/MGraphTest.java   |  79 ++++++++++
 .../iotdb/db/metadata/MManagerBasicTest.java       |  22 +--
 .../org/apache/iotdb/db/metadata/MTreeTest.java    |  43 ++++++
 12 files changed, 426 insertions(+), 211 deletions(-)

diff --git a/docs/Documentation/QuickStart.md b/docs/Documentation/QuickStart.md
index 7c173dd..0fea0c6 100755
--- a/docs/Documentation/QuickStart.md
+++ b/docs/Documentation/QuickStart.md
@@ -217,20 +217,22 @@ Till now, we have already create a table called root.vehicle and add a colomn ca
 IoTDB> SHOW TIMESERIES
 ===  Timeseries Tree  ===
 
-root:{
-    ln:{
-        wf01:{
-            wt01:{
-                status:{
-                     DataType: BOOLEAN,
-                     Encoding: PLAIN,
-                     Compressor: UNCOMPRESSED,
-                     args: {},
-                     StorageGroup: root.vehicle
-                }
-            }
-        }
-    }
+{
+	"root":{
+		"ln":{
+			"wf01":{
+				"wt01":{
+					"status":{
+						"args":"{}",
+						"StorageGroup":"root.ln",
+						"DataType":"BOOLEAN",
+						"Compressor":"UNCOMPRESSED",
+						"Encoding":"PLAIN"
+					}
+				}
+			}
+		}
+	}
 }
 ```
 
@@ -241,27 +243,29 @@ IoTDB> CREATE TIMESERIES root.ln.wf01.wt01.temperature WITH DATATYPE=FLOAT, ENCO
 IoTDB> SHOW TIMESERIES
 ===  Timeseries Tree  ===
 
-root:{
-    ln:{
-        wf01:{
-            wt01:{
-                status:{
-                     DataType: BOOLEAN,
-                     Encoding: PLAIN,
-                     Compressor: UNCOMPRESSED,
-                     args: {},
-                     StorageGroup: root.vehicle
-                },
-                temperature:{
-                     DataType: FLOAT,
-                     Encoding: RLE,
-                     Compressor: UNCOMPRESSED,
-                     args: {},
-                     StorageGroup: root.vehicle
-                }
-            }
-        }
-    }
+{
+	"root":{
+		"ln":{
+			"wf01":{
+				"wt01":{
+					"temperature":{
+						"args":"{}",
+						"StorageGroup":"root.ln",
+						"DataType":"FLOAT",
+						"Compressor":"UNCOMPRESSED",
+						"Encoding":"RLE"
+					},
+					"status":{
+						"args":"{}",
+						"StorageGroup":"root.ln",
+						"DataType":"BOOLEAN",
+						"Compressor":"UNCOMPRESSED",
+						"Encoding":"PLAIN"
+					}
+				}
+			}
+		}
+	}
 }
 ```
 Now, for your conveniect, SHOW TIMESERIES clause also supports extention syntax, the pattern is (for further details, check Chapter x):
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/metadata/MGraph.java b/iotdb/src/main/java/org/apache/iotdb/db/metadata/MGraph.java
index 5cf2847..cc6e1cf 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/metadata/MGraph.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/metadata/MGraph.java
@@ -20,12 +20,14 @@ package org.apache.iotdb.db.metadata;
 
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import org.apache.iotdb.db.exception.MetadataArgsErrorException;
 import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
@@ -59,12 +61,24 @@ public class MGraph implements Serializable {
   }
 
   /**
+   * this is just for compatibility
+   */
+  public void addPathToMTree(String path, String dataType, String encoding)
+      throws PathErrorException {
+    TSDataType tsDataType = TSDataType.valueOf(dataType);
+    TSEncoding tsEncoding = TSEncoding.valueOf(encoding);
+    CompressionType compressionType = CompressionType.valueOf(TSFileConfig.compressor);
+    addPathToMTree(path, tsDataType, tsEncoding, compressionType,
+        Collections.emptyMap());
+  }
+
+  /**
    * Add a seriesPath to Metadata Tree.
    *
    * @param path Format: root.node.(node)*
    */
   public void addPathToMTree(String path, TSDataType dataType, TSEncoding encoding,
-      CompressionType compressor, Map<String, String> props) throws PathErrorException, MetadataArgsErrorException {
+      CompressionType compressor, Map<String, String> props) throws PathErrorException {
     String[] nodes = path.trim().split(DOUB_SEPARATOR);
     if (nodes.length == 0) {
       throw new PathErrorException("Timeseries is null");
@@ -365,9 +379,13 @@ public class MGraph implements Serializable {
    */
   @Override
   public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("===  Timeseries Tree  ===\n\n");
-    sb.append(mtree.toString());
-    return sb.toString();
+    return mtree.toString();
+  }
+
+  /**
+   * combine multiple metadata in string format
+   */
+  public static String combineMetadataInStrings(String[] metadatas) {
+    return MTree.combineMetadataInStrings(metadatas);
   }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/metadata/MManager.java b/iotdb/src/main/java/org/apache/iotdb/db/metadata/MManager.java
index eef21ca..ea4ec6f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/metadata/MManager.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/metadata/MManager.java
@@ -60,6 +60,8 @@ public class MManager {
 
   private static final Logger LOGGER = LoggerFactory.getLogger(MManager.class);
   private static final String ROOT_NAME = MetadataConstant.ROOT;
+  public static final String TIME_SERIES_TREE_HEADER = "===  Timeseries Tree  ===\n\n";
+
   // the lock for read/write
   private ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
   // The file storing the serialize info for metadata
@@ -259,7 +261,7 @@ public class MManager {
    */
   public void addPathToMTree(String path, TSDataType dataType, TSEncoding encoding,
       CompressionType compressor, Map<String, String> props)
-      throws PathErrorException, IOException, MetadataArgsErrorException {
+      throws PathErrorException, IOException {
 
     lock.writeLock().lock();
     try {
@@ -292,7 +294,7 @@ public class MManager {
    * @param encoding the encoding function {@code Encoding} for the timeseries
    */
   public void addPathToMTree(String path, String dataType, String encoding)
-      throws PathErrorException, IOException, MetadataArgsErrorException {
+      throws PathErrorException, IOException {
     TSDataType tsDataType = TSDataType.valueOf(dataType);
     TSEncoding tsEncoding = TSEncoding.valueOf(encoding);
     CompressionType type = CompressionType.valueOf(TSFileConfig.compressor);
@@ -985,13 +987,28 @@ public class MManager {
 
     lock.readLock().lock();
     try {
-      return mgraph.toString();
+      StringBuilder builder = new StringBuilder();
+      builder.append(TIME_SERIES_TREE_HEADER).append(mgraph.toString());
+      return builder.toString();
     } finally {
       lock.readLock().unlock();
     }
   }
 
   /**
+   * combine multiple metadata in string format
+   */
+  public static String combineMetadataInStrings(String[] metadatas) {
+    for (int i = 0; i < metadatas.length; i++) {
+      metadatas[i] = metadatas[i].replace(TIME_SERIES_TREE_HEADER, "");
+    }
+    String res = MGraph.combineMetadataInStrings(metadatas);
+    StringBuilder builder = new StringBuilder();
+    builder.append(TIME_SERIES_TREE_HEADER).append(res);
+    return builder.toString();
+  }
+
+  /**
    * Check whether {@code seriesPath} exists and whether {@code seriesPath} has been set storage
    * level.
    *
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/metadata/MTree.java b/iotdb/src/main/java/org/apache/iotdb/db/metadata/MTree.java
index 2c7e5b0..f5bf6a2 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/metadata/MTree.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/metadata/MTree.java
@@ -18,6 +18,9 @@
  */
 package org.apache.iotdb.db.metadata;
 
+import com.alibaba.fastjson.JSON;
+import com.alibaba.fastjson.JSONObject;
+import com.alibaba.fastjson.serializer.SerializerFeature;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -25,6 +28,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import org.apache.iotdb.db.exception.PathErrorException;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
@@ -67,6 +71,7 @@ public class MTree implements Serializable {
     addTimeseriesPath(timeseriesPath, tsDataType, tsEncoding, compressionType,
         Collections.emptyMap());
   }
+
   /**
    * function for adding timeseries.It should check whether seriesPath exists.
    */
@@ -214,7 +219,7 @@ public class MTree implements Serializable {
   }
 
   /**
-   * check whether the input path is storage group or not
+   * Check whether the input path is storage group or not
    * @param path input path
    * @return if it is storage group, return true. Else return false
    * @apiNote :for cluster
@@ -583,28 +588,45 @@ public class MTree implements Serializable {
    * Get all the storage group seriesPaths for one seriesPath.
    *
    * @return List storage group seriesPath list
+   * @apiNote :for cluster
    */
-  public List<String> getAllFileNamesByPath(String path) throws PathErrorException {
+  public List<String> getAllFileNamesByPath(String pathReg) throws PathErrorException {
+    ArrayList<String> fileNames = new ArrayList<>();
+    String[] nodes = pathReg.split(DOUB_SEPARATOR);
+    if (nodes.length == 0 || !nodes[0].equals(getRoot().getName())) {
+      throw new PathErrorException(String.format(SERIES_NOT_CORRECT, pathReg));
+    }
+    findFileName(getRoot(), nodes, 1, "", fileNames);
+    return fileNames;
+  }
 
-    List<String> sgList = new ArrayList<>();
-    String[] nodes = path.split(DOUB_SEPARATOR);
-    MNode cur = getRoot();
-    for (int i = 1; i < nodes.length; i++) {
-      if (cur == null) {
-        throw new PathErrorException(
-            String.format(NOT_SERIES_PATH,
-                path));
+  /**
+   * Recursively find all fileName according to a specific path
+   * @apiNote :for cluster
+   */
+  private void findFileName(MNode node, String[] nodes, int idx, String parent,
+      ArrayList<String> paths) {
+    if (node.isStorageLevel()) {
+      paths.add(node.getDataFileName());
+      return;
+    }
+    String nodeReg;
+    if (idx >= nodes.length) {
+      nodeReg = "*";
+    } else {
+      nodeReg = nodes[idx];
+    }
+
+    if (!("*").equals(nodeReg)) {
+      if (node.hasChild(nodeReg)) {
+        findFileName(node.getChild(nodeReg), nodes, idx + 1, parent + node.getName() + ".", paths);
       }
-      if (cur.isStorageLevel()) {
-        sgList.add(cur.getDataFileName());
-        return sgList;
+    } else {
+      for (MNode child : node.getChildren().values()) {
+        findFileName(child, nodes, idx + 1, parent + node.getName() + ".", paths);
       }
-      cur = cur.getChild(nodes[i]);
     }
-    if (sgList.isEmpty()) {
-      getAllStorageGroupsOfNode(cur, path, sgList);
-    }
-    return sgList;
+    return;
   }
 
   private void getAllStorageGroupsOfNode(MNode node, String path, List<String> sgList) {
@@ -1012,59 +1034,79 @@ public class MTree implements Serializable {
 
   @Override
   public String toString() {
-    return mnodeToString(getRoot(), 0);
+    return jsonToString(toJson());
   }
 
-  private String mnodeToString(MNode node, int tab) {
-    StringBuilder builder = new StringBuilder();
-    for (int i = 0; i < tab; i++) {
-      builder.append(QUAD_SPACE);
-    }
-    builder.append(node.getName());
+  private static String jsonToString(JSONObject jsonObject) {
+    return JSON.toJSONString(jsonObject, SerializerFeature.PrettyFormat);
+  }
+
+  private JSONObject toJson() {
+    JSONObject jsonObject = new JSONObject();
+    jsonObject.put(getRoot().getName(), mnodeToJSON(getRoot()));
+    return jsonObject;
+  }
+
+  private JSONObject mnodeToJSON(MNode node) {
+    JSONObject jsonObject = new JSONObject();
     if (!node.isLeaf() && node.getChildren().size() > 0) {
-      builder.append(":{\n");
-      int first = 0;
       for (MNode child : node.getChildren().values()) {
-        if (first == 0) {
-          first = 1;
-        } else {
-          builder.append(",\n");
-        }
-        builder.append(mnodeToString(child, tab + 1));
-      }
-      builder.append("\n");
-      for (int i = 0; i < tab; i++) {
-        builder.append(QUAD_SPACE);
+        jsonObject.put(child.getName(), mnodeToJSON(child));
       }
-      builder.append("}");
     } else if (node.isLeaf()) {
-      builder.append(":{\n");
-      builder
-          .append(String.format("%s DataType: %s,\n", getTabs(tab + 1), node.getSchema().getType()));
-      builder
-          .append(String.format("%s Encoding: %s,\n", getTabs(tab + 1), node.getSchema().getEncodingType()));
-
-      builder
-          .append(String.format("%s Compressor: %s,\n", getTabs(tab + 1), node.getSchema().getCompressor()));
-      builder
-          .append(String.format("%s args: %s,\n", getTabs(tab + 1), node.getSchema().getProps()));
-      builder.append(
-          String.format("%s StorageGroup: %s\n", getTabs(tab + 1), node.getDataFileName()));
-      builder.append(getTabs(tab));
-      builder.append("}");
-    }
-    return builder.toString();
-  }
-
-  private String getTabs(int count) {
-    StringBuilder sb = new StringBuilder();
-    for (int i = 0; i < count; i++) {
-      sb.append(QUAD_SPACE);
+      jsonObject.put("DataType", node.getSchema().getType());
+      jsonObject.put("Encoding", node.getSchema().getEncodingType());
+      jsonObject.put("Compressor", node.getSchema().getCompressor());
+      jsonObject.put("args", node.getSchema().getProps().toString());
+      jsonObject.put("StorageGroup", node.getDataFileName());
     }
-    return sb.toString();
+    return jsonObject;
   }
 
   public MNode getRoot() {
     return root;
   }
+
+  /**
+   * combine multiple metadata in string format
+   */
+  public static String combineMetadataInStrings(String[] metadatas) {
+    JSONObject[] jsonObjects = new JSONObject[metadatas.length];
+    for (int i = 0; i < jsonObjects.length; i++) {
+      jsonObjects[i] = JSONObject.parseObject(metadatas[i]);
+    }
+
+    JSONObject root = jsonObjects[0];
+    for (int i = 1; i < jsonObjects.length; i++) {
+      root = combineJSONObjects(root, jsonObjects[i]);
+    }
+    return jsonToString(root);
+  }
+
+  private static JSONObject combineJSONObjects(JSONObject a, JSONObject b) {
+    JSONObject res = new JSONObject();
+
+    Set<String> retainSet = new HashSet<>(a.keySet());
+    retainSet.retainAll(b.keySet());
+    Set<String> aCha = new HashSet<>(a.keySet());
+    Set<String> bCha = new HashSet<>(b.keySet());
+    aCha.removeAll(retainSet);
+    bCha.removeAll(retainSet);
+    for (String key : aCha) {
+      res.put(key, a.getJSONObject(key));
+    }
+    for (String key : bCha) {
+      res.put(key, b.get(key));
+    }
+    for (String key : retainSet) {
+      Object v1 = a.get(key);
+      Object v2 = b.get(key);
+      if (v1 instanceof JSONObject && v2 instanceof JSONObject) {
+        res.put(key, combineJSONObjects((JSONObject) v1, (JSONObject) v2));
+      } else {
+        res.put(key, v1);
+      }
+    }
+    return res;
+  }
 }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/monitor/StatMonitor.java b/iotdb/src/main/java/org/apache/iotdb/db/monitor/StatMonitor.java
index a9a5355..052c324 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/monitor/StatMonitor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/monitor/StatMonitor.java
@@ -169,7 +169,7 @@ public class StatMonitor implements IService {
               Collections.emptyMap());
         }
       }
-    } catch (MetadataArgsErrorException | IOException | PathErrorException e) {
+    } catch (IOException | PathErrorException e) {
       LOGGER.error("Initialize the metadata error.", e);
     }
   }
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java
index 6c603fa..a3bb2a4 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java
@@ -628,7 +628,7 @@ public class OverflowQPExecutor extends QueryProcessExecutor {
         default:
           throw new ProcessorException("unknown namespace type:" + namespaceType);
       }
-    } catch (PathErrorException | IOException | ArgsErrorException | FileNodeManagerException e) {
+    } catch (PathErrorException | IOException | FileNodeManagerException e) {
       throw new ProcessorException(e.getMessage());
     }
     return true;
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
index 3e176bb..f355445 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
@@ -41,6 +41,7 @@ import org.apache.iotdb.db.exception.PathErrorException;
 import org.apache.iotdb.db.exception.ProcessorException;
 import org.apache.iotdb.db.exception.qp.IllegalASTFormatException;
 import org.apache.iotdb.db.exception.qp.QueryProcessorException;
+import org.apache.iotdb.db.metadata.MGraph;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.metadata.Metadata;
 import org.apache.iotdb.db.qp.QueryProcessor;
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java
index e93f44b..167ee79 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java
@@ -73,25 +73,29 @@ public class IoTDBCompleteIT {
         "SHOW TIMESERIES",
         "===  Timeseries Tree  ===\n"
             + "\n"
-            + "root:{\n"
-            + "    vehicle:{\n"
-            + "        d0:{\n"
-            + "            s0:{\n"
-            + "                 DataType: INT32,\n"
-            + "                 Encoding: RLE,\n"
-            + "                 Compressor: UNCOMPRESSED,\n"
-            + "                 args: {},\n"
-            + "                 StorageGroup: root.vehicle\n"
-            + "            }\n"
-            + "        }\n"
-            + "    }\n"
+            + "{\n"
+            + "\t\"root\":{\n"
+            + "\t\t\"vehicle\":{\n"
+            + "\t\t\t\"d0\":{\n"
+            + "\t\t\t\t\"s0\":{\n"
+            + "\t\t\t\t\t\"args\":\"{}\",\n"
+            + "\t\t\t\t\t\"StorageGroup\":\"root.vehicle\",\n"
+            + "\t\t\t\t\t\"DataType\":\"INT32\",\n"
+            + "\t\t\t\t\t\"Compressor\":\"UNCOMPRESSED\",\n"
+            + "\t\t\t\t\t\"Encoding\":\"RLE\"\n"
+            + "\t\t\t\t}\n"
+            + "\t\t\t}\n"
+            + "\t\t}\n"
+            + "\t}\n"
             + "}",
         "DELETE TIMESERIES root.vehicle.d0.s0",
         "SHOW TIMESERIES",
         "===  Timeseries Tree  ===\n"
             + "\n"
-            + "root:{\n"
-            + "    vehicle\n"
+            + "{\n"
+            + "\t\"root\":{\n"
+            + "\t\t\"vehicle\":{}\n"
+            + "\t}\n"
             + "}",
         "CREATE TIMESERIES root.vehicle.d0.s0 WITH DATATYPE=BOOLEAN,ENCODING=PLAIN",
         "CREATE TIMESERIES root.vehicle.d0.s1 WITH DATATYPE=INT64,ENCODING=TS_2DIFF",
@@ -107,70 +111,74 @@ public class IoTDBCompleteIT {
         "SHOW TIMESERIES",
         "===  Timeseries Tree  ===\n"
             + "\n"
-            + "root:{\n"
-            + "    vehicle:{\n"
-            + "        d1:{\n"
-            + "            s5:{\n"
-            + "                 DataType: TEXT,\n"
-            + "                 Encoding: PLAIN,\n"
-            + "                 Compressor: UNCOMPRESSED,\n"
-            + "                 args: {},\n"
-            + "                 StorageGroup: root.vehicle\n"
-            + "            }\n"
-            + "        },\n"
-            + "        d2:{\n"
-            + "            s6:{\n"
-            + "                 DataType: INT32,\n"
-            + "                 Encoding: TS_2DIFF,\n"
-            + "                 Compressor: UNCOMPRESSED,\n"
-            + "                 args: {},\n"
-            + "                 StorageGroup: root.vehicle\n"
-            + "            }\n"
-            + "        },\n"
-            + "        d3:{\n"
-            + "            s7:{\n"
-            + "                 DataType: INT32,\n"
-            + "                 Encoding: RLE,\n"
-            + "                 Compressor: SNAPPY,\n"
-            + "                 args: {},\n"
-            + "                 StorageGroup: root.vehicle\n"
-            + "            }\n"
-            + "        },\n"
-            + "        d4:{\n"
-            + "            s8:{\n"
-            + "                 DataType: INT32,\n"
-            + "                 Encoding: RLE,\n"
-            + "                 Compressor: UNCOMPRESSED,\n"
-            + "                 args: {max_point_number=100},\n"
-            + "                 StorageGroup: root.vehicle\n"
-            + "            }\n"
-            + "        },\n"
-            + "        d5:{\n"
-            + "            s9:{\n"
-            + "                 DataType: FLOAT,\n"
-            + "                 Encoding: PLAIN,\n"
-            + "                 Compressor: SNAPPY,\n"
-            + "                 args: {max_point_number=10},\n"
-            + "                 StorageGroup: root.vehicle\n"
-            + "            }\n"
-            + "        },\n"
-            + "        d6:{\n"
-            + "            s10:{\n"
-            + "                 DataType: DOUBLE,\n"
-            + "                 Encoding: RLE,\n"
-            + "                 Compressor: UNCOMPRESSED,\n"
-            + "                 args: {max_point_number=10},\n"
-            + "                 StorageGroup: root.vehicle\n"
-            + "            }\n"
-            + "        }\n"
-            + "    }\n"
+            + "{\n"
+            + "\t\"root\":{\n"
+            + "\t\t\"vehicle\":{\n"
+            + "\t\t\t\"d4\":{\n"
+            + "\t\t\t\t\"s8\":{\n"
+            + "\t\t\t\t\t\"args\":\"{max_point_number=100}\",\n"
+            + "\t\t\t\t\t\"StorageGroup\":\"root.vehicle\",\n"
+            + "\t\t\t\t\t\"DataType\":\"INT32\",\n"
+            + "\t\t\t\t\t\"Compressor\":\"UNCOMPRESSED\",\n"
+            + "\t\t\t\t\t\"Encoding\":\"RLE\"\n"
+            + "\t\t\t\t}\n"
+            + "\t\t\t},\n"
+            + "\t\t\t\"d5\":{\n"
+            + "\t\t\t\t\"s9\":{\n"
+            + "\t\t\t\t\t\"args\":\"{max_point_number=10}\",\n"
+            + "\t\t\t\t\t\"StorageGroup\":\"root.vehicle\",\n"
+            + "\t\t\t\t\t\"DataType\":\"FLOAT\",\n"
+            + "\t\t\t\t\t\"Compressor\":\"SNAPPY\",\n"
+            + "\t\t\t\t\t\"Encoding\":\"PLAIN\"\n"
+            + "\t\t\t\t}\n"
+            + "\t\t\t},\n"
+            + "\t\t\t\"d6\":{\n"
+            + "\t\t\t\t\"s10\":{\n"
+            + "\t\t\t\t\t\"args\":\"{max_point_number=10}\",\n"
+            + "\t\t\t\t\t\"StorageGroup\":\"root.vehicle\",\n"
+            + "\t\t\t\t\t\"DataType\":\"DOUBLE\",\n"
+            + "\t\t\t\t\t\"Compressor\":\"UNCOMPRESSED\",\n"
+            + "\t\t\t\t\t\"Encoding\":\"RLE\"\n"
+            + "\t\t\t\t}\n"
+            + "\t\t\t},\n"
+            + "\t\t\t\"d1\":{\n"
+            + "\t\t\t\t\"s5\":{\n"
+            + "\t\t\t\t\t\"args\":\"{}\",\n"
+            + "\t\t\t\t\t\"StorageGroup\":\"root.vehicle\",\n"
+            + "\t\t\t\t\t\"DataType\":\"TEXT\",\n"
+            + "\t\t\t\t\t\"Compressor\":\"UNCOMPRESSED\",\n"
+            + "\t\t\t\t\t\"Encoding\":\"PLAIN\"\n"
+            + "\t\t\t\t}\n"
+            + "\t\t\t},\n"
+            + "\t\t\t\"d2\":{\n"
+            + "\t\t\t\t\"s6\":{\n"
+            + "\t\t\t\t\t\"args\":\"{}\",\n"
+            + "\t\t\t\t\t\"StorageGroup\":\"root.vehicle\",\n"
+            + "\t\t\t\t\t\"DataType\":\"INT32\",\n"
+            + "\t\t\t\t\t\"Compressor\":\"UNCOMPRESSED\",\n"
+            + "\t\t\t\t\t\"Encoding\":\"TS_2DIFF\"\n"
+            + "\t\t\t\t}\n"
+            + "\t\t\t},\n"
+            + "\t\t\t\"d3\":{\n"
+            + "\t\t\t\t\"s7\":{\n"
+            + "\t\t\t\t\t\"args\":\"{}\",\n"
+            + "\t\t\t\t\t\"StorageGroup\":\"root.vehicle\",\n"
+            + "\t\t\t\t\t\"DataType\":\"INT32\",\n"
+            + "\t\t\t\t\t\"Compressor\":\"SNAPPY\",\n"
+            + "\t\t\t\t\t\"Encoding\":\"RLE\"\n"
+            + "\t\t\t\t}\n"
+            + "\t\t\t}\n"
+            + "\t\t}\n"
+            + "\t}\n"
             + "}",
         "DELETE TIMESERIES root.vehicle.*",
         "SHOW TIMESERIES",
         "===  Timeseries Tree  ===\n"
             + "\n"
-            + "root:{\n"
-            + "    vehicle\n"
+            + "{\n"
+            + "\t\"root\":{\n"
+            + "\t\t\"vehicle\":{}\n"
+            + "\t}\n"
             + "}"
     };
     executeSQL(sqlS);
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java
index 830328c..355b2af 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java
@@ -360,28 +360,31 @@ public class IoTDBMetadataFetchIT {
     String standard =
         "===  Timeseries Tree  ===\n"
             + "\n"
-            + "root:{\n"
-            + "    ln:{\n"
-            + "        wf01:{\n"
-            + "            wt01:{\n"
-            + "                status:{\n"
-            + "                     DataType: BOOLEAN,\n"
-            + "                     Encoding: PLAIN,\n"
-            + "                     Compressor: UNCOMPRESSED,\n"
-            + "                     args: {},\n"
-            + "                     StorageGroup: root.ln.wf01.wt01\n"
-            + "                },\n"
-            + "                temperature:{\n"
-            + "                     DataType: FLOAT,\n"
-            + "                     Encoding: RLE,\n"
-            + "                     Compressor: SNAPPY,\n"
-            + "                     args: {max_point_number=3},\n"
-            + "                     StorageGroup: root.ln.wf01.wt01\n"
-            + "                }\n"
-            + "            }\n"
-            + "        }\n"
-            + "    }\n"
+            + "{\n"
+            + "\t\"root\":{\n"
+            + "\t\t\"ln\":{\n"
+            + "\t\t\t\"wf01\":{\n"
+            + "\t\t\t\t\"wt01\":{\n"
+            + "\t\t\t\t\t\"temperature\":{\n"
+            + "\t\t\t\t\t\t\"args\":\"{max_point_number=3}\",\n"
+            + "\t\t\t\t\t\t\"StorageGroup\":\"root.ln.wf01.wt01\",\n"
+            + "\t\t\t\t\t\t\"DataType\":\"FLOAT\",\n"
+            + "\t\t\t\t\t\t\"Compressor\":\"SNAPPY\",\n"
+            + "\t\t\t\t\t\t\"Encoding\":\"RLE\"\n"
+            + "\t\t\t\t\t},\n"
+            + "\t\t\t\t\t\"status\":{\n"
+            + "\t\t\t\t\t\t\"args\":\"{}\",\n"
+            + "\t\t\t\t\t\t\"StorageGroup\":\"root.ln.wf01.wt01\",\n"
+            + "\t\t\t\t\t\t\"DataType\":\"BOOLEAN\",\n"
+            + "\t\t\t\t\t\t\"Compressor\":\"UNCOMPRESSED\",\n"
+            + "\t\t\t\t\t\t\"Encoding\":\"PLAIN\"\n"
+            + "\t\t\t\t\t}\n"
+            + "\t\t\t\t}\n"
+            + "\t\t\t}\n"
+            + "\t\t}\n"
+            + "\t}\n"
             + "}";
+
     Assert.assertEquals(standard, metadataInJson);
   }
 }
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/metadata/MGraphTest.java b/iotdb/src/test/java/org/apache/iotdb/db/metadata/MGraphTest.java
new file mode 100644
index 0000000..2d7dd3e
--- /dev/null
+++ b/iotdb/src/test/java/org/apache/iotdb/db/metadata/MGraphTest.java
@@ -0,0 +1,79 @@
+/**
+ * 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.metadata;
+
+import static org.junit.Assert.*;
+
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class MGraphTest {
+
+  @Before
+  public void setUp() throws Exception {
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void testCombineMetadataInStrings() {
+    MGraph root = new MGraph("root");
+    MGraph root1 = new MGraph("root");
+    MGraph root2 = new MGraph("root");
+    MGraph root3 = new MGraph("root");
+    try {
+      root.setStorageLevel("root.a.d0");
+      root.addPathToMTree("root.a.d0.s0", "INT32", "RLE");
+      root.addPathToMTree("root.a.d0.s1", "INT32", "RLE");
+
+      root.setStorageLevel("root.a.d1");
+      root.addPathToMTree("root.a.d1.s0", "INT32", "RLE");
+      root.addPathToMTree("root.a.d1.s1", "INT32", "RLE");
+
+      root.setStorageLevel("root.a.b.d0");
+      root.addPathToMTree("root.a.b.d0.s0", "INT32", "RLE");
+
+      root1.setStorageLevel("root.a.d0");
+      root1.addPathToMTree("root.a.d0.s0", "INT32", "RLE");
+      root1.addPathToMTree("root.a.d0.s1", "INT32", "RLE");
+
+      root2.setStorageLevel("root.a.d1");
+      root2.addPathToMTree("root.a.d1.s0", "INT32", "RLE");
+      root2.addPathToMTree("root.a.d1.s1", "INT32", "RLE");
+
+      root3.setStorageLevel("root.a.b.d0");
+      root3.addPathToMTree("root.a.b.d0.s0", "INT32", "RLE");
+
+      String[] metadatas = new String[3];
+      metadatas[0] = root1.toString();
+      metadatas[1] = root2.toString();
+      metadatas[2] = root3.toString();
+      assertEquals(MGraph.combineMetadataInStrings(metadatas), root.toString());
+    } catch (PathErrorException e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+}
\ No newline at end of file
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java b/iotdb/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
index 2133cc0..36dba77 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/metadata/MManagerBasicTest.java
@@ -79,7 +79,7 @@ public class MManagerBasicTest {
       manager.addPathToMTree("root.laptop.d1.s0", TSDataType.valueOf("INT32"),
           TSEncoding.valueOf("RLE"), compressionType, Collections
               .emptyMap());
-    } catch (PathErrorException | MetadataArgsErrorException | IOException e) {
+    } catch (PathErrorException | IOException e) {
       e.printStackTrace();
       fail(e.getMessage());
     }
@@ -90,7 +90,7 @@ public class MManagerBasicTest {
     try {
       manager.addPathToMTree("root.laptop.d1.s1", TSDataType.valueOf("INT32"),
           TSEncoding.valueOf("RLE"), compressionType, Collections.emptyMap());
-    } catch (PathErrorException | MetadataArgsErrorException | IOException e1) {
+    } catch (PathErrorException | IOException e1) {
       e1.printStackTrace();
       fail(e1.getMessage());
     }
@@ -125,7 +125,7 @@ public class MManagerBasicTest {
     try {
       manager.addPathToMTree("root.laptop.d1.s1", TSDataType.valueOf("INT32"),
           TSEncoding.valueOf("RLE"), compressionType, Collections.emptyMap());
-    } catch (PathErrorException | MetadataArgsErrorException | IOException e1) {
+    } catch (PathErrorException | IOException e1) {
       e1.printStackTrace();
       fail(e1.getMessage());
     }
@@ -133,7 +133,7 @@ public class MManagerBasicTest {
     try {
       manager.addPathToMTree("root.laptop.d1.s0", TSDataType.valueOf("INT32"),
           TSEncoding.valueOf("RLE"), compressionType, Collections.emptyMap());
-    } catch (PathErrorException | MetadataArgsErrorException | IOException e1) {
+    } catch (PathErrorException | IOException e1) {
       e1.printStackTrace();
       fail(e1.getMessage());
     }
@@ -178,7 +178,7 @@ public class MManagerBasicTest {
     try {
       manager.addPathToMTree("root.laptop.d2.s1", TSDataType.valueOf("INT32"),
           TSEncoding.valueOf("RLE"), compressionType, Collections.emptyMap());
-    } catch (PathErrorException | MetadataArgsErrorException | IOException e1) {
+    } catch (PathErrorException | IOException e1) {
       e1.printStackTrace();
       fail(e1.getMessage());
     }
@@ -186,7 +186,7 @@ public class MManagerBasicTest {
     try {
       manager.addPathToMTree("root.laptop.d2.s0", TSDataType.valueOf("INT32"),
           TSEncoding.valueOf("RLE"), compressionType, Collections.emptyMap());
-    } catch (PathErrorException | MetadataArgsErrorException | IOException e1) {
+    } catch (PathErrorException | IOException e1) {
       e1.printStackTrace();
       fail(e1.getMessage());
     }
@@ -207,7 +207,7 @@ public class MManagerBasicTest {
     try {
       manager.addPathToMTree("root.laptop.d1.s0", TSDataType.valueOf("INT32"),
           TSEncoding.valueOf("RLE"), compressionType, Collections.emptyMap());
-    } catch (PathErrorException | MetadataArgsErrorException | IOException e1) {
+    } catch (PathErrorException | IOException e1) {
       e1.printStackTrace();
       fail(e1.getMessage());
     }
@@ -215,7 +215,7 @@ public class MManagerBasicTest {
     try {
       manager.addPathToMTree("root.laptop.d1.s1", TSDataType.valueOf("INT32"),
           TSEncoding.valueOf("RLE"), compressionType, Collections.emptyMap());
-    } catch (PathErrorException | MetadataArgsErrorException | IOException e1) {
+    } catch (PathErrorException | IOException e1) {
       e1.printStackTrace();
       fail(e1.getMessage());
     }
@@ -232,7 +232,7 @@ public class MManagerBasicTest {
     try {
       manager.addPathToMTree("root.laptop.d1.s2", TSDataType.valueOf("INT32"),
           TSEncoding.valueOf("RLE"), compressionType, Collections.emptyMap());
-    } catch (PathErrorException | MetadataArgsErrorException | IOException e1) {
+    } catch (PathErrorException | IOException e1) {
       e1.printStackTrace();
       fail(e1.getMessage());
     }
@@ -248,7 +248,7 @@ public class MManagerBasicTest {
     try {
       manager.addPathToMTree("root.laptop.d1.s3", TSDataType.valueOf("INT32"),
           TSEncoding.valueOf("RLE"), compressionType, Collections.emptyMap());
-    } catch (PathErrorException | MetadataArgsErrorException | IOException e1) {
+    } catch (PathErrorException | IOException e1) {
       e1.printStackTrace();
       fail(e1.getMessage());
     }
@@ -309,7 +309,7 @@ public class MManagerBasicTest {
       list.add("root.laptop.d2");
       assertEquals(list, manager.getAllFileNamesByPath("root.laptop"));
       assertEquals(list, manager.getAllFileNamesByPath("root"));
-    } catch (PathErrorException | IOException | MetadataArgsErrorException e) {
+    } catch (PathErrorException | IOException e) {
       e.printStackTrace();
       fail(e.getMessage());
     }
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java b/iotdb/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java
index 3e45d4b..4f838e1 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/metadata/MTreeTest.java
@@ -22,6 +22,9 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import com.alibaba.fastjson.JSON;
+import com.alibaba.fastjson.JSONObject;
+import com.alibaba.fastjson.serializer.SerializerFeature;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -128,6 +131,46 @@ public class MTreeTest {
   }
 
   @Test
+  public void testCombineMetadataInStrings() {
+    MTree root = new MTree("root");
+    MTree root1 = new MTree("root");
+    MTree root2 = new MTree("root");
+    MTree root3 = new MTree("root");
+    try {
+      root.setStorageGroup("root.a.d0");
+      root.addTimeseriesPath("root.a.d0.s0", "INT32", "RLE");
+      root.addTimeseriesPath("root.a.d0.s1", "INT32", "RLE");
+
+      root.setStorageGroup("root.a.d1");
+      root.addTimeseriesPath("root.a.d1.s0", "INT32", "RLE");
+      root.addTimeseriesPath("root.a.d1.s1", "INT32", "RLE");
+
+      root.setStorageGroup("root.a.b.d0");
+      root.addTimeseriesPath("root.a.b.d0.s0", "INT32", "RLE");
+
+      root1.setStorageGroup("root.a.d0");
+      root1.addTimeseriesPath("root.a.d0.s0", "INT32", "RLE");
+      root1.addTimeseriesPath("root.a.d0.s1", "INT32", "RLE");
+
+      root2.setStorageGroup("root.a.d1");
+      root2.addTimeseriesPath("root.a.d1.s0", "INT32", "RLE");
+      root2.addTimeseriesPath("root.a.d1.s1", "INT32", "RLE");
+
+      root3.setStorageGroup("root.a.b.d0");
+      root3.addTimeseriesPath("root.a.b.d0.s0", "INT32", "RLE");
+
+      String[] metadatas = new String[3];
+      metadatas[0] = root1.toString();
+      metadatas[1] = root2.toString();
+      metadatas[2] = root3.toString();
+      assertEquals(MTree.combineMetadataInStrings(metadatas), root.toString());
+    } catch (PathErrorException e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
   public void testSetStorageGroup() {
     // set storage group first
     MTree root = new MTree("root");