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/30 02:54:16 UTC

[incubator-iotdb] branch master updated: [IOTDB-605] Add more levels of index in TsFileMetadata (#1100)

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 85ef983  [IOTDB-605] Add more levels of index in TsFileMetadata (#1100)
85ef983 is described below

commit 85ef983c7ce98645689649b2f54228309e7cc189
Author: Zesong Sun <sz...@mails.tsinghua.edu.cn>
AuthorDate: Thu Apr 30 10:54:09 2020 +0800

    [IOTDB-605] Add more levels of index in TsFileMetadata (#1100)
    
    * TsFileMetadataIndex pro
---
 .../apache/iotdb/tsfile/TsFileSequenceRead.java    |  15 +-
 .../iotdb/flink/tsfile/util/TSFileConfigUtil.java  |   1 +
 .../util/TSFileConfigUtilCompletenessTest.java     | 102 +++----
 .../resources/conf/iotdb-engine.properties         |   3 +
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |  37 ++-
 .../db/engine/cache/TimeSeriesMetadataCache.java   |  21 +-
 .../iotdb/db/engine/cache/TsFileMetaDataCache.java |  23 +-
 .../apache/iotdb/db/tools/TsFileSketchTool.java    |  66 ++--
 .../org/apache/iotdb/db/utils/FileLoaderUtils.java |  38 ++-
 .../java/org/apache/iotdb/db/utils/MergeUtils.java |  14 +-
 .../writelog/recover/TsFileRecoverPerformer.java   |  28 +-
 .../apache/iotdb/spark/tsfile/DefaultSource.scala  |   2 +-
 .../iotdb/spark/tsfile/NarrowConverter.scala       |   2 +-
 .../apache/iotdb/spark/tsfile/WideConverter.scala  |   7 +-
 tsfile/format-changelist.md                        |   1 +
 .../iotdb/tsfile/common/conf/TSFileConfig.java     |  12 +
 .../iotdb/tsfile/common/conf/TSFileDescriptor.java |   2 +
 .../file/metadata/MetadataIndexConstructor.java    | 151 ++++++++++
 .../tsfile/file/metadata/MetadataIndexEntry.java   |  91 ++++++
 .../tsfile/file/metadata/MetadataIndexNode.java    | 125 ++++++++
 .../tsfile/file/metadata/TimeseriesMetadata.java   |  14 +-
 .../iotdb/tsfile/file/metadata/TsFileMetadata.java |  50 +--
 .../file/metadata/enums/MetadataIndexNodeType.java |  86 ++++++
 .../iotdb/tsfile/read/TsFileSequenceReader.java    | 334 ++++++++++++++-------
 .../tsfile/read/controller/IMetadataQuerier.java   |   2 +-
 .../read/controller/MetadataQuerierByFileImpl.java |  27 +-
 .../tsfile/read/query/executor/TsFileExecutor.java |   3 -
 .../iotdb/tsfile/utils/ReadWriteIOUtils.java       |   7 +
 .../write/writer/RestorableTsFileIOWriter.java     |   9 +-
 .../iotdb/tsfile/write/writer/TsFileIOWriter.java  |  67 ++---
 .../file/metadata/MetadataIndexNodeTest.java       |  46 +++
 .../tsfile/file/metadata/utils/TestHelper.java     |  20 +-
 .../iotdb/tsfile/file/metadata/utils/Utils.java    |  14 +-
 .../iotdb/tsfile/read/reader/ReaderTest.java       |   2 +
 .../iotdb/tsfile/write/TsFileIOWriterTest.java     |   5 +-
 35 files changed, 1008 insertions(+), 419 deletions(-)

diff --git a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java
index 0ec5fb4..8846f6b 100644
--- a/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java
+++ b/example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileSequenceRead.java
@@ -30,14 +30,12 @@ import org.apache.iotdb.tsfile.file.footer.ChunkGroupFooter;
 import org.apache.iotdb.tsfile.file.header.ChunkHeader;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
-import org.apache.iotdb.tsfile.file.metadata.TsFileMetadata;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.reader.page.PageReader;
-import org.apache.iotdb.tsfile.utils.Pair;
 
 public class TsFileSequenceRead {
 
@@ -52,12 +50,12 @@ public class TsFileSequenceRead {
     System.out.println("file magic tail: " + reader.readTailMagic());
     System.out.println("Level 1 metadata position: " + reader.getFileMetadataPos());
     System.out.println("Level 1 metadata size: " + reader.getFileMetadataSize());
-    TsFileMetadata metaData = reader.readFileMetadata();
     // Sequential reading of one ChunkGroup now follows this order:
     // first SeriesChunks (headers and data) in one ChunkGroup, then the CHUNK_GROUP_FOOTER
     // Because we do not know how many chunks a ChunkGroup may have, we should read one byte (the marker) ahead and
     // judge accordingly.
-    reader.position(TSFileConfig.MAGIC_STRING.getBytes().length + TSFileConfig.VERSION_NUMBER.getBytes().length);
+    reader.position((long) TSFileConfig.MAGIC_STRING.getBytes().length + TSFileConfig.VERSION_NUMBER
+        .getBytes().length);
     System.out.println("[Chunk Group]");
     System.out.println("position: " + reader.position());
     byte marker;
@@ -107,13 +105,10 @@ public class TsFileSequenceRead {
       }
     }
     System.out.println("[Metadata]");
-    Map<String, Pair<Long, Integer>> deviceOffsetsMap = metaData.getDeviceMetadataIndex();
-    for (Map.Entry<String, Pair<Long, Integer>>  entry: deviceOffsetsMap.entrySet()) {
-      String deviceId = entry.getKey();
-      Map<String, List<ChunkMetadata>> seriesMetaData =
-          reader.readChunkMetadataInDevice(deviceId);
+    for (String device : reader.getAllDevices()) {
+      Map<String, List<ChunkMetadata>> seriesMetaData = reader.readChunkMetadataInDevice(device);
       System.out.println(String
-          .format("\t[Device]Device %s, Number of Measurements %d", deviceId, seriesMetaData.size()));
+          .format("\t[Device]Device %s, Number of Measurements %d", device, seriesMetaData.size()));
       for (Map.Entry<String, List<ChunkMetadata>> serie : seriesMetaData.entrySet()) {
         System.out.println("\t\tMeasurement:" + serie.getKey());
         for (ChunkMetadata chunkMetadata : serie.getValue()) {
diff --git a/flink-tsfile-connector/src/main/java/org/apache/iotdb/flink/tsfile/util/TSFileConfigUtil.java b/flink-tsfile-connector/src/main/java/org/apache/iotdb/flink/tsfile/util/TSFileConfigUtil.java
index 41596ea..d8b72f1 100644
--- a/flink-tsfile-connector/src/main/java/org/apache/iotdb/flink/tsfile/util/TSFileConfigUtil.java
+++ b/flink-tsfile-connector/src/main/java/org/apache/iotdb/flink/tsfile/util/TSFileConfigUtil.java
@@ -50,6 +50,7 @@ public class TSFileConfigUtil {
 		globalConfig.setKerberosKeytabFilePath(config.getKerberosKeytabFilePath());
 		globalConfig.setKerberosPrincipal(config.getKerberosPrincipal());
 		globalConfig.setMaxNumberOfPointsInPage(config.getMaxNumberOfPointsInPage());
+		globalConfig.setMaxDegreeOfIndexNode(config.getMaxDegreeOfIndexNode());
 		globalConfig.setMaxStringLength(config.getMaxStringLength());
 		globalConfig.setPageCheckSizeThreshold(config.getPageCheckSizeThreshold());
 		globalConfig.setPageSizeInByte(config.getPageSizeInByte());
diff --git a/flink-tsfile-connector/src/test/java/org/apache/iotdb/flink/util/TSFileConfigUtilCompletenessTest.java b/flink-tsfile-connector/src/test/java/org/apache/iotdb/flink/util/TSFileConfigUtilCompletenessTest.java
index 2af10b6..eb914ff 100644
--- a/flink-tsfile-connector/src/test/java/org/apache/iotdb/flink/util/TSFileConfigUtilCompletenessTest.java
+++ b/flink-tsfile-connector/src/test/java/org/apache/iotdb/flink/util/TSFileConfigUtilCompletenessTest.java
@@ -19,66 +19,66 @@
 
 package org.apache.iotdb.flink.util;
 
-import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
-import org.junit.Test;
+import static org.junit.Assert.assertTrue;
 
 import java.lang.reflect.Method;
 import java.util.Arrays;
 import java.util.Set;
 import java.util.stream.Collectors;
-
-import static org.junit.Assert.assertTrue;
+import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
+import org.junit.Test;
 
 /**
  * This test is used to help maintain the {@link org.apache.iotdb.flink.tsfile.util.TSFileConfigUtil}.
  */
 public class TSFileConfigUtilCompletenessTest {
 
-	@Test
-	public void testTSFileConfigUtilCompleteness() {
-		String[] addedSetters = {
-			"setBatchSize",
-			"setBloomFilterErrorRate",
-			"setCompressor",
-			"setCoreSitePath",
-			"setDeltaBlockSize",
-			"setDfsClientFailoverProxyProvider",
-			"setDfsHaAutomaticFailoverEnabled",
-			"setDfsHaNamenodes",
-			"setDfsNameServices",
-			"setDftSatisfyRate",
-			"setEndian",
-			"setFloatPrecision",
-			"setFreqType",
-			"setGroupSizeInByte",
-			"setHdfsIp",
-			"setHdfsPort",
-			"setHdfsSitePath",
-			"setKerberosKeytabFilePath",
-			"setKerberosPrincipal",
-			"setMaxNumberOfPointsInPage",
-			"setMaxStringLength",
-			"setPageCheckSizeThreshold",
-			"setPageSizeInByte",
-			"setPlaMaxError",
-			"setRleBitWidth",
-			"setSdtMaxError",
-			"setTimeEncoder",
-			"setTimeSeriesDataType",
-			"setTSFileStorageFs",
-			"setUseKerberos",
-			"setValueEncoder"
-		};
-		Set<String> newSetters = Arrays.stream(TSFileConfig.class.getMethods())
-			.map(Method::getName)
-			.filter(s -> s.startsWith("set"))
-			.filter(s -> !Arrays.asList(addedSetters).contains(s))
-			.collect(Collectors.toSet());
-		assertTrue(
-			String.format(
-				"New setters in TSFileConfig are detected, please add them to " +
-				"org.apache.iotdb.flink.tsfile.util.TSFileConfigUtil. The setters need to be added: %s",
-				newSetters),
-			newSetters.isEmpty());
-	}
+  @Test
+  public void testTSFileConfigUtilCompleteness() {
+    String[] addedSetters = {
+        "setBatchSize",
+        "setBloomFilterErrorRate",
+        "setCompressor",
+        "setCoreSitePath",
+        "setDeltaBlockSize",
+        "setDfsClientFailoverProxyProvider",
+        "setDfsHaAutomaticFailoverEnabled",
+        "setDfsHaNamenodes",
+        "setDfsNameServices",
+        "setDftSatisfyRate",
+        "setEndian",
+        "setFloatPrecision",
+        "setFreqType",
+        "setGroupSizeInByte",
+        "setHdfsIp",
+        "setHdfsPort",
+        "setHdfsSitePath",
+        "setKerberosKeytabFilePath",
+        "setKerberosPrincipal",
+        "setMaxNumberOfPointsInPage",
+        "setMaxDegreeOfIndexNode",
+        "setMaxStringLength",
+        "setPageCheckSizeThreshold",
+        "setPageSizeInByte",
+        "setPlaMaxError",
+        "setRleBitWidth",
+        "setSdtMaxError",
+        "setTimeEncoder",
+        "setTimeSeriesDataType",
+        "setTSFileStorageFs",
+        "setUseKerberos",
+        "setValueEncoder"
+    };
+    Set<String> newSetters = Arrays.stream(TSFileConfig.class.getMethods())
+        .map(Method::getName)
+        .filter(s -> s.startsWith("set"))
+        .filter(s -> !Arrays.asList(addedSetters).contains(s))
+        .collect(Collectors.toSet());
+    assertTrue(
+        String.format(
+            "New setters in TSFileConfig are detected, please add them to " +
+                "org.apache.iotdb.flink.tsfile.util.TSFileConfigUtil. The setters need to be added: %s",
+            newSetters),
+        newSetters.isEmpty());
+  }
 }
diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties
index 986cec9..4b30f42 100644
--- a/server/src/assembly/resources/conf/iotdb-engine.properties
+++ b/server/src/assembly/resources/conf/iotdb-engine.properties
@@ -454,6 +454,9 @@ value_encoder=PLAIN
 # Data compression method, supports UNCOMPRESSED or SNAPPY. Default value is SNAPPY
 compressor=SNAPPY
 
+# Maximum degree of a metadataIndex node, default value is 1024
+max_degree_of_index_node=1024
+
 
 ####################
 ### MQTT Broker Configuration
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
index 832b5be..0e0a1ad 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
@@ -18,6 +18,14 @@
  */
 package org.apache.iotdb.db.conf;
 
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.time.ZoneId;
+import java.util.Properties;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.DefaultParser;
@@ -32,11 +40,6 @@ import org.apache.iotdb.tsfile.fileSystem.FSType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.*;
-import java.net.URL;
-import java.time.ZoneId;
-import java.util.Properties;
-
 public class IoTDBDescriptor {
 
   private static final Logger logger = LoggerFactory.getLogger(IoTDBDescriptor.class);
@@ -57,7 +60,9 @@ public class IoTDBDescriptor {
 
   public void replaceProps(String[] params) {
     Options options = new Options();
-    Option rpcPort = new Option("rpc_port", "rpc_port", true, "The jdbc service listens on the port");
+    final String RPC_PORT = "rpc_port";
+    Option rpcPort = new Option(RPC_PORT, RPC_PORT, true,
+        "The jdbc service listens on the port");
     rpcPort.setRequired(false);
     options.addOption(rpcPort);
 
@@ -66,8 +71,8 @@ public class IoTDBDescriptor {
       logger.error("replaces properties failed, use default conf params");
       return;
     } else {
-      if (commandLine.hasOption("rpc_port")) {
-        conf.setRpcPort(Integer.parseInt(commandLine.getOptionValue("rpc_port")));
+      if (commandLine.hasOption(RPC_PORT)) {
+        conf.setRpcPort(Integer.parseInt(commandLine.getOptionValue(RPC_PORT)));
         logger.debug("replace rpc port with={}", conf.getRpcPort());
       }
     }
@@ -376,7 +381,7 @@ public class IoTDBDescriptor {
               String.valueOf(conf.getDefaultFillInterval()))));
 
       conf.setTagAttributeTotalSize(
-           Integer.parseInt(properties.getProperty("tag_attribute_total_size",
+          Integer.parseInt(properties.getProperty("tag_attribute_total_size",
               String.valueOf(conf.getTagAttributeTotalSize())))
       );
 
@@ -388,15 +393,18 @@ public class IoTDBDescriptor {
         conf.setMqttPort(Integer.parseInt(properties.getProperty(IoTDBConstant.MQTT_PORT_NAME)));
       }
       if (properties.getProperty(IoTDBConstant.MQTT_HANDLER_POOL_SIZE_NAME) != null) {
-        conf.setMqttHandlerPoolSize(Integer.parseInt(properties.getProperty(IoTDBConstant.MQTT_HANDLER_POOL_SIZE_NAME)));
+        conf.setMqttHandlerPoolSize(
+            Integer.parseInt(properties.getProperty(IoTDBConstant.MQTT_HANDLER_POOL_SIZE_NAME)));
       }
       if (properties.getProperty(IoTDBConstant.MQTT_PAYLOAD_FORMATTER_NAME) != null) {
-        conf.setMqttPayloadFormatter(properties.getProperty(IoTDBConstant.MQTT_PAYLOAD_FORMATTER_NAME));
+        conf.setMqttPayloadFormatter(
+            properties.getProperty(IoTDBConstant.MQTT_PAYLOAD_FORMATTER_NAME));
       }
       if (properties.getProperty(IoTDBConstant.ENABLE_MQTT) != null) {
-        conf.setEnableMQTTService(Boolean.parseBoolean(properties.getProperty(IoTDBConstant.ENABLE_MQTT)));
+        conf.setEnableMQTTService(
+            Boolean.parseBoolean(properties.getProperty(IoTDBConstant.ENABLE_MQTT)));
       }
-      
+
       // At the same time, set TSFileConfig
       TSFileDescriptor.getInstance().getConfig()
           .setTSFileStorageFs(FSType.valueOf(
@@ -525,6 +533,9 @@ public class IoTDBDescriptor {
     TSFileDescriptor.getInstance().getConfig().setCompressor(properties
         .getProperty("compressor",
             TSFileDescriptor.getInstance().getConfig().getCompressor().toString()));
+    TSFileDescriptor.getInstance().getConfig().setMaxDegreeOfIndexNode(Integer.parseInt(properties
+        .getProperty("max_degree_of_index_node", Integer
+            .toString(TSFileDescriptor.getInstance().getConfig().getMaxDegreeOfIndexNode()))));
   }
 
   public void loadHotModifiedProps() throws QueryProcessException {
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java b/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java
index 4791976..e586cde 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/cache/TimeSeriesMetadataCache.java
@@ -26,11 +26,11 @@ import org.apache.iotdb.db.query.control.FileReaderManager;
 import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
 import org.apache.iotdb.tsfile.file.metadata.TsFileMetadata;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.utils.BloomFilter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import java.io.IOException;
-import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
@@ -93,7 +93,7 @@ public class TimeSeriesMetadataCache {
         return null;
       }
       TsFileSequenceReader reader = FileReaderManager.getInstance().get(key.filePath, true);
-      return reader.readDeviceMetadata(key.device).get(key.measurement);
+      return reader.readTimeseriesMetadata(new Path(key.device, key.measurement));
     }
 
     cacheRequestNum.incrementAndGet();
@@ -125,20 +125,9 @@ public class TimeSeriesMetadataCache {
         return null;
       }
       TsFileSequenceReader reader = FileReaderManager.getInstance().get(key.filePath, true);
-      Map<String, TimeseriesMetadata> timeSeriesMetadataMap = reader.readDeviceMetadata(key.device);
-      TimeseriesMetadata res = timeSeriesMetadataMap.get(key.measurement);
-      lruCache.put(key, res);
-
-      if (!allSensors.isEmpty()) {
-        // put TimeSeriesMetadata of all sensors used in this query into cache
-        allSensors.forEach(sensor -> {
-          if (timeSeriesMetadataMap.containsKey(sensor)) {
-            lruCache.put(new TimeSeriesMetadataCacheKey(key.filePath, key.device, sensor),
-                timeSeriesMetadataMap.get(sensor));
-          }
-        });
-      }
-      return res;
+      TimeseriesMetadata timeseriesMetadata = reader.readTimeseriesMetadata(new Path(key.device, key.measurement));
+      lruCache.put(key, timeseriesMetadata);
+      return timeseriesMetadata;
     } catch (IOException e) {
       logger.error("something wrong happened while reading {}", key.filePath);
       throw e;
diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cache/TsFileMetaDataCache.java b/server/src/main/java/org/apache/iotdb/db/engine/cache/TsFileMetaDataCache.java
index 36fea80..ec59e95 100644
--- a/server/src/main/java/org/apache/iotdb/db/engine/cache/TsFileMetaDataCache.java
+++ b/server/src/main/java/org/apache/iotdb/db/engine/cache/TsFileMetaDataCache.java
@@ -18,6 +18,8 @@
  */
 package org.apache.iotdb.db.engine.cache;
 
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.iotdb.db.conf.IoTDBConfig;
@@ -29,9 +31,6 @@ import org.apache.iotdb.tsfile.file.metadata.TsFileMetadata;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicLong;
-
 /**
  * This class is used to cache <code>TsFileMetaData</code> of tsfile in IoTDB.
  */
@@ -53,26 +52,26 @@ public class TsFileMetaDataCache {
   private AtomicLong cacheRequestNum = new AtomicLong();
 
   /**
-   * estimated size of a deviceMetaDataMap entry in TsFileMetaData.
+   * estimated size of metadataIndex entry in TsFileMetaData.
    */
-  private long deviceIndexMapEntrySize = 0;
+  private long metadataIndexEntrySize = 0;
 
   private TsFileMetaDataCache() {
     logger.info("TsFileMetaDataCache size = " + MEMORY_THRESHOLD_IN_B);
     cache = new LRULinkedHashMap<String, TsFileMetadata>(MEMORY_THRESHOLD_IN_B, true) {
       @Override
       protected long calEntrySize(String key, TsFileMetadata value) {
-        if (deviceIndexMapEntrySize == 0 && value.getDeviceMetadataIndex() != null
-            && value.getDeviceMetadataIndex().size() > 0) {
-          deviceIndexMapEntrySize = RamUsageEstimator
-              .sizeOf(value.getDeviceMetadataIndex().entrySet().iterator().next());
+        if (metadataIndexEntrySize == 0 && value.getMetadataIndex() != null
+            && !value.getMetadataIndex().getChildren().isEmpty()) {
+          metadataIndexEntrySize = RamUsageEstimator
+              .sizeOf(value.getMetadataIndex().getChildren().iterator().next());
         }
         // totalChunkNum, invalidChunkNum
         long valueSize = 4 + 4L;
 
-        // deviceMetadataIndex
-        if (value.getDeviceMetadataIndex() != null) {
-          valueSize += value.getDeviceMetadataIndex().size() * deviceIndexMapEntrySize;
+        // metadataIndex
+        if (value.getMetadataIndex() != null) {
+          valueSize += value.getMetadataIndex().getChildren().size() * metadataIndexEntrySize;
         }
 
         // versionInfo
diff --git a/server/src/main/java/org/apache/iotdb/db/tools/TsFileSketchTool.java b/server/src/main/java/org/apache/iotdb/db/tools/TsFileSketchTool.java
index 68c3037..ba5651a 100644
--- a/server/src/main/java/org/apache/iotdb/db/tools/TsFileSketchTool.java
+++ b/server/src/main/java/org/apache/iotdb/db/tools/TsFileSketchTool.java
@@ -19,24 +19,22 @@
 
 package org.apache.iotdb.db.tools;
 
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.file.footer.ChunkGroupFooter;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.MetadataIndexEntry;
 import org.apache.iotdb.tsfile.file.metadata.TsFileMetadata;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 import org.apache.iotdb.tsfile.read.common.Chunk;
 import org.apache.iotdb.tsfile.utils.BloomFilter;
-import org.apache.iotdb.tsfile.utils.Pair;
-
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.stream.Collectors;
 
 public class TsFileSketchTool {
 
@@ -61,17 +59,13 @@ public class TsFileSketchTool {
       // get metadata information
       TsFileSequenceReader reader = new TsFileSequenceReader(filename);
       TsFileMetadata tsFileMetaData = reader.readFileMetadata();
-      List<String> tsDeviceSortedList = tsFileMetaData.getDeviceMetadataIndex()
-          .keySet()
-          .stream()
-          .sorted().collect(Collectors.toList());
+      List<String> devices = reader.getAllDevices();
       Map<String, Map<String, List<ChunkMetadata>>> tsDeviceSeriesMetadataMap = new LinkedHashMap<>();
-      for (String deviceId : tsDeviceSortedList) {
-        Map<String, List<ChunkMetadata>> seriesMetadataMap =
-            reader.readChunkMetadataInDevice(deviceId);
+      for (String deviceId : devices) {
+        Map<String, List<ChunkMetadata>> seriesMetadataMap = reader
+            .readChunkMetadataInDevice(deviceId);
         tsDeviceSeriesMetadataMap.put(deviceId, seriesMetadataMap);
       }
-      
 
       // begin print
       StringBuilder str1 = new StringBuilder();
@@ -88,9 +82,9 @@ public class TsFileSketchTool {
               + "|\t[version number] "
               + reader.readVersionNumber());
       // device begins
-      for (
-          Entry<String, Map<String, List<ChunkMetadata>>> entry : tsDeviceSeriesMetadataMap.entrySet()) {
-        printlnBoth(pw, str1.toString() + "\t[Chunks] of "+ entry.getKey() + 
+      for (Entry<String, Map<String, List<ChunkMetadata>>> entry : tsDeviceSeriesMetadataMap
+          .entrySet()) {
+        printlnBoth(pw, str1.toString() + "\t[Chunks] of " + entry.getKey() +
             ", num of Chunks:" + entry.getValue().size());
         // chunk begins
         long chunkEndPos = 0;
@@ -108,8 +102,8 @@ public class TsFileSketchTool {
             printlnBoth(pw,
                 String.format("%20s", "") + "|\t\t" + chunk.getHeader().getNumOfPages() + " pages");
             chunkEndPos =
-                chunkMetaData.getOffsetOfChunkHeader() + chunk.getHeader().getSerializedSize() + chunk
-                    .getHeader().getDataSize();
+                chunkMetaData.getOffsetOfChunkHeader() + chunk.getHeader().getSerializedSize()
+                    + chunk.getHeader().getDataSize();
           }
         }
         // chunkGroupFooter begins
@@ -127,29 +121,23 @@ public class TsFileSketchTool {
       }
 
       // metadata begins
-      if (tsDeviceSortedList.isEmpty()) {
-        printlnBoth(pw, String.format("%20s",  reader.getFileMetadataPos() - 1)
-                + "|\t[marker] 2");
+      if (tsFileMetaData.getMetadataIndex().getChildren().isEmpty()) {
+        printlnBoth(pw, String.format("%20s", reader.getFileMetadataPos() - 1) + "|\t[marker] 2");
       } else {
         printlnBoth(pw,
-            String.format("%20s", (tsFileMetaData.getDeviceMetadataIndex()
-                .get(tsDeviceSortedList.get(0))).left - 1)
-                + "|\t[marker] 2");
+            String.format("%20s", reader.readFileMetadata().getMetaOffset() + "|\t[marker] 2"));
       }
-      for (Entry<String, Pair<Long,Integer>> entry 
-          : tsFileMetaData.getDeviceMetadataIndex().entrySet()) {
-        printlnBoth(pw,
-            String.format("%20s", entry.getValue().left)
-                + "|\t[DeviceMetadata] of " + entry.getKey());
+      for (MetadataIndexEntry metadataIndex : tsFileMetaData.getMetadataIndex().getChildren()) {
+        printlnBoth(pw, String.format("%20s", metadataIndex.getOffset())
+            + "|\t[MetadataIndex] of " + metadataIndex.getName());
       }
 
       printlnBoth(pw, String.format("%20s", reader.getFileMetadataPos()) + "|\t[TsFileMetaData]");
+      printlnBoth(pw, String.format("%20s", "") + "|\t\t[num of devices] " + tsFileMetaData
+          .getMetadataIndex().getChildren().size());
       printlnBoth(pw,
-          String.format("%20s", "") + "|\t\t[num of devices] " + tsFileMetaData
-              .getDeviceMetadataIndex().size());
-      printlnBoth(pw,
-          String.format("%20s", "") + "|\t\t" + tsFileMetaData.getDeviceMetadataIndex().size()
-              + " key&TsDeviceMetadataIndex");
+          String.format("%20s", "") + "|\t\t" + tsFileMetaData.getMetadataIndex().getChildren()
+              .size() + " key&TsMetadataIndex");
       printlnBoth(pw,
           String.format("%20s", "") + "|\t\t[totalChunkNum] " + tsFileMetaData.getTotalChunkNum());
       printlnBoth(pw,
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java
index 8f4661a..35a07b9 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java
@@ -18,14 +18,15 @@
  */
 package org.apache.iotdb.db.utils;
 
-import org.apache.iotdb.db.engine.cache.ChunkMetadataCache;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
 import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache;
 import org.apache.iotdb.db.engine.modification.Modification;
-import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.control.FileReaderManager;
-import org.apache.iotdb.db.query.reader.chunk.DiskChunkLoader;
 import org.apache.iotdb.db.query.reader.chunk.MemChunkLoader;
 import org.apache.iotdb.db.query.reader.chunk.MemChunkReader;
 import org.apache.iotdb.db.query.reader.chunk.metadata.DiskChunkMetadataLoader;
@@ -42,12 +43,6 @@ import org.apache.iotdb.tsfile.read.reader.IChunkReader;
 import org.apache.iotdb.tsfile.read.reader.IPageReader;
 import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 public class FileLoaderUtils {
 
   private FileLoaderUtils() {
@@ -59,8 +54,7 @@ public class FileLoaderUtils {
       // .resource file does not exist, read file metadata and recover tsfile resource
       try (TsFileSequenceReader reader = new TsFileSequenceReader(
           tsFileResource.getFile().getAbsolutePath())) {
-        TsFileMetadata metaData = reader.readFileMetadata();
-        updateTsFileResource(metaData, reader, tsFileResource);
+        updateTsFileResource(reader, tsFileResource);
       }
       // write .resource file
       tsFileResource.serialize();
@@ -70,21 +64,20 @@ public class FileLoaderUtils {
     tsFileResource.setClosed(true);
   }
 
-  public static void updateTsFileResource(TsFileMetadata metaData, TsFileSequenceReader reader,
+  public static void updateTsFileResource(TsFileSequenceReader reader,
       TsFileResource tsFileResource) throws IOException {
-    for (String device : metaData.getDeviceMetadataIndex().keySet()) {
-      Map<String, TimeseriesMetadata> chunkMetadataListInOneDevice = reader
-          .readDeviceMetadata(device);
-      for (TimeseriesMetadata timeseriesMetaData : chunkMetadataListInOneDevice.values()) {
-        tsFileResource.updateStartTime(device, timeseriesMetaData.getStatistics().getStartTime());
-        tsFileResource.updateEndTime(device, timeseriesMetaData.getStatistics().getEndTime());
+    for (Entry<String, List<TimeseriesMetadata>> entry : reader.getAllTimeseriesMetadata()
+        .entrySet()) {
+      for (TimeseriesMetadata timeseriesMetaData : entry.getValue()) {
+        tsFileResource
+            .updateStartTime(entry.getKey(), timeseriesMetaData.getStatistics().getStartTime());
+        tsFileResource
+            .updateEndTime(entry.getKey(), timeseriesMetaData.getStatistics().getEndTime());
       }
     }
   }
 
-
   /**
-   *
    * @param resource TsFile
    * @param seriesPath Timeseries path
    * @param allSensors measurements queried at the same time of this device
@@ -128,6 +121,7 @@ public class FileLoaderUtils {
 
   /**
    * load all chunk metadata of one time series in one file.
+   *
    * @param timeSeriesMetadata the corresponding TimeSeriesMetadata in that file.
    */
   public static List<ChunkMetadata> loadChunkMetadataList(TimeseriesMetadata timeSeriesMetadata)
@@ -138,6 +132,7 @@ public class FileLoaderUtils {
 
   /**
    * load all page readers in one chunk that satisfying the timeFilter
+   *
    * @param chunkMetaData the corresponding chunk metadata
    * @param timeFilter it should be a TimeFilter instead of a ValueFilter
    */
@@ -159,7 +154,8 @@ public class FileLoaderUtils {
     return chunkReader.loadPageReaderList();
   }
 
-  public static List<ChunkMetadata> getChunkMetadataList(Path path, String filePath) throws IOException {
+  public static List<ChunkMetadata> getChunkMetadataList(Path path, String filePath)
+      throws IOException {
     TsFileSequenceReader tsFileReader = FileReaderManager.getInstance().get(filePath, true);
     return tsFileReader.getChunkMetadataList(path);
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/MergeUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/MergeUtils.java
index d1ddbab..a5a0f0a 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/MergeUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/MergeUtils.java
@@ -29,14 +29,12 @@ import org.apache.iotdb.db.engine.merge.manage.MergeResource;
 import org.apache.iotdb.db.engine.modification.Modification;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
-import org.apache.iotdb.tsfile.file.metadata.TsFileMetadata;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 import org.apache.iotdb.tsfile.read.common.BatchData;
 import org.apache.iotdb.tsfile.read.common.Chunk;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
-import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.chunk.IChunkWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -48,7 +46,7 @@ public class MergeUtils {
   private MergeUtils() {
     // util class
   }
-  
+
   public static void writeTVPair(TimeValuePair timeValuePair, IChunkWriter chunkWriter) {
     switch (chunkWriter.getDataType()) {
       case TEXT:
@@ -145,14 +143,8 @@ public class MergeUtils {
     return new long[] {totalChunkNum, maxChunkNum};
   }
 
-  public static long getFileMetaSize(TsFileResource seqFile, TsFileSequenceReader sequenceReader) throws IOException {
-    long minPos = Long.MAX_VALUE;
-    TsFileMetadata fileMetaData = sequenceReader.readFileMetadata();
-    for (Pair<Long, Integer> deviceMetaData : fileMetaData.getDeviceMetadataIndex().values()) {
-      long timeseriesMetaDataEndOffset = deviceMetaData.left + deviceMetaData.right;
-      minPos = timeseriesMetaDataEndOffset < minPos ? timeseriesMetaDataEndOffset : minPos;
-    }
-    return seqFile.getFileSize() - minPos;
+  public static long getFileMetaSize(TsFileResource seqFile, TsFileSequenceReader sequenceReader) {
+    return seqFile.getFileSize() - sequenceReader.getFileMetadataPos();
   }
 
   /**
diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java b/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java
index 71726cb..56edfc3 100644
--- a/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java
+++ b/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java
@@ -26,6 +26,7 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.concurrent.ExecutionException;
 import org.apache.iotdb.db.conf.IoTDBConstant;
 import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory;
@@ -40,10 +41,8 @@ import org.apache.iotdb.db.writelog.manager.MultiFileLogNodeManager;
 import org.apache.iotdb.tsfile.exception.NotCompatibleTsFileException;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
 import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
-import org.apache.iotdb.tsfile.file.metadata.TsFileMetadata;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
-import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.write.writer.RestorableTsFileIOWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -76,8 +75,9 @@ public class TsFileRecoverPerformer {
   /**
    * 1. recover the TsFile by RestorableTsFileIOWriter and truncate the file to remaining corrected
    * data 2. redo the WALs to recover unpersisted data 3. flush and close the file 4. clean WALs
-   * @return a RestorableTsFileIOWriter if the file is not closed before crush, so this writer
-   * can be used to continue writing
+   *
+   * @return a RestorableTsFileIOWriter if the file is not closed before crush, so this writer can
+   * be used to continue writing
    */
   public RestorableTsFileIOWriter recover() throws StorageGroupProcessorException {
 
@@ -108,12 +108,12 @@ public class TsFileRecoverPerformer {
           // .resource file does not exist, read file metadata and recover tsfile resource
           try (TsFileSequenceReader reader = new TsFileSequenceReader(
               resource.getFile().getAbsolutePath())) {
-            TsFileMetadata metaData = reader.readFileMetadata();
-            FileLoaderUtils.updateTsFileResource(metaData, reader, resource);
+            FileLoaderUtils.updateTsFileResource(reader, resource);
           }
           // write .resource file
           long fileVersion =
-              Long.parseLong(resource.getFile().getName().split(IoTDBConstant.TSFILE_NAME_SEPARATOR)[1]);
+              Long.parseLong(
+                  resource.getFile().getName().split(IoTDBConstant.TSFILE_NAME_SEPARATOR)[1]);
           resource.setHistoricalVersions(Collections.singleton(fileVersion));
           resource.serialize();
         }
@@ -157,14 +157,12 @@ public class TsFileRecoverPerformer {
   private void recoverResourceFromReader() throws IOException {
     try (TsFileSequenceReader reader =
         new TsFileSequenceReader(resource.getFile().getAbsolutePath(), false)) {
-      TsFileMetadata fileMetadata = reader.readFileMetadata();
-      
-      Map<String, Pair<Long, Integer>> deviceMetaDataMap = fileMetadata.getDeviceMetadataIndex();
-      for (Map.Entry<String, Pair<Long, Integer>>  entry: deviceMetaDataMap.entrySet()) {
-        String deviceId = entry.getKey();
-        for (TimeseriesMetadata timeseriesMetadata : reader.readDeviceMetadata(deviceId).values()) {
-          resource.updateStartTime(deviceId, timeseriesMetadata.getStatistics().getStartTime());
-          resource.updateStartTime(deviceId, timeseriesMetadata.getStatistics().getEndTime());
+      for (Entry<String, List<TimeseriesMetadata>> entry : reader.getAllTimeseriesMetadata()
+          .entrySet()) {
+        for (TimeseriesMetadata timeseriesMetaData : entry.getValue()) {
+          resource
+              .updateStartTime(entry.getKey(), timeseriesMetaData.getStatistics().getStartTime());
+          resource.updateEndTime(entry.getKey(), timeseriesMetaData.getStatistics().getEndTime());
         }
       }
     }
diff --git a/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/DefaultSource.scala b/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/DefaultSource.scala
index f230491..257a0f2 100755
--- a/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/DefaultSource.scala
+++ b/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/DefaultSource.scala
@@ -115,7 +115,7 @@ private[tsfile] class DefaultSource extends FileFormat with DataSourceRegister {
       }
 
       if (options.getOrElse(DefaultSource.isNarrowForm, "").equals("narrow_form")) {
-        val deviceNames = tsFileMetaData.getDeviceMetadataIndex.keySet()
+        val deviceNames = reader.getAllDevices()
         val measurementNames = reader.getAllMeasurements.keySet()
 
         // construct queryExpression based on queriedSchema and filters
diff --git a/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/NarrowConverter.scala b/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/NarrowConverter.scala
index f922dd6..9c820d0 100644
--- a/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/NarrowConverter.scala
+++ b/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/NarrowConverter.scala
@@ -166,7 +166,7 @@ object NarrowConverter extends Converter {
     * @return query expression
     */
   def toQueryExpression(schema: StructType,
-                        device_name: util.Set[String],
+                        device_name: util.List[String],
                         measurement_name: util.Set[String],
                         filters: Seq[Filter],
                         in: TsFileSequenceReader,
diff --git a/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/WideConverter.scala b/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/WideConverter.scala
index e5b7de2..c3db9a9 100755
--- a/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/WideConverter.scala
+++ b/spark-tsfile/src/main/scala/org/apache/iotdb/spark/tsfile/WideConverter.scala
@@ -62,7 +62,7 @@ object WideConverter extends Converter {
   def getSeries(tsFileMetaData: TsFileMetadata, reader: TsFileSequenceReader): util.ArrayList[Series] = {
     val series = new util.ArrayList[Series]()
 
-    val devices = tsFileMetaData.getDeviceMetadataIndex.keySet()
+    val devices = reader.getAllDevices
     val measurements = reader.getAllMeasurements
 
     devices.foreach(d => {
@@ -91,8 +91,7 @@ object WideConverter extends Converter {
     files.foreach(f => {
       val in = new HDFSInput(f.getPath, conf)
       val reader = new TsFileSequenceReader(in)
-      val tsFileMetaData = reader.readFileMetadata
-      val devices = tsFileMetaData.getDeviceMetadataIndex.keySet()
+      val devices = reader.getAllDevices
       val measurements = reader.getAllMeasurements
 
       devices.foreach(d => {
@@ -133,7 +132,7 @@ object WideConverter extends Converter {
     } else { // Remove nonexistent schema according to the current file's metadata.
       // This may happen when queried TsFiles in the same folder do not have the same schema.
 
-      val devices = tsFileMetaData.getDeviceMetadataIndex.keySet()
+      val devices = reader.getAllDevices
       val measurementIds = reader.getAllMeasurements.keySet()
       requiredSchema.foreach(f => {
         if (!QueryConstant.RESERVED_TIME.equals(f.name)) {
diff --git a/tsfile/format-changelist.md b/tsfile/format-changelist.md
index cf4260f..3f82f3b 100644
--- a/tsfile/format-changelist.md
+++ b/tsfile/format-changelist.md
@@ -30,6 +30,7 @@ Last Updated on 2019-11-28 by Jialin Qiao.
 | 855  | [IOTDB-587] New TsFile version 2                             | HTHou           | Remove ChunkGroupMetadata, store ChunkMetadata list by series, Add TimeseriesMetadata for each series |
 | 1024 | [IOTDB-585] Fix recover version bug                          | qiaojialin      | Add MetaMarker.VERSION and version behind each flushing memtable (flushAllChunkGroups) |
 | 1047 | [IOTDB-593] Add metaOffset in TsFileMetadata                 | qiaojialin      | Add metaOffset in TsFileMetadata |
+| 1100 | [IOTDB-605] Add more levels of index in TsFileMetadata       | sunzesong       | Update the structure of deviceMetadata to a tree-level indexed TsFileMetadata |
 
 # 0.8.0 (version-0) -> version-1
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java
index c5c8627..8f04ad9 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileConfig.java
@@ -83,6 +83,10 @@ public class TSFileConfig implements Serializable {
    */
   private int maxNumberOfPointsInPage = 1024 * 1024;
   /**
+   * The maximum degree of a metadataIndex node, default value is 1024
+   */
+  private int maxDegreeOfIndexNode = 1024;
+  /**
    * Data type for input timestamp, TsFile supports INT32 or INT64.
    */
   private String timeSeriesDataType = "INT64";
@@ -229,6 +233,14 @@ public class TSFileConfig implements Serializable {
     this.maxNumberOfPointsInPage = maxNumberOfPointsInPage;
   }
 
+  public int getMaxDegreeOfIndexNode() {
+    return maxDegreeOfIndexNode;
+  }
+
+  public void setMaxDegreeOfIndexNode(int maxDegreeOfIndexNode) {
+    this.maxDegreeOfIndexNode = maxDegreeOfIndexNode;
+  }
+
   public String getTimeSeriesDataType() {
     return timeSeriesDataType;
   }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileDescriptor.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileDescriptor.java
index 2ccee2f..dd79cec 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileDescriptor.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/common/conf/TSFileDescriptor.java
@@ -120,6 +120,8 @@ public class TSFileDescriptor {
       }
       conf.setMaxNumberOfPointsInPage(Integer.parseInt(
           properties.getProperty("max_number_of_points_in_page", Integer.toString(conf.getMaxNumberOfPointsInPage()))));
+      conf.setMaxDegreeOfIndexNode(Integer.parseInt(
+          properties.getProperty("max_degree_of_index_node", Integer.toString(conf.getMaxDegreeOfIndexNode()))));
       conf.setTimeSeriesDataType(properties.getProperty("time_series_data_type", conf.getTimeSeriesDataType()));
       conf.setMaxStringLength(
           Integer.parseInt(properties.getProperty("max_string_length", Integer.toString(conf.getMaxStringLength()))));
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexConstructor.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexConstructor.java
new file mode 100644
index 0000000..3d72972
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexConstructor.java
@@ -0,0 +1,151 @@
+/*
+ * 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.tsfile.file.metadata;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Queue;
+import java.util.TreeMap;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.file.metadata.enums.MetadataIndexNodeType;
+import org.apache.iotdb.tsfile.write.writer.TsFileOutput;
+
+public class MetadataIndexConstructor {
+
+  private static final int MAX_DEGREE_OF_INDEX_NODE = TSFileDescriptor.getInstance().getConfig()
+      .getMaxDegreeOfIndexNode();
+
+  private MetadataIndexConstructor() {
+    throw new IllegalStateException("Utility class");
+  }
+
+  /**
+   * Construct metadata index tree
+   *
+   * @param deviceTimeseriesMetadataMap device - >List<TimeseriesMetadata>
+   * @param out tsfile output
+   */
+  public static MetadataIndexNode constructMetadataIndex(Map<String, List<TimeseriesMetadata>>
+      deviceTimeseriesMetadataMap, TsFileOutput out) throws IOException {
+    Map<String, MetadataIndexNode> deviceMetadataIndexMap = new TreeMap<>();
+
+    // for timeseriesMetadata of each device
+    for (Entry<String, List<TimeseriesMetadata>> entry : deviceTimeseriesMetadataMap.entrySet()) {
+      if (entry.getValue().isEmpty()) {
+        continue;
+      }
+      Queue<MetadataIndexNode> measurementMetadataIndexQueue = new ArrayDeque<>();
+      TimeseriesMetadata timeseriesMetadata;
+      MetadataIndexNode currentIndexNode = new MetadataIndexNode();
+      for (int i = 0; i < entry.getValue().size(); i++) {
+        timeseriesMetadata = entry.getValue().get(i);
+        // when constructing from leaf node, every "degree number of nodes" are related to an entry
+        if (i % MAX_DEGREE_OF_INDEX_NODE == 0) {
+          if (currentIndexNode.isFull()) {
+            addCurrentIndexNodeToQueue(currentIndexNode, measurementMetadataIndexQueue,
+                out);
+            currentIndexNode = new MetadataIndexNode();
+          }
+          currentIndexNode.addEntry(new MetadataIndexEntry(timeseriesMetadata.getMeasurementId(),
+              out.getPosition(), MetadataIndexNodeType.LEAF_MEASUREMENT));
+        }
+        timeseriesMetadata.serializeTo(out.wrapAsStream());
+      }
+      addCurrentIndexNodeToQueue(currentIndexNode, measurementMetadataIndexQueue, out);
+      deviceMetadataIndexMap.put(entry.getKey(), generateRootNode(measurementMetadataIndexQueue,
+          out, MetadataIndexNodeType.INTERNAL_MEASUREMENT));
+    }
+
+    // if not exceed the max child nodes num, ignore the device index and directly point to the measurement
+    if (deviceMetadataIndexMap.size() <= MAX_DEGREE_OF_INDEX_NODE) {
+      MetadataIndexNode metadataIndexNode = new MetadataIndexNode();
+      for (Map.Entry<String, MetadataIndexNode> entry : deviceMetadataIndexMap.entrySet()) {
+        metadataIndexNode.addEntry(new MetadataIndexEntry(entry.getKey(), out.getPosition(),
+            MetadataIndexNodeType.INTERNAL_MEASUREMENT));
+        entry.getValue().serializeTo(out.wrapAsStream());
+      }
+      metadataIndexNode.setEndOffset(out.getPosition());
+      return metadataIndexNode;
+    }
+
+    // else, build level index for devices
+    Queue<MetadataIndexNode> deviceMetadaIndexQueue = new ArrayDeque<>();
+    MetadataIndexNode currentIndexNode = new MetadataIndexNode();
+    MetadataIndexNode deviceMetadataIndexNode;
+
+    for (Map.Entry<String, MetadataIndexNode> entry : deviceMetadataIndexMap.entrySet()) {
+      // when constructing from internal node, each node is related to an entry
+      if (currentIndexNode.isFull()) {
+        addCurrentIndexNodeToQueue(currentIndexNode, deviceMetadaIndexQueue, out);
+        currentIndexNode = new MetadataIndexNode();
+      }
+      currentIndexNode.addEntry(new MetadataIndexEntry(entry.getKey(),
+          out.getPosition(), MetadataIndexNodeType.LEAF_DEVICE));
+      entry.getValue().serializeTo(out.wrapAsStream());
+    }
+    addCurrentIndexNodeToQueue(currentIndexNode, deviceMetadaIndexQueue, out);
+    deviceMetadataIndexNode = generateRootNode(deviceMetadaIndexQueue,
+        out, MetadataIndexNodeType.INTERNAL_DEVICE);
+    deviceMetadataIndexNode.setEndOffset(out.getPosition());
+    return deviceMetadataIndexNode;
+  }
+
+  /**
+   * Generate root node, using the nodes in the queue as leaf nodes. The final metadata tree has two
+   * levels: measurement leaf nodes will generate to measurement root node; device leaf nodes will
+   * generate to device root node
+   *
+   * @param metadataIndexNodeQueue queue of metadataIndexNode
+   * @param out tsfile output
+   * @param type MetadataIndexNode type
+   */
+  private static MetadataIndexNode generateRootNode(Queue<MetadataIndexNode> metadataIndexNodeQueue,
+      TsFileOutput out, MetadataIndexNodeType type) throws IOException {
+    int queueSize = metadataIndexNodeQueue.size();
+    MetadataIndexNode metadataIndexNode;
+    MetadataIndexNode currentIndexNode = new MetadataIndexNode();
+    while (queueSize != 1) {
+      for (int i = 0; i < queueSize; i++) {
+        metadataIndexNode = metadataIndexNodeQueue.poll();
+        // when constructing from internal node, each node is related to an entry
+        if (currentIndexNode.isFull()) {
+          addCurrentIndexNodeToQueue(currentIndexNode, metadataIndexNodeQueue, out);
+          currentIndexNode = new MetadataIndexNode();
+        }
+        currentIndexNode.addEntry(new MetadataIndexEntry(metadataIndexNode.peek().getName(),
+            out.getPosition(), type));
+        metadataIndexNode.serializeTo(out.wrapAsStream());
+      }
+      addCurrentIndexNodeToQueue(currentIndexNode, metadataIndexNodeQueue, out);
+      currentIndexNode = new MetadataIndexNode();
+      queueSize = metadataIndexNodeQueue.size();
+    }
+    return metadataIndexNodeQueue.poll();
+  }
+
+  private static void addCurrentIndexNodeToQueue(MetadataIndexNode currentIndexNode,
+      Queue<MetadataIndexNode> metadataIndexNodeQueue, TsFileOutput out) throws IOException {
+    currentIndexNode.setEndOffset(out.getPosition());
+    metadataIndexNodeQueue.add(currentIndexNode);
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexEntry.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexEntry.java
new file mode 100644
index 0000000..6d53878
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexEntry.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.tsfile.file.metadata;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import org.apache.iotdb.tsfile.file.metadata.enums.MetadataIndexNodeType;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+public class MetadataIndexEntry {
+
+  private String name;
+  private long offset;
+
+  /**
+   * type of the child node at offset
+   */
+  private MetadataIndexNodeType childNodeType;
+
+  public MetadataIndexEntry() {
+  }
+
+  public MetadataIndexEntry(String name, long offset, MetadataIndexNodeType childNodeType) {
+    this.name = name;
+    this.offset = offset;
+    this.childNodeType = childNodeType;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public long getOffset() {
+    return offset;
+  }
+
+  public MetadataIndexNodeType getChildNodeType() {
+    return childNodeType;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public void setOffset(long offset) {
+    this.offset = offset;
+  }
+
+  public void setChildNodeType(MetadataIndexNodeType childNodeType) {
+    this.childNodeType = childNodeType;
+  }
+
+  public String toString() {
+    return "<" + name + "," + offset + "," + childNodeType + ">";
+  }
+
+  public int serializeTo(OutputStream outputStream) throws IOException {
+    int byteLen = 0;
+    byteLen += ReadWriteIOUtils.write(name, outputStream);
+    byteLen += ReadWriteIOUtils.write(offset, outputStream);
+    byteLen += ReadWriteIOUtils.write(childNodeType.serialize(), outputStream);
+    return byteLen;
+  }
+
+  public static MetadataIndexEntry deserializeFrom(ByteBuffer buffer) {
+    MetadataIndexEntry metadataIndex = new MetadataIndexEntry();
+    metadataIndex.setName(ReadWriteIOUtils.readString(buffer));
+    metadataIndex.setOffset(ReadWriteIOUtils.readLong(buffer));
+    metadataIndex
+        .setChildNodeType(MetadataIndexNodeType.deserialize(ReadWriteIOUtils.readByte(buffer)));
+    return metadataIndex;
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexNode.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexNode.java
new file mode 100644
index 0000000..275521e
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexNode.java
@@ -0,0 +1,125 @@
+/*
+ * 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.tsfile.file.metadata;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+public class MetadataIndexNode {
+
+  private static final int MAX_DEGREE_OF_INDEX_NODE = TSFileDescriptor.getInstance().getConfig()
+      .getMaxDegreeOfIndexNode();
+  private List<MetadataIndexEntry> children;
+  private long endOffset;
+
+  public MetadataIndexNode() {
+    children = new ArrayList<>();
+    endOffset = -1L;
+  }
+
+  public MetadataIndexNode(List<MetadataIndexEntry> children, long endOffset) {
+    this.children = children;
+    this.endOffset = endOffset;
+  }
+
+  public List<MetadataIndexEntry> getChildren() {
+    return children;
+  }
+
+  public long getEndOffset() {
+    return endOffset;
+  }
+
+  public void setEndOffset(long endOffset) {
+    this.endOffset = endOffset;
+  }
+
+  public void addEntry(MetadataIndexEntry metadataIndexEntry) {
+    this.children.add(metadataIndexEntry);
+  }
+
+  boolean isFull() {
+    return children.size() == MAX_DEGREE_OF_INDEX_NODE;
+  }
+
+  MetadataIndexEntry peek() {
+    if (children.isEmpty()) {
+      return null;
+    }
+    return children.get(0);
+  }
+
+  public int serializeTo(OutputStream outputStream) throws IOException {
+    int byteLen = 0;
+    byteLen += ReadWriteIOUtils.write(children.size(), outputStream);
+    for (MetadataIndexEntry metadataIndexEntry : children) {
+      byteLen += metadataIndexEntry.serializeTo(outputStream);
+    }
+    byteLen += ReadWriteIOUtils.write(endOffset, outputStream);
+    return byteLen;
+  }
+
+  public static MetadataIndexNode deserializeFrom(ByteBuffer buffer) {
+    List<MetadataIndexEntry> children = new ArrayList<>();
+    int size = ReadWriteIOUtils.readInt(buffer);
+    for (int i = 0; i < size; i++) {
+      children.add(MetadataIndexEntry.deserializeFrom(buffer));
+    }
+    long offset = ReadWriteIOUtils.readLong(buffer);
+    return new MetadataIndexNode(children, offset);
+  }
+
+  public Pair<MetadataIndexEntry, Long> getChildIndexEntry(String key) {
+    int index = binarySearchInChildren(key);
+    long childEndOffset;
+    if (index != children.size() - 1) {
+      childEndOffset = children.get(index + 1).getOffset();
+    } else {
+      childEndOffset = this.endOffset;
+    }
+    return new Pair<>(children.get(index), childEndOffset);
+  }
+
+  int binarySearchInChildren(String key) {
+    int low = 0;
+    int high = children.size() - 1;
+
+    while (low <= high) {
+      int mid = (low + high) >>> 1;
+      MetadataIndexEntry midVal = children.get(mid);
+      int cmp = midVal.getName().compareTo(key);
+
+      if (cmp < 0) {
+        low = mid + 1;
+      } else if (cmp > 0) {
+        high = mid - 1;
+      } else {
+        return mid; // key found
+      }
+    }
+    return low - 1;  // key not found
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java
index e0fa68f..0522ed7 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java
@@ -19,15 +19,14 @@
 
 package org.apache.iotdb.tsfile.file.metadata;
 
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
-import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
-import org.apache.iotdb.tsfile.read.controller.IChunkMetadataLoader;
-import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
 import java.util.List;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.read.controller.IChunkMetadataLoader;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 public class TimeseriesMetadata {
 
@@ -36,9 +35,10 @@ public class TimeseriesMetadata {
 
   private String measurementId;
   private TSDataType tsDataType;
-  
+
   private Statistics<?> statistics;
-// modified is true when there are modifications of the series, or from unseq file
+
+  // modified is true when there are modifications of the series, or from unseq file
   private boolean modified;
 
   private IChunkMetadataLoader chunkMetadataLoader;
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TsFileMetadata.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TsFileMetadata.java
index 3195c08..6098284 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TsFileMetadata.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TsFileMetadata.java
@@ -19,19 +19,17 @@
 
 package org.apache.iotdb.tsfile.file.metadata;
 
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.utils.BloomFilter;
 import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
 
 /**
  * TSFileMetaData collects all metadata info and saves in its data structure.
@@ -48,8 +46,8 @@ public class TsFileMetadata {
   // bloom filter
   private BloomFilter bloomFilter;
 
-  // DeviceId -> offset and length of Map<String, TimeseriesMetadata>
-  private Map<String, Pair<Long, Integer>> deviceMetadataIndex;
+  // List of <name, offset, childMetadataIndexType>
+  private MetadataIndexNode metadataIndex;
 
   // offset -> version
   private List<Pair<Long, Long>> versionInfo;
@@ -66,19 +64,8 @@ public class TsFileMetadata {
   public static TsFileMetadata deserializeFrom(ByteBuffer buffer) {
     TsFileMetadata fileMetaData = new TsFileMetadata();
 
-    // deviceMetadataIndex
-    int deviceNum = ReadWriteIOUtils.readInt(buffer);
-    Map<String, Pair<Long, Integer>> deviceMetaDataMap = new HashMap<>();
-    if (deviceNum > 0) {
-      for (int i = 0; i < deviceNum; i++) {
-        String deviceId = ReadWriteIOUtils.readString(buffer);
-        long offset = ReadWriteIOUtils.readLong(buffer);
-        int length = ReadWriteIOUtils.readInt(buffer);
-        deviceMetaDataMap.put(deviceId, new Pair<>(offset, length));
-      }
-    }
-    fileMetaData.setDeviceMetadataIndex(deviceMetaDataMap);
-
+    // metadataIndex
+    fileMetaData.metadataIndex = MetadataIndexNode.deserializeFrom(buffer);
     fileMetaData.totalChunkNum = ReadWriteIOUtils.readInt(buffer);
     fileMetaData.invalidChunkNum = ReadWriteIOUtils.readInt(buffer);
 
@@ -120,14 +107,9 @@ public class TsFileMetadata {
   public int serializeTo(OutputStream outputStream) throws IOException {
     int byteLen = 0;
 
-    // deviceMetadataIndex
-    if (deviceMetadataIndex != null) {
-      byteLen += ReadWriteIOUtils.write(deviceMetadataIndex.size(), outputStream);
-      for (Map.Entry<String, Pair<Long, Integer>> entry : deviceMetadataIndex.entrySet()) {
-        byteLen += ReadWriteIOUtils.write(entry.getKey(), outputStream);
-        byteLen += ReadWriteIOUtils.write(entry.getValue().left, outputStream);
-        byteLen += ReadWriteIOUtils.write(entry.getValue().right, outputStream);
-      }
+    // metadataIndex
+    if (metadataIndex != null) {
+      byteLen += metadataIndex.serializeTo(outputStream);
     } else {
       byteLen += ReadWriteIOUtils.write(0, outputStream);
     }
@@ -152,7 +134,7 @@ public class TsFileMetadata {
   /**
    * use the given outputStream to serialize bloom filter.
    *
-   * @param outputStream      -output stream to determine byte length
+   * @param outputStream -output stream to determine byte length
    * @return -byte length
    */
   public int serializeBloomFilter(OutputStream outputStream, Set<Path> paths)
@@ -208,12 +190,12 @@ public class TsFileMetadata {
     this.metaOffset = metaOffset;
   }
 
-  public Map<String, Pair<Long, Integer>> getDeviceMetadataIndex() {
-    return deviceMetadataIndex;
+  public MetadataIndexNode getMetadataIndex() {
+    return metadataIndex;
   }
 
-  public void setDeviceMetadataIndex(Map<String, Pair<Long, Integer>> deviceMetadataIndex) {
-    this.deviceMetadataIndex = deviceMetadataIndex;
+  public void setMetadataIndex(MetadataIndexNode metadataIndex) {
+    this.metadataIndex = metadataIndex;
   }
 
   public void setVersionInfo(List<Pair<Long, Long>> versionInfo) {
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/MetadataIndexNodeType.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/MetadataIndexNodeType.java
new file mode 100644
index 0000000..cb0a2c2
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/MetadataIndexNodeType.java
@@ -0,0 +1,86 @@
+/*
+ * 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.tsfile.file.metadata.enums;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public enum MetadataIndexNodeType {
+  INTERNAL_DEVICE, LEAF_DEVICE, INTERNAL_MEASUREMENT, LEAF_MEASUREMENT;
+
+  /**
+   * deserialize byte number.
+   *
+   * @param i byte number
+   * @return MetadataIndexNodeType
+   */
+  public static MetadataIndexNodeType deserialize(byte i) {
+    if (i >= 4) {
+      throw new IllegalArgumentException("Invalid input: " + i);
+    }
+    switch (i) {
+      case 0:
+        return INTERNAL_DEVICE;
+      case 1:
+        return LEAF_DEVICE;
+      case 2:
+        return INTERNAL_MEASUREMENT;
+      default:
+        return LEAF_MEASUREMENT;
+    }
+  }
+
+  public static MetadataIndexNodeType deserializeFrom(ByteBuffer buffer) {
+    return deserialize(buffer.get());
+  }
+
+  public static int getSerializedSize() {
+    return Byte.BYTES;
+  }
+
+  public void serializeTo(ByteBuffer byteBuffer) {
+    byteBuffer.put(serialize());
+  }
+
+  public void serializeTo(DataOutputStream outputStream) throws IOException {
+    outputStream.write(serialize());
+  }
+
+  /**
+   * return a serialize child metadata index type.
+   *
+   * @return -enum type
+   */
+  public byte serialize() {
+    switch (this) {
+      case INTERNAL_DEVICE:
+        return 0;
+      case LEAF_DEVICE:
+        return 1;
+      case INTERNAL_MEASUREMENT:
+        return 2;
+      case LEAF_MEASUREMENT:
+        return 3;
+      default:
+        return -1;
+    }
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
index eb0c6ae..1e3c8a7 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
@@ -18,6 +18,23 @@
  */
 package org.apache.iotdb.tsfile.read;
 
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.compress.IUnCompressor;
@@ -28,9 +45,12 @@ import org.apache.iotdb.tsfile.file.header.ChunkHeader;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
 import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetadata;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.MetadataIndexEntry;
+import org.apache.iotdb.tsfile.file.metadata.MetadataIndexNode;
 import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
 import org.apache.iotdb.tsfile.file.metadata.TsFileMetadata;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
+import org.apache.iotdb.tsfile.file.metadata.enums.MetadataIndexNodeType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
@@ -45,15 +65,6 @@ import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.*;
-import java.util.Map.Entry;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 public class TsFileSequenceReader implements AutoCloseable {
 
   private static final Logger logger = LoggerFactory.getLogger(TsFileSequenceReader.class);
@@ -88,7 +99,7 @@ public class TsFileSequenceReader implements AutoCloseable {
   /**
    * construct function for TsFileSequenceReader.
    *
-   * @param file             -given file name
+   * @param file -given file name
    * @param loadMetadataSize -whether load meta data size
    */
   public TsFileSequenceReader(String file, boolean loadMetadataSize) throws IOException {
@@ -129,7 +140,7 @@ public class TsFileSequenceReader implements AutoCloseable {
   /**
    * construct function for TsFileSequenceReader.
    *
-   * @param input            -given input
+   * @param input -given input
    * @param loadMetadataSize -load meta data size
    */
   public TsFileSequenceReader(TsFileInput input, boolean loadMetadataSize) throws IOException {
@@ -147,10 +158,10 @@ public class TsFileSequenceReader implements AutoCloseable {
   /**
    * construct function for TsFileSequenceReader.
    *
-   * @param input            the input of a tsfile. The current position should be a markder and
-   *                         then a chunk Header, rather than the magic number
-   * @param fileMetadataPos  the position of the file metadata in the TsFileInput from the beginning
-   *                         of the input to the current position
+   * @param input the input of a tsfile. The current position should be a markder and then a chunk
+   * Header, rather than the magic number
+   * @param fileMetadataPos the position of the file metadata in the TsFileInput from the beginning
+   * of the input to the current position
    * @param fileMetadataSize the byte size of the file metadata in the input
    */
   public TsFileSequenceReader(TsFileInput input, long fileMetadataPos, int fileMetadataSize) {
@@ -213,7 +224,7 @@ public class TsFileSequenceReader implements AutoCloseable {
    * this function does not modify the position of the file reader.
    *
    * @param movePosition whether move the position of the file reader after reading the magic header
-   *                     to the end of the magic head string.
+   * to the end of the magic head string.
    */
   public String readHeadMagic(boolean movePosition) throws IOException {
     ByteBuffer magicStringBytes = ByteBuffer.allocate(TSFileConfig.MAGIC_STRING.getBytes().length);
@@ -244,6 +255,7 @@ public class TsFileSequenceReader implements AutoCloseable {
 
   /**
    * this function does not modify the position of the file reader.
+   *
    * @throws IOException io error
    */
   public TsFileMetadata readFileMetadata() throws IOException {
@@ -254,8 +266,8 @@ public class TsFileSequenceReader implements AutoCloseable {
   }
 
   /**
-   * this function reads measurements and TimeseriesMetaDatas in given device
-   * Thread Safe
+   * this function reads measurements and TimeseriesMetaDatas in given device Thread Safe
+   *
    * @param device name
    * @return the map measurementId -> TimeseriesMetaData in one device
    * @throws IOException io error
@@ -270,7 +282,7 @@ public class TsFileSequenceReader implements AutoCloseable {
       if (cachedDeviceMetadata.containsKey(device)) {
         return cachedDeviceMetadata.get(device);
       }
-    } finally{
+    } finally {
       cacheLock.readLock().unlock();
     }
 
@@ -280,9 +292,6 @@ public class TsFileSequenceReader implements AutoCloseable {
         return cachedDeviceMetadata.get(device);
       }
       readFileMetadata();
-      if (!tsFileMetaData.getDeviceMetadataIndex().containsKey(device)) {
-        return new HashMap<>();
-      }
       Map<String, TimeseriesMetadata> deviceMetadata = readDeviceMetadataFromDisk(device);
       cachedDeviceMetadata.put(device, deviceMetadata);
       return deviceMetadata;
@@ -291,21 +300,77 @@ public class TsFileSequenceReader implements AutoCloseable {
     }
   }
 
-  private Map<String, TimeseriesMetadata> readDeviceMetadataFromDisk(String device) throws IOException {
+  private Map<String, TimeseriesMetadata> readDeviceMetadataFromDisk(String device)
+      throws IOException {
     readFileMetadata();
-    if (!tsFileMetaData.getDeviceMetadataIndex().containsKey(device)) {
-      return Collections.emptyMap();
-    }
-    Pair<Long, Integer> deviceMetadataIndex = tsFileMetaData.getDeviceMetadataIndex().get(device);
+    List<TimeseriesMetadata> timeseriesMetadataList = getDeviceTimeseriesMetadata(device);
     Map<String, TimeseriesMetadata> deviceMetadata = new HashMap<>();
-    ByteBuffer buffer = readData(deviceMetadataIndex.left, deviceMetadataIndex.right);
-    while (buffer.hasRemaining()) {
-      TimeseriesMetadata tsMetaData = TimeseriesMetadata.deserializeFrom(buffer);
-      deviceMetadata.put(tsMetaData.getMeasurementId(), tsMetaData);
+    for (TimeseriesMetadata timeseriesMetadata : timeseriesMetadataList) {
+      deviceMetadata.put(timeseriesMetadata.getMeasurementId(), timeseriesMetadata);
     }
     return deviceMetadata;
   }
 
+  public TimeseriesMetadata readTimeseriesMetadata(Path path) throws IOException {
+    readFileMetadata();
+    MetadataIndexNode deviceMetadataIndexNode = tsFileMetaData.getMetadataIndex();
+    Pair<MetadataIndexEntry, Long> metadataIndexPair = getMetaDataAndEndOffset(
+        deviceMetadataIndexNode, path.getDevice(), MetadataIndexNodeType.INTERNAL_DEVICE);
+    ByteBuffer buffer = readData(metadataIndexPair.left.getOffset(), metadataIndexPair.right);
+    while (!metadataIndexPair.left.getChildNodeType()
+        .equals(MetadataIndexNodeType.LEAF_MEASUREMENT)) {
+      MetadataIndexNode metadataIndexNode = MetadataIndexNode.deserializeFrom(buffer);
+      metadataIndexPair = getMetaDataAndEndOffset(metadataIndexNode,
+          path.getMeasurement(), MetadataIndexNodeType.INTERNAL_MEASUREMENT);
+    }
+    List<TimeseriesMetadata> timeseriesMetadataList = new ArrayList<>();
+    buffer = readData(metadataIndexPair.left.getOffset(), metadataIndexPair.right);
+    while (buffer.hasRemaining()) {
+      timeseriesMetadataList.add(TimeseriesMetadata.deserializeFrom(buffer));
+    }
+    String[] measurementNameList = timeseriesMetadataList.stream()
+        .map(TimeseriesMetadata::getMeasurementId).collect(Collectors.toList())
+        .toArray(new String[timeseriesMetadataList.size()]);
+
+    // return null if path does not exist in the TsFile
+    int searchResult;
+    return (searchResult = Arrays.binarySearch(measurementNameList, path.getMeasurement())) >= 0
+        ? timeseriesMetadataList.get(searchResult) : null;
+  }
+
+  public List<String> getAllDevices() throws IOException {
+    if (tsFileMetaData == null) {
+      readFileMetadata();
+    }
+    return getAllDevices(tsFileMetaData.getMetadataIndex());
+  }
+
+  private List<String> getAllDevices(MetadataIndexNode metadataIndexNode) throws IOException {
+    Set<String> deviceSet = new TreeSet<>();
+    int metadataIndexListSize = metadataIndexNode.getChildren().size();
+    for (int i = 0; i < metadataIndexListSize; i++) {
+      MetadataIndexEntry metadataIndex = metadataIndexNode.getChildren().get(i);
+      switch (metadataIndex.getChildNodeType()) {
+        case LEAF_MEASUREMENT:
+        case INTERNAL_MEASUREMENT:
+          for (MetadataIndexEntry index : metadataIndexNode.getChildren()) {
+            deviceSet.add(index.getName());
+          }
+          break;
+        case LEAF_DEVICE:
+        case INTERNAL_DEVICE:
+          long endOffset = metadataIndexNode.getEndOffset();
+          if (i != metadataIndexListSize - 1) {
+            endOffset = metadataIndexNode.getChildren().get(i + 1).getOffset();
+          }
+          ByteBuffer buffer = readData(metadataIndex.getOffset(), endOffset);
+          MetadataIndexNode node = MetadataIndexNode.deserializeFrom(buffer);
+          deviceSet.addAll(getAllDevices(node));
+          break;
+      }
+    }
+    return new ArrayList<>(deviceSet);
+  }
 
   /**
    * read all ChunkMetaDatas of given device
@@ -314,31 +379,24 @@ public class TsFileSequenceReader implements AutoCloseable {
    * @return measurement -> ChunkMetadata list
    * @throws IOException io error
    */
-  public Map<String, List<ChunkMetadata>> readChunkMetadataInDevice(String device) throws IOException {
+  public Map<String, List<ChunkMetadata>> readChunkMetadataInDevice(String device)
+      throws IOException {
     if (tsFileMetaData == null) {
       readFileMetadata();
     }
-    if (tsFileMetaData.getDeviceMetadataIndex() == null
-        || !tsFileMetaData.getDeviceMetadataIndex().containsKey(device)) {
-      return new HashMap<>();
-    }
 
-    Pair<Long, Integer> deviceMetaData = tsFileMetaData.getDeviceMetadataIndex().get(device);
-    ByteBuffer buffer = readData(deviceMetaData.left, deviceMetaData.right);
     long start = 0;
     int size = 0;
-    while (buffer.hasRemaining()) {
-      TimeseriesMetadata timeseriesMetaData = TimeseriesMetadata.deserializeFrom(buffer);
+    List<TimeseriesMetadata> timeseriesMetadataMap = getDeviceTimeseriesMetadata(device);
+    for (TimeseriesMetadata timeseriesMetadata : timeseriesMetadataMap) {
       if (start == 0) {
-        start = timeseriesMetaData.getOffsetOfChunkMetaDataList();
+        start = timeseriesMetadata.getOffsetOfChunkMetaDataList();
       }
-      size += timeseriesMetaData.getDataSizeOfChunkMetaDataList();
+      size += timeseriesMetadata.getDataSizeOfChunkMetaDataList();
     }
     // read buffer of all ChunkMetadatas of this device
-    buffer = readData(start, size);
-
+    ByteBuffer buffer = readData(start, size);
     Map<String, List<ChunkMetadata>> seriesMetadata = new HashMap<>();
-
     while (buffer.hasRemaining()) {
       ChunkMetadata chunkMetadata = ChunkMetadata.deserializeFrom(buffer);
       seriesMetadata.computeIfAbsent(chunkMetadata.getMeasurementUid(), key -> new ArrayList<>())
@@ -361,20 +419,96 @@ public class TsFileSequenceReader implements AutoCloseable {
    */
   public List<Path> getAllPaths() throws IOException {
     List<Path> paths = new ArrayList<>();
+    for (String device : getAllDevices()) {
+      Map<String, TimeseriesMetadata> timeseriesMetadataMap = readDeviceMetadata(device);
+      for (String measurementId : timeseriesMetadataMap.keySet()) {
+        paths.add(new Path(device, measurementId));
+      }
+    }
+    return paths;
+  }
+
+  /**
+   * Traverse the metadata index from MetadataIndexEntry to get TimeseriesMetadatas
+   *
+   * @param metadataIndex MetadataIndexEntry
+   * @param buffer byte buffer
+   * @param timeseriesMetadataMap map: deviceId -> timeseriesMetadata list
+   */
+  private void generateMetadataIndex(MetadataIndexEntry metadataIndex, ByteBuffer buffer,
+      Map<String, List<TimeseriesMetadata>> timeseriesMetadataMap) throws IOException {
+    switch (metadataIndex.getChildNodeType()) {
+      case INTERNAL_DEVICE:
+      case LEAF_DEVICE:
+      case INTERNAL_MEASUREMENT:
+        MetadataIndexNode metadataIndexNode = MetadataIndexNode.deserializeFrom(buffer);
+        int metadataIndexListSize = metadataIndexNode.getChildren().size();
+        for (int i = 0; i < metadataIndexListSize; i++) {
+          long endOffset = metadataIndexNode.getEndOffset();
+          if (i != metadataIndexListSize - 1) {
+            endOffset = metadataIndexNode.getChildren().get(i + 1).getOffset();
+          }
+          ByteBuffer nextBuffer = readData(metadataIndexNode.getChildren().get(i).getOffset(),
+              endOffset);
+          generateMetadataIndex(metadataIndexNode.getChildren().get(i), nextBuffer,
+              timeseriesMetadataMap);
+        }
+        break;
+      case LEAF_MEASUREMENT:
+        String deviceId = metadataIndex.getName();
+        List<TimeseriesMetadata> timeseriesMetadataList = new ArrayList<>();
+        while (buffer.hasRemaining()) {
+          timeseriesMetadataList.add(TimeseriesMetadata.deserializeFrom(buffer));
+        }
+        if (timeseriesMetadataMap.containsKey(deviceId)) {
+          timeseriesMetadataList.addAll(timeseriesMetadataMap.get(deviceId));
+        }
+        timeseriesMetadataMap.put(deviceId, timeseriesMetadataList);
+        break;
+    }
+  }
+
+  public Map<String, List<TimeseriesMetadata>> getAllTimeseriesMetadata() throws IOException {
     if (tsFileMetaData == null) {
       readFileMetadata();
     }
-    Map<String, Pair<Long, Integer>> deviceMetaDataMap = tsFileMetaData.getDeviceMetadataIndex();
-    for (Map.Entry<String, Pair<Long, Integer>> entry : deviceMetaDataMap.entrySet()) {
-      String deviceId = entry.getKey();
-      Pair<Long, Integer> deviceMetaData = entry.getValue();
-      ByteBuffer buffer = readData(deviceMetaData.left, deviceMetaData.right);
-      while (buffer.hasRemaining()) {
-        TimeseriesMetadata tsMetaData = TimeseriesMetadata.deserializeFrom(buffer);
-        paths.add(new Path(deviceId, tsMetaData.getMeasurementId()));
+    Map<String, List<TimeseriesMetadata>> timeseriesMetadataMap = new HashMap<>();
+    List<MetadataIndexEntry> metadataIndexEntryList = tsFileMetaData.getMetadataIndex()
+        .getChildren();
+    for (int i = 0; i < metadataIndexEntryList.size(); i++) {
+      MetadataIndexEntry metadataIndexEntry = metadataIndexEntryList.get(i);
+      long endOffset = tsFileMetaData.getMetadataIndex().getEndOffset();
+      if (i != metadataIndexEntryList.size() - 1) {
+        endOffset = metadataIndexEntryList.get(i + 1).getOffset();
       }
+      ByteBuffer buffer = readData(metadataIndexEntry.getOffset(), endOffset);
+      generateMetadataIndex(metadataIndexEntry, buffer, timeseriesMetadataMap);
     }
-    return paths;
+    return timeseriesMetadataMap;
+  }
+
+  private List<TimeseriesMetadata> getDeviceTimeseriesMetadata(String device) throws IOException {
+    MetadataIndexNode metadataIndexNode = tsFileMetaData.getMetadataIndex();
+    Pair<MetadataIndexEntry, Long> metadataIndexPair = getMetaDataAndEndOffset(
+        metadataIndexNode, device, MetadataIndexNodeType.INTERNAL_DEVICE);
+    ByteBuffer buffer = readData(metadataIndexPair.left.getOffset(), metadataIndexPair.right);
+    Map<String, List<TimeseriesMetadata>> timeseriesMetadataMap = new TreeMap<>();
+    generateMetadataIndex(metadataIndexPair.left, buffer, timeseriesMetadataMap);
+    List<TimeseriesMetadata> deviceTimeseriesMetadata = new ArrayList<>();
+    for (List<TimeseriesMetadata> timeseriesMetadataList : timeseriesMetadataMap.values()) {
+      deviceTimeseriesMetadata.addAll(timeseriesMetadataList);
+    }
+    return deviceTimeseriesMetadata;
+  }
+
+  private Pair<MetadataIndexEntry, Long> getMetaDataAndEndOffset(MetadataIndexNode metadataIndex,
+      String name, MetadataIndexNodeType type) throws IOException {
+    Pair<MetadataIndexEntry, Long> childIndexEntry = metadataIndex.getChildIndexEntry(name);
+    if (!childIndexEntry.left.getChildNodeType().equals(type)) {
+      return childIndexEntry;
+    }
+    ByteBuffer buffer = readData(childIndexEntry.left.getOffset(), childIndexEntry.right);
+    return getMetaDataAndEndOffset(MetadataIndexNode.deserializeFrom(buffer), name, type);
   }
 
   /**
@@ -391,7 +525,7 @@ public class TsFileSequenceReader implements AutoCloseable {
   /**
    * read data from current position of the input, and deserialize it to a CHUNK_GROUP_FOOTER.
    *
-   * @param position   the offset of the chunk group footer in the file
+   * @param position the offset of the chunk group footer in the file
    * @param markerRead true if the offset does not contains the marker , otherwise false
    * @return a CHUNK_GROUP_FOOTER
    * @throws IOException io error
@@ -424,9 +558,9 @@ public class TsFileSequenceReader implements AutoCloseable {
   /**
    * read the chunk's header.
    *
-   * @param position        the file offset of this chunk's header
+   * @param position the file offset of this chunk's header
    * @param chunkHeaderSize the size of chunk's header
-   * @param markerRead      true if the offset does not contains the marker , otherwise false
+   * @param markerRead true if the offset does not contains the marker , otherwise false
    */
   private ChunkHeader readChunkHeader(long position, int chunkHeaderSize, boolean markerRead)
       throws IOException {
@@ -531,13 +665,13 @@ public class TsFileSequenceReader implements AutoCloseable {
    * changed.
    *
    * @param position the start position of data in the tsFileInput, or the current position if
-   *                 position = -1
-   * @param size     the size of data that want to read
+   * position = -1
+   * @param size the size of data that want to read
    * @return data that been read.
    */
   private ByteBuffer readData(long position, int size) throws IOException {
     ByteBuffer buffer = ByteBuffer.allocate(size);
-    if (position == -1) {
+    if (position < 0) {
       if (ReadWriteIOUtils.readAsPossible(tsFileInput, buffer) != size) {
         throw new IOException("reach the end of the data");
       }
@@ -551,6 +685,19 @@ public class TsFileSequenceReader implements AutoCloseable {
   }
 
   /**
+   * read data from tsFileInput, from the current position (if position = -1), or the given
+   * position.
+   *
+   * @param start the start position of data in the tsFileInput, or the current position if position
+   * = -1
+   * @param end the end position of data that want to read
+   * @return data that been read.
+   */
+  private ByteBuffer readData(long start, long end) throws IOException {
+    return readData(start, (int) (end - start));
+  }
+
+  /**
    * notice, the target bytebuffer are not flipped.
    */
   public int readRaw(long position, int length, ByteBuffer target) throws IOException {
@@ -560,10 +707,10 @@ public class TsFileSequenceReader implements AutoCloseable {
   /**
    * Self Check the file and return the position before where the data is safe.
    *
-   * @param newSchema   the schema on each time series in the file
-   * @param chunkGroupMetadataList  ChunkGroupMetadata List
-   * @param fastFinish  if true and the file is complete, then newSchema and newMetaData parameter
-   *                    will be not modified.
+   * @param newSchema the schema on each time series in the file
+   * @param chunkGroupMetadataList ChunkGroupMetadata List
+   * @param fastFinish if true and the file is complete, then newSchema and newMetaData parameter
+   * will be not modified.
    * @return the position of the file that is fine. All data after the position in the file should
    * be truncated.
    */
@@ -589,7 +736,7 @@ public class TsFileSequenceReader implements AutoCloseable {
     String deviceID;
 
     int headerLength = TSFileConfig.MAGIC_STRING.getBytes().length + TSFileConfig.VERSION_NUMBER
-            .getBytes().length;
+        .getBytes().length;
     if (fileSize < headerLength) {
       return TsFileCheckStatus.INCOMPATIBLE_FILE;
     }
@@ -697,10 +844,7 @@ public class TsFileSequenceReader implements AutoCloseable {
    * @return List of ChunkMetaData
    */
   public List<ChunkMetadata> getChunkMetadataList(Path path) throws IOException {
-    Map<String, TimeseriesMetadata> timeseriesMetaDataMap =
-        readDeviceMetadata(path.getDevice());
-
-    TimeseriesMetadata timeseriesMetaData = timeseriesMetaDataMap.get(path.getMeasurement());
+    TimeseriesMetadata timeseriesMetaData = readTimeseriesMetadata(path);
     if (timeseriesMetaData == null) {
       return new ArrayList<>();
     }
@@ -712,7 +856,6 @@ public class TsFileSequenceReader implements AutoCloseable {
   /**
    * get ChunkMetaDatas in given TimeseriesMetaData
    *
-   * @param timeseriesMetaData
    * @return List of ChunkMetaData
    */
   public List<ChunkMetadata> readChunkMetaDataList(TimeseriesMetadata timeseriesMetaData)
@@ -732,23 +875,17 @@ public class TsFileSequenceReader implements AutoCloseable {
     return chunkMetadataList;
   }
 
-
   /**
    * get all measurements in this file
+   *
    * @return measurement -> datatype
    */
-  public Map<String, TSDataType> getAllMeasurements() throws IOException{
-    if (tsFileMetaData == null) {
-      readFileMetadata();
-    }
+  public Map<String, TSDataType> getAllMeasurements() throws IOException {
     Map<String, TSDataType> result = new HashMap<>();
-    for (Map.Entry<String, Pair<Long, Integer>> entry : tsFileMetaData.getDeviceMetadataIndex()
-        .entrySet()) {
-      // read TimeseriesMetaData from file
-      ByteBuffer buffer = readData(entry.getValue().left, entry.getValue().right);
-      while (buffer.hasRemaining()) {
-        TimeseriesMetadata timeserieMetaData = TimeseriesMetadata.deserializeFrom(buffer);
-        result.put(timeserieMetaData.getMeasurementId(), timeserieMetaData.getTSDataType());
+    for (String device : getAllDevices()) {
+      Map<String, TimeseriesMetadata> timeseriesMetadataMap = readDeviceMetadata(device);
+      for (TimeseriesMetadata timeseriesMetadata : timeseriesMetadataMap.values()) {
+        result.put(timeseriesMetadata.getMeasurementId(), timeseriesMetadata.getTSDataType());
       }
     }
     return result;
@@ -758,39 +895,33 @@ public class TsFileSequenceReader implements AutoCloseable {
    * get device names which has valid chunks in [start, end)
    *
    * @param start start of the partition
-   * @param end   end of the partition
+   * @param end end of the partition
    * @return device names in range
    */
   public List<String> getDeviceNameInRange(long start, long end) throws IOException {
     List<String> res = new ArrayList<>();
-
-    readFileMetadata();
-    for (Map.Entry<String, Pair<Long, Integer>> entry : tsFileMetaData.getDeviceMetadataIndex()
-        .entrySet()) {
-
-      Map<String, List<ChunkMetadata>> seriesMetadataMap = readChunkMetadataInDevice(
-          entry.getKey());
-
+    for (String device : getAllDevices()) {
+      Map<String, List<ChunkMetadata>> seriesMetadataMap = readChunkMetadataInDevice(device);
       if (hasDataInPartition(seriesMetadataMap, start, end)) {
-        res.add(entry.getKey());
+        res.add(device);
       }
     }
-
     return res;
   }
 
   /**
    * Check if the device has at least one Chunk in this partition
    *
-   * @param seriesMetadataMap     chunkMetaDataList of each measurement
+   * @param seriesMetadataMap chunkMetaDataList of each measurement
    * @param start the start position of the space partition
-   * @param end   the end position of the space partition
+   * @param end the end position of the space partition
    */
   private boolean hasDataInPartition(Map<String, List<ChunkMetadata>> seriesMetadataMap,
       long start, long end) {
     for (List<ChunkMetadata> chunkMetadataList : seriesMetadataMap.values()) {
       for (ChunkMetadata chunkMetadata : chunkMetadataList) {
-        LocateStatus location = MetadataQuerierByFileImpl.checkLocateStatus(chunkMetadata, start, end);
+        LocateStatus location = MetadataQuerierByFileImpl
+            .checkLocateStatus(chunkMetadata, start, end);
         if (location == LocateStatus.in) {
           return true;
         }
@@ -799,14 +930,11 @@ public class TsFileSequenceReader implements AutoCloseable {
     return false;
   }
 
-
   /**
-   * The location of a chunkGroupMetaData with respect to a space partition constraint.
-   * <p>
-   * in - the middle point of the chunkGroupMetaData is located in the current space partition.
-   * before - the middle point of the chunkGroupMetaData is located before the current space
-   * partition. after - the middle point of the chunkGroupMetaData is located after the current
-   * space partition.
+   * The location of a chunkGroupMetaData with respect to a space partition constraint. <p> in - the
+   * middle point of the chunkGroupMetaData is located in the current space partition. before - the
+   * middle point of the chunkGroupMetaData is located before the current space partition. after -
+   * the middle point of the chunkGroupMetaData is located after the current space partition.
    */
   public enum LocateStatus {
     in, before, after
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IMetadataQuerier.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IMetadataQuerier.java
index 014e827..347ee15 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IMetadataQuerier.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/IMetadataQuerier.java
@@ -42,7 +42,7 @@ public interface IMetadataQuerier {
    *
    * <p>
    * call this method before calling getChunkMetaDataList() will accelerate the reading of chunk
-   * metadata, which will only read TsDeviceMetaData once
+   * metadata, which will only read TsMetaData once
    */
   void loadChunkMetaDatas(List<Path> paths) throws IOException;
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerierByFileImpl.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerierByFileImpl.java
index aa86346..bb58ec4 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerierByFileImpl.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerierByFileImpl.java
@@ -18,7 +18,17 @@
  */
 package org.apache.iotdb.tsfile.read.controller;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+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.TreeMap;
 import org.apache.iotdb.tsfile.common.cache.LRUCache;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
 import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
@@ -28,8 +38,6 @@ import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader.LocateStatus;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.common.TimeRange;
-import java.io.IOException;
-import java.util.*;
 
 public class MetadataQuerierByFileImpl implements IMetadataQuerier {
 
@@ -101,8 +109,9 @@ public class MetadataQuerierByFileImpl implements IMetadataQuerier {
       String selectedDevice = deviceMeasurements.getKey();
       // s1, s2, s3
       Set<String> selectedMeasurements = deviceMeasurements.getValue();
-      if (fileMetaData.getDeviceMetadataIndex() == null
-          || !fileMetaData.getDeviceMetadataIndex().containsKey(selectedDevice)) {
+      List<String> devices = this.tsFileReader.getAllDevices();
+      String[] deviceNames = devices.toArray(new String[devices.size()]);
+      if (Arrays.binarySearch(deviceNames, selectedDevice) < 0) {
         continue;
       }
 
@@ -177,7 +186,8 @@ public class MetadataQuerierByFileImpl implements IMetadataQuerier {
 
     TreeMap<String, Set<String>> deviceMeasurementsMap = new TreeMap<>();
     for (Path path : paths) {
-      deviceMeasurementsMap.computeIfAbsent(path.getDevice(), key -> new HashSet<>()).add(path.getMeasurement());
+      deviceMeasurementsMap.computeIfAbsent(path.getDevice(), key -> new HashSet<>())
+          .add(path.getMeasurement());
     }
     for (Map.Entry<String, Set<String>> deviceMeasurements : deviceMeasurementsMap.entrySet()) {
       String selectedDevice = deviceMeasurements.getKey();
@@ -233,17 +243,14 @@ public class MetadataQuerierByFileImpl implements IMetadataQuerier {
     return resTimeRanges;
   }
 
-
   /**
    * Check the location of a given chunkGroupMetaData with respect to a space partition constraint.
    *
-   * @param chunkMetaData          the given chunkMetaData
+   * @param chunkMetaData the given chunkMetaData
    * @param spacePartitionStartPos the start position of the space partition
-   * @param spacePartitionEndPos   the end position of the space partition
+   * @param spacePartitionEndPos the end position of the space partition
    * @return LocateStatus
    */
-
-
   public static LocateStatus checkLocateStatus(ChunkMetadata chunkMetaData,
       long spacePartitionStartPos, long spacePartitionEndPos) {
     long startOffsetOfChunk = chunkMetaData.getOffsetOfChunkHeader();
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/TsFileExecutor.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/TsFileExecutor.java
index 014d145..2ef99a8 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/TsFileExecutor.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/TsFileExecutor.java
@@ -41,12 +41,9 @@ import org.apache.iotdb.tsfile.read.reader.series.EmptyFileSeriesReader;
 import org.apache.iotdb.tsfile.read.reader.series.AbstractFileSeriesReader;
 import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReader;
 import org.apache.iotdb.tsfile.utils.BloomFilter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class TsFileExecutor implements QueryExecutor {
 
-  private static final Logger logger = LoggerFactory.getLogger(TsFileExecutor.class);
   private IMetadataQuerier metadataQuerier;
   private IChunkLoader chunkLoader;
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
index 5003803..c8dbea3 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java
@@ -73,6 +73,13 @@ public class ReadWriteIOUtils {
   }
 
   /**
+   * read a byte from byteBuffer.
+   */
+  public static byte readByte(ByteBuffer buffer) {
+    return buffer.get();
+  }
+
+  /**
    * read bytes array in given size
    *
    * @param buffer buffer
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/RestorableTsFileIOWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/RestorableTsFileIOWriter.java
index a55e8a5..8a12b05 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/RestorableTsFileIOWriter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/RestorableTsFileIOWriter.java
@@ -32,13 +32,11 @@ import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.exception.NotCompatibleTsFileException;
 import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetadata;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
-import org.apache.iotdb.tsfile.file.metadata.TsFileMetadata;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer;
 import org.apache.iotdb.tsfile.read.TsFileCheckStatus;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 import org.apache.iotdb.tsfile.read.common.Path;
-import org.apache.iotdb.tsfile.utils.Pair;
 import org.apache.iotdb.tsfile.utils.VersionUtils;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 import org.slf4j.Logger;
@@ -128,12 +126,7 @@ public class RestorableTsFileIOWriter extends TsFileIOWriter {
       // this tsfile is complete
       if (reader.isComplete()) {
         reader.loadMetadataSize();
-        TsFileMetadata metaData = reader.readFileMetadata();
-        for (Pair<Long, Integer> deviceMetaData : metaData.getDeviceMetadataIndex().values()) {
-          if (position > deviceMetaData.left) {
-            position = deviceMetaData.left;
-          }
-        }
+        position = reader.getFileMetadataPos();
       }
     }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java
index 06460b0..a95b776 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java
@@ -18,7 +18,15 @@
  */
 package org.apache.iotdb.tsfile.write.writer;
 
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
 import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.file.MetaMarker;
@@ -26,6 +34,8 @@ import org.apache.iotdb.tsfile.file.footer.ChunkGroupFooter;
 import org.apache.iotdb.tsfile.file.header.ChunkHeader;
 import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetadata;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.MetadataIndexConstructor;
+import org.apache.iotdb.tsfile.file.metadata.MetadataIndexNode;
 import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
 import org.apache.iotdb.tsfile.file.metadata.TsFileMetadata;
 import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
@@ -43,14 +53,6 @@ import org.apache.iotdb.tsfile.utils.VersionUtils;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
 
 /**
  * TsFileIOWriter is used to construct metadata and write data stored in memory to output stream.
@@ -62,6 +64,7 @@ public class TsFileIOWriter {
   protected static final TSFileConfig config = TSFileDescriptor.getInstance().getConfig();
   private static final Logger logger = LoggerFactory.getLogger(TsFileIOWriter.class);
   private static final Logger resourceLogger = LoggerFactory.getLogger("FileMonitor");
+
   static {
     magicStringBytes = BytesUtils.stringToBytes(TSFileConfig.MAGIC_STRING);
     versionNumberBytes = TSFileConfig.VERSION_NUMBER.getBytes();
@@ -73,7 +76,6 @@ public class TsFileIOWriter {
   protected int invalidChunkNum;
   protected File file;
 
-
   // current flushed Chunk
   private ChunkMetadata currentChunkMetadata;
   // current flushed ChunkGroup
@@ -154,7 +156,8 @@ public class TsFileIOWriter {
     ChunkGroupFooter chunkGroupFooter = new ChunkGroupFooter(currentChunkGroupDeviceId, dataSize,
         chunkMetadataList.size());
     chunkGroupFooter.serializeTo(out.wrapAsStream());
-    chunkGroupMetadataList.add(new ChunkGroupMetadata(currentChunkGroupDeviceId, chunkMetadataList));
+    chunkGroupMetadataList
+        .add(new ChunkGroupMetadata(currentChunkGroupDeviceId, chunkMetadataList));
     currentChunkGroupDeviceId = null;
     chunkMetadataList = null;
   }
@@ -162,11 +165,11 @@ public class TsFileIOWriter {
   /**
    * start a {@linkplain ChunkMetadata ChunkMetaData}.
    *
-   * @param measurementSchema    - schema of this time series
+   * @param measurementSchema - schema of this time series
    * @param compressionCodecName - compression name of this time series
-   * @param tsDataType           - data type
-   * @param statistics           - Chunk statistics
-   * @param dataSize             - the serialized size of all pages
+   * @param tsDataType - data type
+   * @param statistics - Chunk statistics
+   * @param dataSize - the serialized size of all pages
    * @throws IOException if I/O error occurs
    */
   public void startFlushChunk(MeasurementSchema measurementSchema,
@@ -225,17 +228,16 @@ public class TsFileIOWriter {
 
     // group ChunkMetadata by series
     Map<Path, List<ChunkMetadata>> chunkMetadataListMap = new TreeMap<>();
-    for (ChunkGroupMetadata chunkGroupMetadata: chunkGroupMetadataList) {
+    for (ChunkGroupMetadata chunkGroupMetadata : chunkGroupMetadataList) {
       for (ChunkMetadata chunkMetadata : chunkGroupMetadata.getChunkMetadataList()) {
         Path series = new Path(chunkGroupMetadata.getDevice(), chunkMetadata.getMeasurementUid());
         chunkMetadataListMap.computeIfAbsent(series, k -> new ArrayList<>()).add(chunkMetadata);
       }
     }
 
-    Map<String, Pair<Long, Integer>> deviceMetaDataMap = flushAllChunkMetadataList(chunkMetadataListMap);
-
+    MetadataIndexNode metadataIndex = flushMetadataIndex(chunkMetadataListMap);
     TsFileMetadata tsFileMetaData = new TsFileMetadata();
-    tsFileMetaData.setDeviceMetadataIndex(deviceMetaDataMap);
+    tsFileMetaData.setMetadataIndex(metadataIndex);
     tsFileMetaData.setVersionInfo(versionInfo);
     tsFileMetaData.setTotalChunkNum(totalChunkNum);
     tsFileMetaData.setInvalidChunkNum(invalidChunkNum);
@@ -273,10 +275,11 @@ public class TsFileIOWriter {
   }
 
   /**
-   * Flush ChunkMetadataList and TimeseriesMetaData
-   * @return DeviceMetaDataMap in TsFileMetaData
+   * Flush TsFileMetadata, including ChunkMetadataList and TimeseriesMetaData
+   *
+   * @return MetadataIndexEntry list in TsFileMetadata
    */
-  private Map<String, Pair<Long, Integer>> flushAllChunkMetadataList(
+  private MetadataIndexNode flushMetadataIndex(
       Map<Path, List<ChunkMetadata>> chunkMetadataListMap) throws IOException {
 
     // convert ChunkMetadataList to this field
@@ -304,22 +307,9 @@ public class TsFileIOWriter {
       deviceTimeseriesMetadataMap.computeIfAbsent(device, k -> new ArrayList<>())
           .add(timeseriesMetaData);
     }
-    // create DeviceMetaDataMap device -> Pair<TimeseriesMetaDataOffset, TimeseriesMetaDataLength> 
-    Map<String, Pair<Long, Integer>> deviceMetadataMap = new HashMap<>();
-    for (Map.Entry<String, List<TimeseriesMetadata>> entry : deviceTimeseriesMetadataMap
-        .entrySet()) {
-      String device = entry.getKey();
-      List<TimeseriesMetadata> timeseriesMetadataList = entry.getValue();
-      long offsetOfFirstTimeseriesMetaDataInDevice = out.getPosition();
-      int size = 0;
-      for (TimeseriesMetadata timeseriesMetaData : timeseriesMetadataList) {
-        size += timeseriesMetaData.serializeTo(out.wrapAsStream());
-      }
-      deviceMetadataMap
-          .put(device, new Pair<>(offsetOfFirstTimeseriesMetaDataInDevice, size));
-    }
-    // return
-    return deviceMetadataMap;
+
+    // construct TsFileMetadata and return
+    return MetadataIndexConstructor.constructMetadataIndex(deviceTimeseriesMetadataMap, out);
   }
 
   /**
@@ -422,8 +412,7 @@ public class TsFileIOWriter {
   }
 
   /**
-   * write MetaMarker.VERSION with version
-   * Then, cache offset-version in versionInfo
+   * write MetaMarker.VERSION with version Then, cache offset-version in versionInfo
    */
   public void writeVersion(long version) throws IOException {
     ReadWriteIOUtils.write(MetaMarker.VERSION, out.wrapAsStream());
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexNodeTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexNodeTest.java
new file mode 100644
index 0000000..33b7459
--- /dev/null
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexNodeTest.java
@@ -0,0 +1,46 @@
+/*
+ * 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.tsfile.file.metadata;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.tsfile.file.metadata.enums.MetadataIndexNodeType;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class MetadataIndexNodeTest {
+
+  @Test
+  public void testBinarySearchInChildren() {
+    List<MetadataIndexEntry> list = new ArrayList<>();
+    MetadataIndexNodeType type = MetadataIndexNodeType.LEAF_MEASUREMENT;
+    list.add(new MetadataIndexEntry("s0", -1L, type));
+    list.add(new MetadataIndexEntry("s5", -1L, type));
+    list.add(new MetadataIndexEntry("s10", -1L, type));
+    list.add(new MetadataIndexEntry("s15", -1L, type));
+    list.add(new MetadataIndexEntry("s20", -1L, type));
+
+    MetadataIndexNode metadataIndexNode = new MetadataIndexNode(list, -1L);
+    Assert.assertEquals(0, metadataIndexNode.binarySearchInChildren("s0"));
+    Assert.assertEquals(2, metadataIndexNode.binarySearchInChildren("s10"));
+    Assert.assertEquals(2, metadataIndexNode.binarySearchInChildren("s13"));
+    Assert.assertEquals(4, metadataIndexNode.binarySearchInChildren("s23"));
+  }
+}
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/file/metadata/utils/TestHelper.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/file/metadata/utils/TestHelper.java
index d805f68..6611717 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/file/metadata/utils/TestHelper.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/file/metadata/utils/TestHelper.java
@@ -19,13 +19,14 @@
 package org.apache.iotdb.tsfile.file.metadata.utils;
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
 import org.apache.iotdb.tsfile.file.header.PageHeaderTest;
+import org.apache.iotdb.tsfile.file.metadata.MetadataIndexEntry;
+import org.apache.iotdb.tsfile.file.metadata.MetadataIndexNode;
 import org.apache.iotdb.tsfile.file.metadata.TimeseriesMetadata;
 import org.apache.iotdb.tsfile.file.metadata.TsFileMetadata;
+import org.apache.iotdb.tsfile.file.metadata.enums.MetadataIndexNodeType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
@@ -36,23 +37,24 @@ public class TestHelper {
 
   public static TsFileMetadata createSimpleFileMetaData() {
     TsFileMetadata metaData = new TsFileMetadata();
-    metaData.setDeviceMetadataIndex(generateDeviceMetaDataIndex());
+    metaData.setMetadataIndex(generateMetaDataIndex());
     metaData.setVersionInfo(generateVersionInfo());
     return metaData;
   }
 
-  private static Map<String, Pair<Long, Integer>> generateDeviceMetaDataIndex() {
-    Map<String, Pair<Long, Integer>> deviceMetaDataIndex = new HashMap<>();
+  private static MetadataIndexNode generateMetaDataIndex() {
+    MetadataIndexNode metaDataIndex = new MetadataIndexNode();
     for (int i = 0; i < 5; i++) {
-      deviceMetaDataIndex.put("d" + i, new Pair<Long, Integer>((long) i * 5, 5));
+      metaDataIndex.addEntry(new MetadataIndexEntry("d" + i, (long) i * 5,
+          MetadataIndexNodeType.LEAF_MEASUREMENT));
     }
-    return deviceMetaDataIndex;
+    return metaDataIndex;
   }
 
   private static List<Pair<Long, Long>> generateVersionInfo() {
     List<Pair<Long, Long>> versionInfo = new ArrayList<>();
     for (int i = 0; i < 5; i++) {
-      versionInfo.add(new Pair<Long, Long>((long) i * 5, 0L));
+      versionInfo.add(new Pair<>((long) i * 5, 0L));
     }
     return versionInfo;
   }
@@ -60,7 +62,7 @@ public class TestHelper {
   public static MeasurementSchema createSimpleMeasurementSchema(String measurementuid) {
     return new MeasurementSchema(measurementuid, TSDataType.INT64, TSEncoding.RLE);
   }
-  
+
   public static TimeseriesMetadata createSimpleTimseriesMetaData(String measurementuid) {
     Statistics<?> statistics = Statistics.getStatsByType(PageHeaderTest.DATA_TYPE);
     statistics.setEmpty(false);
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/file/metadata/utils/Utils.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/file/metadata/utils/Utils.java
index c39294b..4dff2b2 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/file/metadata/utils/Utils.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/file/metadata/utils/Utils.java
@@ -27,9 +27,9 @@ import java.util.List;
 import java.util.Map;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata;
+import org.apache.iotdb.tsfile.file.metadata.MetadataIndexNode;
 import org.apache.iotdb.tsfile.file.metadata.TsFileMetadata;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
-import org.apache.iotdb.tsfile.utils.Pair;
 import org.junit.Assert;
 
 public class Utils {
@@ -119,13 +119,11 @@ public class Utils {
 
   public static boolean isFileMetaDataEqual(TsFileMetadata metadata1, TsFileMetadata metadata2) {
     if (Utils.isTwoObjectsNotNULL(metadata1, metadata2, "File MetaData")) {
-      if (Utils
-          .isTwoObjectsNotNULL(metadata1.getDeviceMetadataIndex(), metadata2.getDeviceMetadataIndex(),
-              "Delta object metadata list")) {
-
-        Map<String, Pair<Long, Integer>> deviceMetaDataMap1 = metadata1.getDeviceMetadataIndex();
-        Map<String, Pair<Long, Integer>> deviceMetaDataMap2 = metadata2.getDeviceMetadataIndex();
-        return deviceMetaDataMap1.size() == deviceMetaDataMap2.size();
+      if (Utils.isTwoObjectsNotNULL(metadata1.getMetadataIndex(), metadata2.getMetadataIndex(),
+          "Metadata Index")) {
+        MetadataIndexNode metaDataIndex1 = metadata1.getMetadataIndex();
+        MetadataIndexNode metaDataIndex2 = metadata2.getMetadataIndex();
+        return metaDataIndex1.getChildren().size() == metaDataIndex2.getChildren().size();
       }
     }
     return false;
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/reader/ReaderTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/reader/ReaderTest.java
index a844e80..e9f7a5f 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/reader/ReaderTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/reader/ReaderTest.java
@@ -51,6 +51,7 @@ public class ReaderTest {
   @Before
   public void before() throws IOException {
     TSFileDescriptor.getInstance().getConfig().setTimeEncoder("TS_2DIFF");
+    TSFileDescriptor.getInstance().getConfig().setMaxDegreeOfIndexNode(3);
     TsFileGeneratorForTest.generateFile(rowCount, 10 * 1024 * 1024, 10000);
     fileReader = new TsFileSequenceReader(FILE_PATH);
     metadataQuerierByFile = new MetadataQuerierByFileImpl(fileReader);
@@ -59,6 +60,7 @@ public class ReaderTest {
   @After
   public void after() throws IOException {
     fileReader.close();
+    TSFileDescriptor.getInstance().getConfig().setMaxDegreeOfIndexNode(1024);
     TsFileGeneratorForTest.after();
   }
 
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileIOWriterTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileIOWriterTest.java
index 29afae4..61e4786 100644
--- a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileIOWriterTest.java
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileIOWriterTest.java
@@ -31,9 +31,8 @@ import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
 import org.apache.iotdb.tsfile.file.metadata.utils.TestHelper;
 import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
 import org.apache.iotdb.tsfile.read.common.Path;
-import org.apache.iotdb.tsfile.utils.Pair;
-import org.apache.iotdb.tsfile.write.schema.Schema;
 import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.Schema;
 import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter;
 import org.junit.After;
 import org.junit.Assert;
@@ -109,6 +108,6 @@ public class TsFileIOWriterTest {
 
     // FileMetaData
     TsFileMetadata metaData = reader.readFileMetadata();
-    Assert.assertEquals(1, metaData.getDeviceMetadataIndex().size());
+    Assert.assertEquals(1, metaData.getMetadataIndex().getChildren().size());
   }
 }