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/28 07:54:20 UTC

[incubator-iotdb] branch master updated: [IOTDB-611] Add documents introducing Data Query design fundamentals (#1101)

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 bf10d9e  [IOTDB-611] Add documents introducing Data Query design fundamentals (#1101)
bf10d9e is described below

commit bf10d9e41184765d0b07d4c065a93bfd4b6c892e
Author: wshao08 <59...@users.noreply.github.com>
AuthorDate: Tue Apr 28 15:54:01 2020 +0800

    [IOTDB-611] Add documents introducing Data Query design fundamentals (#1101)
    
    * add Query Fundamentals doc
---
 docs/SystemDesign/1-TsFile/3-Write.md              |   2 +-
 docs/SystemDesign/5-DataQuery/1-DataQuery.md       |   4 +-
 .../5-DataQuery/2-QueryFundamentals.md             | 128 +++++++++++++++++++++
 .../5-DataQuery/3-ModificationHandle.md            |  73 ------------
 .../{2-SeriesReader.md => 3-SeriesReader.md}       |   0
 docs/zh/SystemDesign/5-DataQuery/1-DataQuery.md    |   4 +-
 .../5-DataQuery/2-QueryFundamentals.md             | 110 ++++++++++++++++++
 .../5-DataQuery/3-ModificationHandle.md            |  73 ------------
 .../{2-SeriesReader.md => 3-SeriesReader.md}       |   0
 site/src/main/.vuepress/config.js                  |   8 +-
 10 files changed, 247 insertions(+), 155 deletions(-)

diff --git a/docs/SystemDesign/1-TsFile/3-Write.md b/docs/SystemDesign/1-TsFile/3-Write.md
index cf62c0b..5a28f2d 100644
--- a/docs/SystemDesign/1-TsFile/3-Write.md
+++ b/docs/SystemDesign/1-TsFile/3-Write.md
@@ -57,7 +57,7 @@ When the data in the memory reaches a certain threshold, the persistence operati
 
 After the persistence is complete, the corresponding metadata information is cached in memory for querying and generating the metadata at the end of the file.
 
-## 3、关闭文件
+## 3、File Close
 
 - TsFileWriter.close()
 
diff --git a/docs/SystemDesign/5-DataQuery/1-DataQuery.md b/docs/SystemDesign/5-DataQuery/1-DataQuery.md
index dae36a8..cf8b86b 100644
--- a/docs/SystemDesign/5-DataQuery/1-DataQuery.md
+++ b/docs/SystemDesign/5-DataQuery/1-DataQuery.md
@@ -35,8 +35,8 @@ In order to achieve the above kinds of queries, a basic query component for a si
 
 ## Related documents
 
-* [Basic query components](/SystemDesign/5-DataQuery/2-SeriesReader.html)
-* [Modification handle](/SystemDesign/5-DataQuery/3-ModificationHandle.html)
+* [Query fundamentals](/SystemDesign/5-DataQuery/2-QueryFundamentals.html)
+* [Basic query components](/SystemDesign/5-DataQuery/3-SeriesReader.html)
 * [Raw data query](/SystemDesign/5-DataQuery/4-RawDataQuery.html)
 * [Aggregate query](/SystemDesign/5-DataQuery/5-AggregationQuery.html)
 * [Downsampling query](/SystemDesign/5-DataQuery/6-GroupByQuery.html)
diff --git a/docs/SystemDesign/5-DataQuery/2-QueryFundamentals.md b/docs/SystemDesign/5-DataQuery/2-QueryFundamentals.md
new file mode 100644
index 0000000..de54511
--- /dev/null
+++ b/docs/SystemDesign/5-DataQuery/2-QueryFundamentals.md
@@ -0,0 +1,128 @@
+<!--
+
+    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.
+
+-->
+
+# Query Fundamentals
+
+This chapter introduces some basic concepts, terms and things to pay attention in IoTDB Query design. 
+Designers and developers who hope to start with IoTDB query design may find this guide helpful, as some concepts will be treated as common sense and not explained in detail in the following chapters. 
+
+## Sequential and un-sequential TsFiles
+
+IoTDB uses TsFile as its data storage format. Sequential and un-sequential TsFiles are generated separately in terms of their different data insert patterns.
+Basically when time series data are written in strict ascending time order, only sequential TsFiles will be formed. 
+After these sequential TsFiles are flushed onto disk, the current maximum timestamp of these sequential data will be recorded, and all the timeseries data with timestamps less than this maximum timestamp will be kept in un-sequential files.
+
+IoTDB stores sequential and un-sequential TsFiles separately under `data/sequence` and `data/unsequence` directory. These files can be uniformly accessed via `getQueryDataSource()` in `QueryResourceManager.java`, by giving a full path of the timeseries.
+
+It should be noted that, in the following query documents, we tend to use `seq file` to represent Sequential files for short. For un-sequential ones we use `unseq file`. Sometimes `unordered file` or `out-of-order file` are also used as aliases of un-sequential files.
+
+## General query process
+
+The multi-level structure of TsFile is introduced in [1-TsFile](/#/SystemDesign/progress/chap1/sec1). 
+For each timeseries, we always follow the query routine across 5 levels: TsFileResource -> TimeseriesMetadata -> ChunkMetadata -> IPageReader -> BatchData
+
+The file access utility methods are in `org.apache.iotdb.db.utils.FileLoaderUtils`
+
+* `loadTimeSeriesMetadata()` reads the TimeseriesMetadata of a timeseries in a TsFileResource. If a time filter is set for this method, only those TimeseriesMetadata that satisfies this filter will be returned. `loadTimeSeriesMetadata()` returns null otherwise.
+* `loadChunkMetadataList()` can load a ChunkMetadata list for a TimeseriesMetadata。
+* `loadPageReaderList()` loads a page list contained in chunkMetadata,and can be accessed with `PageReader`。
+
+The implementation of the above methods must consider two cases of reading: 
+1. Reading memory data
+2. Reading disk data.
+
+Memory data reading means to read data cached in "Memtable" which is not yet flushed into disk storage.
+In `loadTimeSeriesMetadata()`, it obtains an unsealed TimeseriesMetadata using `TsFileResource.getTimeSeriesMetadata()`.
+We call it unsealed because users may be still writing data into this Timeseries, and it will remain unsealed until IoTDB flushes it into disk.
+
+`DiskChunkMetadataLoader` and `MemChunkMetadataLoader` provide access to read disk and memory chunk metadata.
+
+It is almost the same in `loadPageReaderList()` to read Page data. 
+`MemChunkLoader` and `DiskChunkLoader` support for memory page loading and disk page loading. 
+
+## Data orderings in TsFiles
+
+Timeseries data in seq files is in "overall" ascending order. Specifically, all the ChunkMetadata in a timeseries stored in seq files are in the right order.
+Therefore, if we have `ChunkMetadata1` and `ChunkMetadata2` kept in a seq file, it is guaranteed that 
+```
+chunkMetadata1.endtime <= chunkMetadata2.startTime.
+```
+
+While it is different that the ChunkMetadatas are stored unordered in unseq files. Some chunks might be positioned in the right order but most of them are overlapped with each other. There might be overlapping between seq file chunks and unseq file chunks as well.
+
+The page data in a single chunk is always sequential, no matter it is stored in seq files or unseq files. 
+That means, two orderings within pages are guaranteed:
+* Timestamps of data in a single page are in ascending order.
+* Different page timestamps are in ascending order. e.g. Page1.maxTime <= Page2.minTime.
+
+This certain ordering will be fully utilized to accelerate query process within our design.
+
+## Modification Handling
+
+Data deletion in IoTDB records a series of mods file for disk data. The data is not really deleted, so we need to consider the existence of modifications in query.
+
+### Related class
+
+Modification file: org.apache.iotdb.db.engine.modification.ModificationFile
+
+Deletion operation: org.apache.iotdb.db.engine.modification.Modification
+
+### Query with Modifications
+
+For any TsFile data units, their metadata structures including TimeseriesMetadata, ChunkMetadata and PageHeader use a `modified` flag to indicate whether this data unit is modified or not.
+Upon setting this `modified` flag to "true", the integrity of this data unit is supposed to be damaged and some statistics turns invalid. 
+
+
+![](https://user-images.githubusercontent.com/7240743/78339324-deca5d80-75c6-11ea-8fa8-dbd94232b756.png)
+
+Modifications affects timeseries reading process in the 5 levels mentioned before:
+* TsFileResource -> TimeseriesMetadata
+
+```
+// Set the statistics in TimeseriesMetadata unusable if the timeseries contains deletion operations 
+FileLoaderUtils.loadTimeseriesMetadata()
+```
+
+* TimeseriesMetadata -> List\<ChunkMetadata\>
+
+```
+// For each ChunkMetadata, find the largest timestamp in all deletion operations whose version is larger than it. Set deleted time to ChunkMetadata. 
+// set the statistics in ChunkMetadata unusable if it is affected by deletion
+FileLoaderUtils.loadChunkMetadataList()
+```
+
+E.g., the got ChunkMetadatas are:
+
+![](https://user-images.githubusercontent.com/7240743/78339335-e427a800-75c6-11ea-815f-16dc5b6ebfa3.png)
+
+* ChunkMetadata -> List\<IPageReader\>
+
+```
+// Skip the fully deleted page, set deleteAt into PageReader,Set the page statistics unusable if it is affected by deletion
+FileLoaderUtils.loadPageReaderList()
+```
+
+* IPageReader -> BatchData
+
+```
+// For disk page, skip the data points that be deleted and filtered out. For memory data, skip data points be filtered out.
+IPageReader.getAllSatisfiedPageData()
+```
\ No newline at end of file
diff --git a/docs/SystemDesign/5-DataQuery/3-ModificationHandle.md b/docs/SystemDesign/5-DataQuery/3-ModificationHandle.md
deleted file mode 100644
index 2871332..0000000
--- a/docs/SystemDesign/5-DataQuery/3-ModificationHandle.md
+++ /dev/null
@@ -1,73 +0,0 @@
-<!--
-
-    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.
-
--->
-
-# Modification handling in query
-
-Background:
-
-Data deletion only record a mods file for disk data, the data is not really deleted. Therefore, we need to consider the modifications in query.
-
-If a TsFile is influenced by deletion, a deletion operation will be recorded in its mods file. The log contains 3 parts: path, deleted time, version
-
-## Related class
-
-mods file: org.apache.iotdb.db.engine.modification.ModificationFile
-
-deletion operation: org.apache.iotdb.db.engine.modification.Modification
-
-## Query process
-
-![](https://user-images.githubusercontent.com/7240743/78339324-deca5d80-75c6-11ea-8fa8-dbd94232b756.png)
-
-Each timeseries is treated independently in query process. For each timeseries, there are 5 levels: TsFileResource -> TimeseriesMetadata -> ChunkMetadata -> IPageReader -> BatchData
-
-* TsFileResource -> TimeseriesMetadata
-
-```
-// Set the statistics in TimeseriesMetadata unusable if the timeseries contains deletion operations 
-FileLoaderUtils.loadTimeseriesMetadata()
-```
-
-* TimeseriesMetadata -> List\<ChunkMetadata\>
-
-```
-// For each ChunkMetadata, find the largest timestamp in all deletion operations whose version is larger than it. Set deleted time to ChunkMetadata. 
-// set the statistics in ChunkMetadata is unusable if it is affected by deletion
-FileLoaderUtils.loadChunkMetadataList()
-```
-
-E.g., the got ChunkMetadatas are:
-
-![](https://user-images.githubusercontent.com/7240743/78339335-e427a800-75c6-11ea-815f-16dc5b6ebfa3.png)
-
-* ChunkMetadata -> List\<IPageReader\>
-
-```
-// Skip the fully deleted page, set deleteAt into PageReader,Set the page statistics unusalbe if it is affected by deletion
-FileLoaderUtils.loadPageReaderList()
-```
-
-* IPageReader -> BatchData
-
-```
-// For disk page, skip the data points that be deleted and filterd out. For memory data, skip data points be filtered out.
-IPageReader.getAllSatisfiedPageData()
-```
\ No newline at end of file
diff --git a/docs/SystemDesign/5-DataQuery/2-SeriesReader.md b/docs/SystemDesign/5-DataQuery/3-SeriesReader.md
similarity index 100%
rename from docs/SystemDesign/5-DataQuery/2-SeriesReader.md
rename to docs/SystemDesign/5-DataQuery/3-SeriesReader.md
diff --git a/docs/zh/SystemDesign/5-DataQuery/1-DataQuery.md b/docs/zh/SystemDesign/5-DataQuery/1-DataQuery.md
index c2dda8a..66ae84d 100644
--- a/docs/zh/SystemDesign/5-DataQuery/1-DataQuery.md
+++ b/docs/zh/SystemDesign/5-DataQuery/1-DataQuery.md
@@ -35,8 +35,8 @@
 
 ## 相关文档
 
-* [基础查询组件](/zh/SystemDesign/5-DataQuery/2-SeriesReader.html)
-* [查询中的数据删改处理](/zh/SystemDesign/5-DataQuery/3-ModificationHandle.html)
+* [查询基础介绍](/zh/SystemDesign/5-DataQuery/2-QueryFundamentals.html)
+* [基础查询组件](/zh/SystemDesign/5-DataQuery/3-SeriesReader.html)
 * [原始数据查询](/zh/SystemDesign/5-DataQuery/4-RawDataQuery.html)
 * [聚合查询](/zh/SystemDesign/5-DataQuery/5-AggregationQuery.html)
 * [降采样查询](/zh/SystemDesign/5-DataQuery/6-GroupByQuery.html)
diff --git a/docs/zh/SystemDesign/5-DataQuery/2-QueryFundamentals.md b/docs/zh/SystemDesign/5-DataQuery/2-QueryFundamentals.md
new file mode 100644
index 0000000..f3421b3
--- /dev/null
+++ b/docs/zh/SystemDesign/5-DataQuery/2-QueryFundamentals.md
@@ -0,0 +1,110 @@
+<!--
+
+    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.
+
+-->
+
+# 查询基础介绍
+
+## 顺序和乱序tsFile文件
+
+在对某一个设备插入数据的过程中,由于插入数据的时间戳的特点会产生顺序和乱序的tsFile文件。如果我们按照时间戳递增的顺序插入数据,那么只会产生顺序文件。顺序数据被写入到磁盘后,一旦新写入的数据时间戳在顺序文件的最大时间戳之前则会产生乱序文件。
+
+IoTDB会将顺序和乱序文件分开存储在data/sequence和data/unsequence文件目录下。在查询过程中也会对顺序和乱序文件中的数据分别进行处理,我们总会使用`QueryResourceManager.java`中的`getQueryDataSource()`方法通过时间序列的全路径得到存储该时间序列的顺序和乱序文件。
+
+
+## 读取TsFile的一般流程
+
+TsFile 各级结构在前面的[1-TsFile](/#/SystemDesign/progress/chap1/sec1)文档中已有介绍,读取一个时间序列的过程需要按照层级各级展开TsFileResource -> TimeseriesMetadata -> ChunkMetadata -> IPageReader -> BatchData。
+
+文件读取的功能方法在
+`org.apache.iotdb.db.utils.FileLoaderUtils`
+
+* `loadTimeSeriesMetadata()`用来读取一个TsFileResource对应于某一个时间序列的 TimeseriesMetadata,该方法同时接受一个时间戳的Filter条件来保证该方法返回满足条件的 TimeseriesMetadata,若没有满足条件的 TimeseriesMetadata则返回null。
+* `loadChunkMetadataList()`得到这个timeseries所包含的所有ChunkMetadata列表。
+* `loadPageReaderList()`可以用来读取一个 ChunkMetadata 对应的 Chunk 所包含的所有page列表,用PageReader来进行访问。
+
+以上在对于时间序列数据的各种读取方法中总会涉及到读取内存和磁盘数据两种情况。
+
+读取内存数据是指读取存在于 Memtable 中但尚未被写入磁盘的数据,例如`loadTimeSeriesMetadata()`中使用`TsFileResource.getTimeSeriesMetadata()`得到一个未被封口的 TimeseriesMetadata。一旦这个 TimeseriesMetadata被刷新到磁盘中之后,我们将只能通过访问磁盘读取到其中的数据。磁盘和内存读取metadata的相关类为 DiskChunkMetadataLoader 和 MemChunkMetadataLoader。
+
+`loadPageReaderList()`读取page数据也是一样,分别通过两个辅助类 MemChunkLoader 和 DiskChunkLoader 进行处理。
+
+
+
+## 顺序和乱序文件的数据特点
+
+对于顺序和乱序文件的数据,其数据在文件中的分部特征有所不同。
+顺序文件的 TimeseriesMetadata 中所包含的 ChunkMetadata 也是有序的,也就是说如果按照chunkMetadata1, chunkMetadata2的顺序存储,那么将会保证chunkMetadata1.endtime <= chunkMetadata2.startTime。
+
+乱序文件的 TimeseriesMetadata 中所包含的 ChunkMetadata 是无序的,乱序文件中多个 Chunk 所覆盖的数据可能存在重叠,同时也可能与顺序文件中的 Chunk 数据存在重叠。
+
+每个 Chunk 结构内部所包含的 Page 数据总是有序的,不管是从属于顺序文件还是乱序文件。也就是说前一个 Page 的最大时间戳不小于后一个的最小时间戳。因此在查询过程中可以充分利用这种有序性,通过统计信息对 Page 数据进行提前筛选。
+
+
+
+## 查询中的数据修改处理
+
+IoTDB的数据删除操作对磁盘数据只记录了 mods 文件,并未真正执行删除逻辑,因此查询时需要考虑数据删除的逻辑。
+
+如果一个文件中有数据被删除了,将删除操作记录到 mods 文件中。记录三列:删除的时间序列,删除范围的最大时间点,删除操作对应的版本。
+
+### 相关类
+Modification文件: org.apache.iotdb.db.engine.modification.ModificationFile
+
+删除操作: org.apache.iotdb.db.engine.modification.Modification
+
+### 查询流程处理Modification
+
+对于任意的 TimeseriesMetadata,ChunkMetadata和PageHeader都有相应的modified标记,表示当前的数据块是否存在更改。由于数据删除都是从一个时间节点删除该时间前面的数据,因此如果存在数据删除会导致数据块统计信息中的startTime失效。因此在使用统计信息中的startTime之前必须检查数据块是否包含modification。对于 TimeseriesMetadata,如果删除时间点等于endTime也会导致统计信息中的endTime失效。
+
+
+![](https://user-images.githubusercontent.com/7240743/78339324-deca5d80-75c6-11ea-8fa8-dbd94232b756.png)
+
+如上图所示,数据修改会对前面提到的TsFile层级数据读取产生影响
+* TsFileResource -> TimeseriesMetadata
+
+```
+// 只要这个时间序列有对应的删除操作,就标记 TimeseriesMetadata 中的统计信息不可用
+FileLoaderUtils.loadTimeseriesMetadata()
+```
+
+* TimeseriesMetadata -> List\<ChunkMetadata\>
+
+```
+// 对于每个 ChunkMetadata,找到比其 version 大的所有删除操作中最大时间戳, 设置到  ChunkMetadata 的 deleteAt 中,并标记 统计信息不可用
+FileLoaderUtils.loadChunkMetadataList()
+```
+
+对于以上示例,读取到的 ChunkMetadataList 为
+
+![](https://user-images.githubusercontent.com/7240743/78339335-e427a800-75c6-11ea-815f-16dc5b6ebfa3.png)
+
+* ChunkMetadata -> List\<IPageReader\>
+
+```
+// 跳过被完全删除的 Page,将 deleteAt 设置到 PageReader 里,将数据被部分删除的 page 标记统计信息不可用
+FileLoaderUtils.loadPageReaderList()
+```
+
+* IPageReader -> BatchData
+
+```
+// 对于磁盘数据,跳过被删除的和过滤掉的,对于内存数据,跳过被过滤掉的
+IPageReader.getAllSatisfiedPageData()
+```
\ No newline at end of file
diff --git a/docs/zh/SystemDesign/5-DataQuery/3-ModificationHandle.md b/docs/zh/SystemDesign/5-DataQuery/3-ModificationHandle.md
deleted file mode 100644
index d3357ed..0000000
--- a/docs/zh/SystemDesign/5-DataQuery/3-ModificationHandle.md
+++ /dev/null
@@ -1,73 +0,0 @@
-<!--
-
-    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.
-
--->
-
-# 查询中的数据修改处理
-
-背景介绍:
-
-数据删除操作对磁盘数据只记录了 mods 文件,并未真正执行删除逻辑,因此查询时需要考虑数据删除的逻辑。
-
-如果一个文件中有数据被删除了,将删除操作记录到 mods 文件中。记录三列:删除的时间序列,删除范围的最大时间点,删除操作对应的版本。
-
-## 相关类
-
-mods 文件: org.apache.iotdb.db.engine.modification.ModificationFile
-
-删除操作: org.apache.iotdb.db.engine.modification.Modification
-
-## 查询流程
-
-![](https://user-images.githubusercontent.com/7240743/78339324-deca5d80-75c6-11ea-8fa8-dbd94232b756.png)
-
-查询时每个时间序列会单独处理。针对一个时间序列,由大到小有 5 个层次:TsFileResource -> TimeseriesMetadata -> ChunkMetadata -> IPageReader -> BatchData
-
-* TsFileResource -> TimeseriesMetadata
-
-```
-// 只要这个时间序列有对应的删除操作,就标记 TimeseriesMetadata 中的统计信息不可用
-FileLoaderUtils.loadTimeseriesMetadata()
-```
-
-* TimeseriesMetadata -> List\<ChunkMetadata\>
-
-```
-// 对于每个 ChunkMetadata,找到比其 version 大的所有删除操作中最大时间戳, 设置到  ChunkMetadata 的 deleteAt 中,并标记 统计信息不可用
-FileLoaderUtils.loadChunkMetadataList()
-```
-
-对于以上示例,读取到的 ChunkMetadataList 为
-
-![](https://user-images.githubusercontent.com/7240743/78339335-e427a800-75c6-11ea-815f-16dc5b6ebfa3.png)
-
-* ChunkMetadata -> List\<IPageReader\>
-
-```
-// 跳过被完全删除的 Page,将 deleteAt 设置到 PageReader 里,将数据被部分删除的 page 标记统计信息不可用
-FileLoaderUtils.loadPageReaderList()
-```
-
-* IPageReader -> BatchData
-
-```
-// 对于磁盘数据,跳过被删除的和过滤掉的,对于内存数据,跳过被过滤掉的
-IPageReader.getAllSatisfiedPageData()
-```
-
diff --git a/docs/zh/SystemDesign/5-DataQuery/2-SeriesReader.md b/docs/zh/SystemDesign/5-DataQuery/3-SeriesReader.md
similarity index 100%
rename from docs/zh/SystemDesign/5-DataQuery/2-SeriesReader.md
rename to docs/zh/SystemDesign/5-DataQuery/3-SeriesReader.md
diff --git a/site/src/main/.vuepress/config.js b/site/src/main/.vuepress/config.js
index 3e58525..a2b0eb4 100644
--- a/site/src/main/.vuepress/config.js
+++ b/site/src/main/.vuepress/config.js
@@ -438,8 +438,8 @@ var config = {
 						title: '5-DataQuery',
 						children: [
 							['5-DataQuery/1-DataQuery','DataQuery'],
-							['5-DataQuery/2-SeriesReader','SeriesReader'],
-							['5-DataQuery/3-ModificationHandle','ModificationHandle'],
+							['5-DataQuery/2-QueryFundamentals','QueryFundamentals'],
+							['5-DataQuery/3-SeriesReader','SeriesReader'],
 							['5-DataQuery/4-RawDataQuery','RawDataQuery'],
 							['5-DataQuery/5-AggregationQuery','AggregationQuery'],
 							['5-DataQuery/6-GroupByQuery','GroupByQuery'],
@@ -853,8 +853,8 @@ var config = {
 						title: '5-数据查询',
 						children: [
 							['5-DataQuery/1-DataQuery','数据查询'],
-							['5-DataQuery/2-SeriesReader','查询基础组件'],
-							['5-DataQuery/3-ModificationHandle','查询中的数据修改处理'],
+							['5-DataQuery/2-QueryFundamentals','查询基础介绍'],
+							['5-DataQuery/3-SeriesReader','查询基础组件'],
 							['5-DataQuery/4-RawDataQuery','原始数据查询'],
 							['5-DataQuery/5-AggregationQuery','聚合查询'],
 							['5-DataQuery/6-GroupByQuery','降采样查询'],