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/14 01:19:35 UTC

[incubator-iotdb] branch master updated: [IOTDB-456] GroupByFill (#1039)

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 537230d  [IOTDB-456] GroupByFill (#1039)
537230d is described below

commit 537230d2c3a307bd1692b00a8cbfe5b81c61ed92
Author: Jackie Tien <Ja...@foxmail.com>
AuthorDate: Tue Apr 14 09:19:24 2020 +0800

    [IOTDB-456] GroupByFill (#1039)
    
    * add previousUntilLast
---
 docs/SystemDesign/5-DataQuery/1-DataQuery.md       |   4 +-
 .../5-DataQuery/10-GroupByFillQuery.md             | 177 ++++++++
 .../2-DML Data Manipulation Language.md            |  35 +-
 .../5-Operation Manual/4-SQL Reference.md          |  24 ++
 docs/zh/SystemDesign/5-DataQuery/1-DataQuery.md    |   2 +
 .../5-DataQuery/10-GroupByFillQuery.md             | 176 ++++++++
 .../5-Operation Manual/4-SQL Reference.md          |  24 ++
 .../org/apache/iotdb/db/qp/strategy/SqlBase.g4     |  22 +-
 .../apache/iotdb/db/qp/constant/SQLConstant.java   |   2 +
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |  16 +-
 .../org/apache/iotdb/db/qp/logical/Operator.java   |   2 +-
 .../iotdb/db/qp/logical/crud/SelectOperator.java   |   5 +-
 .../iotdb/db/qp/physical/crud/GroupByFillPlan.java |  43 ++
 .../iotdb/db/qp/strategy/LogicalGenerator.java     | 197 +++------
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |  16 +-
 .../dataset/groupby/GroupByEngineDataSet.java      |  10 +
 .../query/dataset/groupby/GroupByFillDataSet.java  | 118 ++++++
 .../iotdb/db/query/executor/FillQueryExecutor.java |   2 -
 .../iotdb/db/query/executor/IQueryRouter.java      |  16 +-
 .../iotdb/db/query/executor/LastQueryExecutor.java |  16 +-
 .../iotdb/db/query/executor/QueryRouter.java       |  10 +
 .../apache/iotdb/db/query/fill/PreviousFill.java   |  42 +-
 .../org/apache/iotdb/db/service/TSServiceImpl.java |   1 +
 .../iotdb/db/integration/IoTDBGroupByFillIT.java   | 465 +++++++++++++++++++++
 .../apache/iotdb/db/integration/IoTDBLastIT.java   |  14 +-
 .../apache/iotdb/db/qp/plan/PhysicalPlanTest.java  | 177 +++++++-
 site/src/main/.vuepress/config.js                  |   6 +-
 27 files changed, 1418 insertions(+), 204 deletions(-)

diff --git a/docs/SystemDesign/5-DataQuery/1-DataQuery.md b/docs/SystemDesign/5-DataQuery/1-DataQuery.md
index d9a2f6c..dae36a8 100644
--- a/docs/SystemDesign/5-DataQuery/1-DataQuery.md
+++ b/docs/SystemDesign/5-DataQuery/1-DataQuery.md
@@ -29,6 +29,7 @@ There are several types of data queries
 * Single point supplementary null query
 * Latest data query
 * Align by device query
+* Group by fill query
 
 In order to achieve the above kinds of queries, a basic query component for a single time series is designed in the IoTDB query engine, and on this basis, various query functions are implemented.
 
@@ -41,4 +42,5 @@ In order to achieve the above kinds of queries, a basic query component for a si
 * [Downsampling query](/SystemDesign/5-DataQuery/6-GroupByQuery.html)
 * [Recent timestamp query](/SystemDesign/5-DataQuery/7-LastQuery.html)
 * [Align by device query](/SystemDesign/5-DataQuery/8-AlignByDeviceQuery.html)
-* [Fill function](/SystemDesign/5-DataQuery/9-FillFunction.html)
\ No newline at end of file
+* [Fill function](/SystemDesign/5-DataQuery/9-FillFunction.html)
+* [Group by fill query](/SystemDesign/5-DataQuery/10-GroupByFillQuery.html)
diff --git a/docs/SystemDesign/5-DataQuery/10-GroupByFillQuery.md b/docs/SystemDesign/5-DataQuery/10-GroupByFillQuery.md
new file mode 100644
index 0000000..7f8e0b5
--- /dev/null
+++ b/docs/SystemDesign/5-DataQuery/10-GroupByFillQuery.md
@@ -0,0 +1,177 @@
+<!--
+
+    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.
+
+-->
+
+# Group by fill
+
+The main logic of GroupByFill query is in `GroupByFillDataSet`
+
+* `org.apache.iotdb.db.query.dataset.groupby.GroupByFillDataSet`
+
+GroupByFill is used to fill the null value of the group by result.
+
+Note that:
+* In group by fill, sliding step is not supported in group by clause
+* Now, only last_value aggregation function is supported in group by fill.
+* Linear fill is not supported in group by fill.
+
+## Difference between PREVIOUSUNTILLAST fill and PREVIOUS fill
+
+PREVIOUS will fill every interval generated from group by if possible, However, PREVIOUSUNTILLAST will just fill until the last time of the specified time series and the interval after the last time won't be filled and will be null.
+
+Here is an example:
+
+Timestamp of point A is 1, point B is 5, point C is 20, point D is 30, point N is 8 and point M is 38.
+
+Raw Data is like: 
+<img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/16079446/78784824-9f41ae00-79d8-11ea-9920-0825e081cae0.png">
+
+`select temperature FROM root.ln.wf01.wt01 where time >= 1 and time <= 38`
+
+| Time   | root.ln.wf01.wt01.temperature  |
+| ------ | ------------------------------ |
+| 1      | 21                             |
+| 3      | 23                             |
+| 5      | 25                             |
+| 20     | 26                             |
+| 27     | 29                             |
+| 28     | 30                             |
+| 30     | 40                             |
+
+
+When we use Previous fill, even though the data between D and M is from the future, we should also use value of point D to fill them.
+
+`SELECT last_value(temperature) as last_temperature FROM root.ln.wf01.wt01 GROUP BY([8, 39), 5m) FILL (int32[previous])`
+
+| Time   | last_temperature |
+| ------ | ---------------- |
+| 8      | 25               |
+| 13     | 25               |
+| 18     | 26               |
+| 23     | 29               |
+| 28     | 40               |
+| 33     | 40               |
+| 38     | 40               |
+
+However, When we use NONLASTPREVIOUS fill, because data between D and M is from the future, we won't fill them and still return null.
+
+`SELECT last_value(temperature) as last_temperature FROM root.ln.wf01.wt01 GROUP BY([8, 39), 5m) FILL (int32[PREVIOUSUNTILLAST])`
+
+| Time   | last_temperature |
+| ------ | ---------------- |
+| 8      | 25               |
+| 13     | 25               |
+| 18     | 26               |
+| 23     | 29               |
+| 28     | 40               |
+| 33     | null             |
+| 38     | null             |
+
+## Core query logic
+
+We maintain two primary variable in `GroupByFillDataSet`
+
+```
+// the first value for each time series
+private Object[] previousValue;
+// last timestamp for each time series
+private long[] lastTimeArray;
+```
+### `previousValue`
+
+`previousValue` maintain the previous value before current time interval for each time series and we initialize it by calling the `initPreviousParis` method in the constructor of `GroupByFillDataSet`.
+
+```
+  private void initPreviousParis(QueryContext context, GroupByFillPlan groupByFillPlan)
+          throws StorageEngineException, IOException, QueryProcessException {
+    previousValue = new Object[paths.size()];
+    for (int i = 0; i < paths.size(); i++) {
+      Path path = paths.get(i);
+      TSDataType dataType = dataTypes.get(i);
+      IFill fill = new PreviousFill(dataType, groupByEngineDataSet.getStartTime(), -1L);
+      fill.constructReaders(path, groupByFillPlan.getAllMeasurementsInDevice(path.getDevice()), context);
+
+      TimeValuePair timeValuePair = fill.getFillResult();
+      if (timeValuePair == null || timeValuePair.getValue() == null) {
+        previousValue[i] = null;
+      } else {
+        previousValue[i] = timeValuePair.getValue().getValue();
+      }
+    }
+  }
+```
+
+`initPreviousParis` construct a Single point supplementary null query for each time series and the parameter `queryTime` is set to the start time of group by query and the parameter `beforeRange` is set to -1.
+
+### `lastTimeArray`
+
+`lastTimeArray` maintain the last timestamp for each time series and is used in PREVIOUSUNTILLAST fill way. initialize it by calling the `initLastTimeArray` method in the constructor of `GroupByFillDataSet`.
+
+```
+  private void initLastTimeArray(QueryContext context)
+      throws IOException, StorageEngineException, QueryProcessException {
+    lastTimeArray = new long[paths.size()];
+    Arrays.fill(lastTimeArray, Long.MAX_VALUE);
+    for (int i = 0; i < paths.size(); i++) {
+      TimeValuePair lastTimeValuePair =
+          LastQueryExecutor.calculateLastPairForOneSeries(paths.get(i), dataTypes.get(i), context);
+      if (lastTimeValuePair.getValue() != null) {
+        lastTimeArray[i] = lastTimeValuePair.getTimestamp();
+      }
+    }
+  }
+```
+
+`initPreviousParis` construct a last query for each time series.
+
+### The process of filling
+
+The logic of filling is in the `nextWithoutConstraint` method:
+
+```
+protected RowRecord nextWithoutConstraint() throws IOException {
+
+    // get group by result without filling through groupByEngineDataSet
+    RowRecord rowRecord = groupByEngineDataSet.nextWithoutConstraint();
+
+    // judge whether each time series is needed to be filled
+    for (int i = 0; i < paths.size(); i++) {
+      Field field = rowRecord.getFields().get(i);
+      // current group by result is null
+      if (field.getDataType() == null) {
+        // the previous value is not null and (fill type is not previous until last or now time is before last time)
+        if (previousValue[i] != null
+            && (!((PreviousFill) fillTypes.get(dataTypes.get(i))).isUntilLast()
+            || rowRecord.getTimestamp() <= lastTimeArray[i])) {
+          rowRecord.getFields().set(i, Field.getField(previousValue[i], dataTypes.get(i)));
+        }
+      } else {
+        // current group by result is not null,no need to fill
+        // use now value update previous value
+        previousValue[i] = field.getObjectValue(field.getDataType());
+      }
+    }
+    return rowRecord;
+  }
+```
+
+
+
+
diff --git a/docs/UserGuide/5-Operation Manual/2-DML Data Manipulation Language.md b/docs/UserGuide/5-Operation Manual/2-DML Data Manipulation Language.md
index 5f15354..e8fe509 100644
--- a/docs/UserGuide/5-Operation Manual/2-DML Data Manipulation Language.md	
+++ b/docs/UserGuide/5-Operation Manual/2-DML Data Manipulation Language.md	
@@ -240,6 +240,33 @@ Then the system will use the time and value filtering condition in the WHERE cla
 
 <center><img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/16079446/69116088-001e2780-0ac6-11ea-9a01-dc45271d1dad.png"></center>
 
+#### Down-Frequency Aggregate Query with Fill Clause
+
+In group by fill, sliding step is not supported in group by clause
+
+Now, only last_value aggregation function is supported in group by fill.
+
+There is no limit about time in group by fill.
+
+Linear fill is not supported in group by fill.
+
+
+##### Difference Between PREVIOUSUNTILLAST And PREVIOUS
+
+* PREVIOUS will fill any null value as long as there exist value is not null before it.
+* PREVIOUSUNTILLAST won't fill the result whose time is after the last time of that time series.
+
+The SQL statement is:
+
+```
+SELECT last_value(temperature) as last_temperature FROM root.ln.wf01.wt01 GROUP BY([8, 39), 5m) FILL (int32[PREVIOUSUNTILLAST])
+```
+which means:
+
+using PREVIOUSUNTILLAST Fill way to fill the origin down-frequency aggregate query result.
+
+
+
 The path after SELECT in GROUP BY statement must be aggregate function, otherwise the system will give the corresponding error prompt, as shown below:
 
 <center><img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/16079446/69116099-0b715300-0ac6-11ea-8074-84e04797b8c7.png"></center>
@@ -390,10 +417,10 @@ When the fill method is not specified, each data type bears its own default fill
 |Data Type|Default Fill Methods and Parameters|
 |:---|:---|
 |boolean|previous, 600000|
-|int32|linear, 600000, 600000|
-|int64|linear, 600000, 600000|
-|float|linear, 600000, 600000|
-|double|linear, 600000, 600000|
+|int32|previous, 600000|
+|int64|previous, 600000|
+|float|previous, 600000|
+|double|previous, 600000|
 |text|previous, 600000|
 </center>
 
diff --git a/docs/UserGuide/5-Operation Manual/4-SQL Reference.md b/docs/UserGuide/5-Operation Manual/4-SQL Reference.md
index 3f60cd0..627892a 100644
--- a/docs/UserGuide/5-Operation Manual/4-SQL Reference.md	
+++ b/docs/UserGuide/5-Operation Manual/4-SQL Reference.md	
@@ -314,6 +314,30 @@ Note: the statement needs to satisfy this constraint: <PrefixPath>(FromClause) +
 Note: Integer in <TimeUnit> needs to be greater than 0
 ```
 
+* Group By Fill Statement
+
+```
+SELECT <SelectClause> FROM <FromClause> WHERE  <WhereClause> GROUP BY <GroupByClause> (FILL <GROUPBYFillClause>)?
+GroupByClause : LPAREN <TimeInterval> COMMA <TimeUnit> RPAREN
+GROUPBYFillClause : LPAREN <TypeClause> RPAREN
+TypeClause : <AllClause> | <Int32Clause> | <Int64Clause> | <FloatClause> | <DoubleClause> | <BoolClause> | <TextClause> 
+AllClause: ALL LBRACKET (<PreviousUntilLastClause> | <PreviousClause>)  RBRACKET
+Int32Clause: INT32 LBRACKET (<PreviousUntilLastClause> | <PreviousClause>)  RBRACKET
+Int64Clause: INT64 LBRACKET (<PreviousUntilLastClause> | <PreviousClause>)  RBRACKET
+FloatClause: FLOAT LBRACKET (<PreviousUntilLastClause> | <PreviousClause>)  RBRACKET
+DoubleClause: DOUBLE LBRACKET (<PreviousUntilLastClause> | <PreviousClause>)  RBRACKET
+BoolClause: BOOLEAN LBRACKET (<PreviousUntilLastClause> | <PreviousClause>)  RBRACKET
+TextClause: TEXT LBRACKET (<PreviousUntilLastClause> | <PreviousClause>)  RBRACKET
+PreviousClause : PREVIOUS
+PreviousUntilLastClause : PREVIOUSUNTILLAST
+Eg: SELECT last_value(temperature) FROM root.ln.wf01.wt01 GROUP BY([20, 100), 5m) FILL (float[PREVIOUS])
+Eg: SELECT last_value(power) FROM root.ln.wf01.wt01 GROUP BY([20, 100), 5m) FILL (int32[PREVIOUSUNTILLAST])
+Eg: SELECT last_value(temperature), last_value(power) FROM root.ln.wf01.wt01 GROUP BY([20, 100), 5m) FILL (ALL[PREVIOUS])
+Note: In group by fill, sliding step is not supported in group by clause
+Note: Now, only last_value aggregation function is supported in group by fill.
+Note: Linear fill is not supported in group by fill.
+```
+
 * Limit Statement
 
 ```
diff --git a/docs/zh/SystemDesign/5-DataQuery/1-DataQuery.md b/docs/zh/SystemDesign/5-DataQuery/1-DataQuery.md
index 880ce68..c2dda8a 100644
--- a/docs/zh/SystemDesign/5-DataQuery/1-DataQuery.md
+++ b/docs/zh/SystemDesign/5-DataQuery/1-DataQuery.md
@@ -29,6 +29,7 @@
 * 单点补空值查询
 * 最新数据查询
 * 按设备对齐查询
+* 降采样补空值查询
 
 为了实现以上几种查询,IoTDB 查询引擎中设计了针对单个时间序列的基础查询组件,在此基础上,实现了多种查询功能。
 
@@ -42,3 +43,4 @@
 * [最近时间戳查询](/zh/SystemDesign/5-DataQuery/7-LastQuery.html)
 * [按设备对齐查询](/zh/SystemDesign/5-DataQuery/8-AlignByDeviceQuery.html)
 * [空值填充](/zh/SystemDesign/5-DataQuery/9-FillFunction.html)
+* [降采样补空值查询](/zh/SystemDesign/5-DataQuery/10-GroupByFillQuery.html)
diff --git a/docs/zh/SystemDesign/5-DataQuery/10-GroupByFillQuery.md b/docs/zh/SystemDesign/5-DataQuery/10-GroupByFillQuery.md
new file mode 100644
index 0000000..c34df6b
--- /dev/null
+++ b/docs/zh/SystemDesign/5-DataQuery/10-GroupByFillQuery.md
@@ -0,0 +1,176 @@
+<!--
+
+    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.
+
+-->
+
+# 降采样补空值查询
+
+GroupByFill 查询的主要逻辑在 `GroupByFillDataSet`
+
+* `org.apache.iotdb.db.query.dataset.groupby.GroupByFillDataSet`
+
+GroupByFill是对原降采样结果进行填充,目前仅支持使用前值填充的方式。
+
+* 在Group By Fill中,Group by子句不支持滑动步长,否则抛异常
+* Fill子句中仅能使用Previous和PREVIOUSUNTILLAST这两种插值方式,Linear不支持
+* Previous和PREVIOUSUNTILLAST对fill的时间不做限制
+* 填充只针对last_value这一聚合函数,其他的函数不支持,如果其他函数的聚合值查询结果为null,依旧为null,不进行填充
+
+## PREVIOUSUNTILLAST与PREVIOUS填充的区别
+
+Previous填充方式的语意没有变,只要前面有值,就可以拿过来填充;
+PREVIOUSUNTILLAST考虑到在某些业务场景下,所填充的值的时间不能大于该时间序列last的时间戳(从业务角度考虑,取历史数据不能取未来历史数据)
+看下面的例子,或许更容易理解
+
+A点时间戳为1,B为5,C为20,D为30,N为8,M为38
+
+
+
+原始数据为<img style="width:100%; max-width:800px; max-height:600px; margin-left:auto; margin-right:auto; display:block;" src="https://user-images.githubusercontent.com/16079446/78784824-9f41ae00-79d8-11ea-9920-0825e081cae0.png">
+
+`select temperature FROM root.ln.wf01.wt01 where time >= 1 and time <= 38`
+
+| Time   | root.ln.wf01.wt01.temperature  |
+| ------ | ------------------------------ |
+| 1      | 21                             |
+| 3      | 23                             |
+| 5      | 25                             |
+| 20     | 26                             |
+| 27     | 29                             |
+| 28     | 30                             |
+| 30     | 40                             |
+
+
+当我们使用Previous插值方式时,即使D到M这一段是未来的数据,我们也会用D点的数据进行填充
+
+`SELECT last_value(temperature) as last_temperature FROM root.ln.wf01.wt01 GROUP BY([8, 39), 5m) FILL (int32[previous])`
+
+| Time   | last_temperature |
+| ------ | ---------------- |
+| 8      | 25               |
+| 13     | 25               |
+| 18     | 26               |
+| 23     | 29               |
+| 28     | 40               |
+| 33     | 40               |
+| 38     | 40               |
+
+当我们使用NONLASTPREVIOUS插值方式时,因为D到M这一段是未来的数据,我们不会进行插值,还是返回null
+
+`SELECT last_value(temperature) as last_temperature FROM root.ln.wf01.wt01 GROUP BY([8, 39), 5m) FILL (int32[PREVIOUSUNTILLAST])`
+
+| Time   | last_temperature |
+| ------ | ---------------- |
+| 8      | 25               |
+| 13     | 25               |
+| 18     | 26               |
+| 23     | 29               |
+| 28     | 40               |
+| 33     | null             |
+| 38     | null             |
+
+## 核心查询逻辑
+
+在`GroupByFillDataSet`中维护了两个主要变量
+
+```
+// the first value for each time series
+private Object[] previousValue;
+// last timestamp for each time series
+private long[] lastTimeArray;
+```
+### `previousValue`
+
+`previousValue`这个变量维护了当前时间窗口的前一个降采样值,在`GroupByFillDataSet`构造函数中调用了`initPreviousParis`方法对其进行初始化。
+
+```
+  private void initPreviousParis(QueryContext context, GroupByFillPlan groupByFillPlan)
+          throws StorageEngineException, IOException, QueryProcessException {
+    previousValue = new Object[paths.size()];
+    for (int i = 0; i < paths.size(); i++) {
+      Path path = paths.get(i);
+      TSDataType dataType = dataTypes.get(i);
+      IFill fill = new PreviousFill(dataType, groupByEngineDataSet.getStartTime(), -1L);
+      fill.constructReaders(path, groupByFillPlan.getAllMeasurementsInDevice(path.getDevice()), context);
+
+      TimeValuePair timeValuePair = fill.getFillResult();
+      if (timeValuePair == null || timeValuePair.getValue() == null) {
+        previousValue[i] = null;
+      } else {
+        previousValue[i] = timeValuePair.getValue().getValue();
+      }
+    }
+  }
+```
+
+`initPreviousParis`方法主要为每个时间序列构造了一个单点补空值查询,`queryTime`设置为降采样时间窗口的起始值,`beforeRange`不作限制。
+
+### `lastTimeArray`
+
+`lastTimeArray`这个变量维护了每个时间序列的最近时间戳值,主要用于`PREVIOUSUNTILLAST`这一填充方式,在`GroupByFillDataSet`构造函数中调用了`initLastTimeArray`方法对其进行初始化。
+
+```
+  private void initLastTimeArray(QueryContext context)
+      throws IOException, StorageEngineException, QueryProcessException {
+    lastTimeArray = new long[paths.size()];
+    Arrays.fill(lastTimeArray, Long.MAX_VALUE);
+    for (int i = 0; i < paths.size(); i++) {
+      TimeValuePair lastTimeValuePair =
+          LastQueryExecutor.calculateLastPairForOneSeries(paths.get(i), dataTypes.get(i), context);
+      if (lastTimeValuePair.getValue() != null) {
+        lastTimeArray[i] = lastTimeValuePair.getTimestamp();
+      }
+    }
+  }
+```
+`initPreviousParis`方法主要为每个时间序列构造了一个最近时间戳 Last 查询
+
+### 填充过程
+
+填充过程在`nextWithoutConstraint`方法中完成,主要逻辑如下:
+
+```
+protected RowRecord nextWithoutConstraint() throws IOException {
+
+    // 首先通过groupByEngineDataSet,获得原始的降采样查询结果行
+    RowRecord rowRecord = groupByEngineDataSet.nextWithoutConstraint();
+
+    // 接下来对每个时间序列判断需不需要填充
+    for (int i = 0; i < paths.size(); i++) {
+      Field field = rowRecord.getFields().get(i);
+      // 当前值为null,需要进行填充
+      if (field.getDataType() == null) {
+        // 当前一个值不为null 并且 (填充方式不是PREVIOUSUNTILLAST 或者 当前时间小于改时间序列的最近时间戳)
+        if (previousValue[i] != null
+            && (!((PreviousFill) fillTypes.get(dataTypes.get(i))).isUntilLast()
+            || rowRecord.getTimestamp() <= lastTimeArray[i])) {
+          rowRecord.getFields().set(i, Field.getField(previousValue[i], dataTypes.get(i)));
+        }
+      } else {
+        // 当前值不为null,不需要填充,用当前值更新previousValue数组
+        previousValue[i] = field.getObjectValue(field.getDataType());
+      }
+    }
+    return rowRecord;
+  }
+```
+
+
+
+
diff --git a/docs/zh/UserGuide/5-Operation Manual/4-SQL Reference.md b/docs/zh/UserGuide/5-Operation Manual/4-SQL Reference.md
index 89040eb..1a55d08 100644
--- a/docs/zh/UserGuide/5-Operation Manual/4-SQL Reference.md	
+++ b/docs/zh/UserGuide/5-Operation Manual/4-SQL Reference.md	
@@ -306,6 +306,30 @@ Note: the statement needs to satisfy this constraint: <PrefixPath>(FromClause) +
 Note: Integer in <TimeUnit> needs to be greater than 0
 ```
 
+* Group By Fill语句
+
+```
+SELECT <SelectClause> FROM <FromClause> WHERE  <WhereClause> GROUP BY <GroupByClause> (FILL <GROUPBYFillClause>)?
+GroupByClause : LPAREN <TimeInterval> COMMA <TimeUnit> RPAREN
+GROUPBYFillClause : LPAREN <TypeClause> RPAREN
+TypeClause : <AllClause> | <Int32Clause> | <Int64Clause> | <FloatClause> | <DoubleClause> | <BoolClause> | <TextClause> 
+AllClause: ALL LBRACKET (<PreviousUntilLastClause> | <PreviousClause>)  RBRACKET
+Int32Clause: INT32 LBRACKET (<PreviousUntilLastClause> | <PreviousClause>)  RBRACKET
+Int64Clause: INT64 LBRACKET (<PreviousUntilLastClause> | <PreviousClause>)  RBRACKET
+FloatClause: FLOAT LBRACKET (<PreviousUntilLastClause> | <PreviousClause>)  RBRACKET
+DoubleClause: DOUBLE LBRACKET (<PreviousUntilLastClause> | <PreviousClause>)  RBRACKET
+BoolClause: BOOLEAN LBRACKET (<PreviousUntilLastClause> | <PreviousClause>)  RBRACKET
+TextClause: TEXT LBRACKET (<PreviousUntilLastClause> | <PreviousClause>)  RBRACKET
+PreviousClause : PREVIOUS
+PreviousUntilLastClause : PREVIOUSUNTILLAST
+Eg: SELECT last_value(temperature) FROM root.ln.wf01.wt01 GROUP BY([20, 100), 5m) FILL (float[PREVIOUS])
+Eg: SELECT last_value(power) FROM root.ln.wf01.wt01 GROUP BY([20, 100), 5m) FILL (int32[PREVIOUSUNTILLAST])
+Eg: SELECT last_value(temperature), last_value(power) FROM root.ln.wf01.wt01 GROUP BY([20, 100), 5m) FILL (ALL[PREVIOUS])
+Note: In group by fill, sliding step is not supported in group by clause
+Note: Now, only last_value aggregation function is supported in group by fill.
+Note: Linear fill is not supported in group by fill.
+```
+
 * Limit & SLimit 语句
 
 ```
diff --git a/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4 b/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
index 73120bb..114bc5f 100644
--- a/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
+++ b/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
@@ -153,6 +153,7 @@ fromClause
 specialClause
     : specialLimit
     | groupByClause specialLimit?
+    | groupByFillClause
     | fillClause slimitClause? alignByDeviceClauseOrDisableAlign?
     | alignByDeviceClauseOrDisableAlign?
     ;
@@ -205,9 +206,18 @@ groupByClause
       RR_BRACKET
     ;
 
+groupByFillClause
+    : GROUP BY LR_BRACKET
+      timeInterval
+      COMMA DURATION
+      RR_BRACKET
+      FILL LR_BRACKET typeClause (COMMA typeClause)* RR_BRACKET
+     ;
+
 typeClause
     : dataType LS_BRACKET linearClause RS_BRACKET
-    | dataType LS_BRACKET  previousClause RS_BRACKET
+    | dataType LS_BRACKET previousClause RS_BRACKET
+    | dataType LS_BRACKET previousUntilLastClause RS_BRACKET
     ;
 
 linearClause
@@ -218,6 +228,10 @@ previousClause
     : PREVIOUS (COMMA DURATION)?
     ;
 
+previousUntilLastClause
+    : PREVIOUSUNTILLAST (COMMA DURATION)?
+    ;
+
 indexWithClause
     : WITH indexValue (COMMA indexValue)?
     ;
@@ -307,7 +321,7 @@ nodeNameWithoutStar
     ;
 
 dataType
-    : INT32 | INT64 | FLOAT | DOUBLE | BOOLEAN | TEXT
+    : INT32 | INT64 | FLOAT | DOUBLE | BOOLEAN | TEXT | ALL
     ;
 
 dateFormat
@@ -445,6 +459,10 @@ PREVIOUS
     : P R E V I O U S
     ;
 
+PREVIOUSUNTILLAST
+    : P R E V I O U S U N T I L L A S T
+    ;
+
 METADATA
     : M E T A D A T A
     ;
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
index cda3485..4dba13c 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
@@ -65,6 +65,8 @@ public class SQLConstant {
   public static final String AVG = "avg";
   public static final String SUM = "sum";
 
+  public static final String ALL = "all";
+
   public static final int KW_AND = 1;
   public static final int KW_OR = 2;
   public static final int KW_NOT = 3;
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
index f2d45e2..0c8b028 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java
@@ -74,17 +74,7 @@ import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode;
 import org.apache.iotdb.db.qp.logical.sys.AuthorOperator;
 import org.apache.iotdb.db.qp.logical.sys.AuthorOperator.AuthorType;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
-import org.apache.iotdb.db.qp.physical.crud.AlignByDevicePlan;
-import org.apache.iotdb.db.qp.physical.crud.BatchInsertPlan;
-import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
-import org.apache.iotdb.db.qp.physical.crud.FillQueryPlan;
-import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
-import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
-import org.apache.iotdb.db.qp.physical.crud.RawDataQueryPlan;
-import org.apache.iotdb.db.qp.physical.crud.LastQueryPlan;
-import org.apache.iotdb.db.qp.physical.crud.UpdatePlan;
+import org.apache.iotdb.db.qp.physical.crud.*;
 import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
 import org.apache.iotdb.db.qp.physical.sys.CountPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
@@ -234,9 +224,13 @@ public class PlanExecutor implements IPlanExecutor {
     if (queryPlan instanceof AlignByDevicePlan) {
       queryDataSet = new AlignByDeviceDataSet((AlignByDevicePlan) queryPlan, context, queryRouter);
     } else {
+
       if (queryPlan.getPaths() == null || queryPlan.getPaths().isEmpty()) {
         // no time series are selected, return EmptyDataSet
         return new EmptyDataSet();
+      } else if (queryPlan instanceof GroupByFillPlan) {
+        GroupByFillPlan groupByFillPlan = (GroupByFillPlan) queryPlan;
+        return queryRouter.groupByFill(groupByFillPlan, context);
       } else if (queryPlan instanceof GroupByPlan) {
         GroupByPlan groupByPlan = (GroupByPlan) queryPlan;
         return queryRouter.groupBy(groupByPlan, context);
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
index 00ecc6f..47af477 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
@@ -74,6 +74,6 @@ public abstract class Operator {
     DELETE_ROLE, GRANT_ROLE_PRIVILEGE, REVOKE_ROLE_PRIVILEGE, LIST_USER, LIST_ROLE,
     LIST_USER_PRIVILEGE, LIST_ROLE_PRIVILEGE, LIST_USER_ROLES, LIST_ROLE_USERS,
     GRANT_WATERMARK_EMBEDDING, REVOKE_WATERMARK_EMBEDDING,
-    TTL, DELETE_STORAGE_GROUP, LOAD_CONFIGURATION, SHOW, LOAD_FILES, REMOVE_FILE, MOVE_FILE, LAST
+    TTL, DELETE_STORAGE_GROUP, LOAD_CONFIGURATION, SHOW, LOAD_FILES, REMOVE_FILE, MOVE_FILE, LAST, GROUP_BY_FILL
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SelectOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SelectOperator.java
index a0e566e..961f112 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SelectOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/SelectOperator.java
@@ -18,11 +18,12 @@
  */
 package org.apache.iotdb.db.qp.logical.crud;
 
-import java.util.ArrayList;
-import java.util.List;
 import org.apache.iotdb.db.qp.logical.Operator;
 import org.apache.iotdb.tsfile.read.common.Path;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * this class maintains information from select clause.
  */
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/GroupByFillPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/GroupByFillPlan.java
new file mode 100644
index 0000000..196f56a
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/GroupByFillPlan.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.qp.physical.crud;
+
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.db.query.fill.IFill;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+import java.util.Map;
+
+public class GroupByFillPlan extends GroupByPlan {
+
+  private Map<TSDataType, IFill> fillTypes;
+
+  public GroupByFillPlan() {
+    super();
+    setOperatorType(Operator.OperatorType.GROUP_BY_FILL);
+  }
+
+  public Map<TSDataType, IFill> getFillType() {
+    return fillTypes;
+  }
+
+  public void setFillType(Map<TSDataType, IFill> fillTypes) {
+    this.fillTypes = fillTypes;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
index 4d5257d..358fbbf 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
@@ -18,129 +18,16 @@
  */
 package org.apache.iotdb.db.qp.strategy;
 
-import java.io.File;
-import java.time.ZoneId;
-import java.util.ArrayList;
-import java.util.EnumMap;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
 import org.antlr.v4.runtime.tree.TerminalNode;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.runtime.SQLParserException;
 import org.apache.iotdb.db.qp.constant.DatetimeUtils;
 import org.apache.iotdb.db.qp.constant.SQLConstant;
 import org.apache.iotdb.db.qp.logical.RootOperator;
-import org.apache.iotdb.db.qp.logical.crud.BasicFunctionOperator;
-import org.apache.iotdb.db.qp.logical.crud.DeleteDataOperator;
-import org.apache.iotdb.db.qp.logical.crud.FilterOperator;
-import org.apache.iotdb.db.qp.logical.crud.FromOperator;
-import org.apache.iotdb.db.qp.logical.crud.InOperator;
-import org.apache.iotdb.db.qp.logical.crud.InsertOperator;
-import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
-import org.apache.iotdb.db.qp.logical.crud.SelectOperator;
-import org.apache.iotdb.db.qp.logical.crud.UpdateOperator;
-import org.apache.iotdb.db.qp.logical.sys.AuthorOperator;
+import org.apache.iotdb.db.qp.logical.crud.*;
+import org.apache.iotdb.db.qp.logical.sys.*;
 import org.apache.iotdb.db.qp.logical.sys.AuthorOperator.AuthorType;
-import org.apache.iotdb.db.qp.logical.sys.CountOperator;
-import org.apache.iotdb.db.qp.logical.sys.CreateTimeSeriesOperator;
-import org.apache.iotdb.db.qp.logical.sys.DataAuthOperator;
-import org.apache.iotdb.db.qp.logical.sys.DeleteStorageGroupOperator;
-import org.apache.iotdb.db.qp.logical.sys.DeleteTimeSeriesOperator;
-import org.apache.iotdb.db.qp.logical.sys.LoadConfigurationOperator;
-import org.apache.iotdb.db.qp.logical.sys.LoadDataOperator;
-import org.apache.iotdb.db.qp.logical.sys.LoadFilesOperator;
-import org.apache.iotdb.db.qp.logical.sys.MoveFileOperator;
-import org.apache.iotdb.db.qp.logical.sys.RemoveFileOperator;
-import org.apache.iotdb.db.qp.logical.sys.SetStorageGroupOperator;
-import org.apache.iotdb.db.qp.logical.sys.SetTTLOperator;
-import org.apache.iotdb.db.qp.logical.sys.ShowChildPathsOperator;
-import org.apache.iotdb.db.qp.logical.sys.ShowDevicesOperator;
-import org.apache.iotdb.db.qp.logical.sys.ShowOperator;
-import org.apache.iotdb.db.qp.logical.sys.ShowTTLOperator;
-import org.apache.iotdb.db.qp.logical.sys.ShowTimeSeriesOperator;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.AlignByDeviceClauseContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.AlterUserContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.AndExpressionContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.AttributeClausesContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.AutoCreateSchemaContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ConstantContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.CountNodesContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.CountTimeseriesContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.CreateRoleContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.CreateTimeseriesContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.CreateUserContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.DateExpressionContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.DeleteStatementContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.DeleteStorageGroupContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.DeleteTimeseriesContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.DropRoleContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.DropUserContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.FillClauseContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.FromClauseContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.FullPathContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.FunctionCallContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.FunctionElementContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.GrantRoleContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.GrantRoleToUserContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.GrantUserContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.GrantWatermarkEmbeddingContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.GroupByClauseContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.InClauseContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.InsertColumnSpecContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.InsertStatementContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.InsertValuesSpecContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.LimitClauseContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.LastClauseContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ListAllRoleOfUserContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ListAllUserOfRoleContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ListPrivilegesRoleContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ListPrivilegesUserContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ListRoleContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ListRolePrivilegesContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ListUserContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ListUserPrivilegesContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.LoadConfigurationStatementContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.LoadFilesContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.LoadStatementContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.MoveFileContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.NodeNameContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.NodeNameWithoutStarContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.OffsetClauseContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.OrExpressionContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.PredicateContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.PrefixPathContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.PrivilegesContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.PropertyContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.RemoveFileContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.RevokeRoleContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.RevokeRoleFromUserContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.RevokeUserContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.RevokeWatermarkEmbeddingContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.RootOrIdContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.SelectConstElementContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.SelectElementContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.SelectStatementContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.SetColContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.SetStorageGroupContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.SetTTLStatementContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ShowAllTTLStatementContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ShowChildPathsContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ShowDevicesContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ShowStorageGroupContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ShowTTLStatementContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ShowTimeseriesContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.ShowVersionContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.SlimitClauseContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.SoffsetClauseContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.SuffixPathContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.TimeIntervalContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.TypeClauseContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.UnsetTTLStatementContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.UpdateStatementContext;
-import org.apache.iotdb.db.qp.strategy.SqlBaseParser.WhereClauseContext;
+import org.apache.iotdb.db.qp.strategy.SqlBaseParser.*;
 import org.apache.iotdb.db.query.fill.IFill;
 import org.apache.iotdb.db.query.fill.LinearFill;
 import org.apache.iotdb.db.query.fill.PreviousFill;
@@ -152,6 +39,10 @@ import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
 import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.utils.StringContainer;
 
+import java.io.File;
+import java.time.ZoneId;
+import java.util.*;
+
 /**
  * This class is a listener and you can get an operator which is a logical plan.
  */
@@ -668,25 +559,47 @@ public class LogicalGenerator extends SqlBaseBaseListener {
   }
 
   @Override
-  public void enterGroupByClause(GroupByClauseContext ctx) {
-    super.enterGroupByClause(ctx);
+  public void enterGroupByFillClause(SqlBaseParser.GroupByFillClauseContext ctx) {
+    super.enterGroupByFillClause(ctx);
     queryOp.setGroupBy(true);
+    queryOp.setFill(true);
 
     // parse timeUnit
-    queryOp.setUnit(parseDuration(ctx.DURATION(0).getText()));
+    queryOp.setUnit(parseDuration(ctx.DURATION().getText()));
     queryOp.setSlidingStep(queryOp.getUnit());
-    // parse sliding step
-    if (ctx.DURATION().size() == 2) {
-      queryOp.setSlidingStep(parseDuration(ctx.DURATION(1).getText()));
-      if (queryOp.getSlidingStep() < queryOp.getUnit()) {
-        throw new SQLParserException(
-            "The third parameter sliding step shouldn't be smaller than the second parameter time interval.");
+
+    parseTimeInterval(ctx.timeInterval());
+
+    List<TypeClauseContext> list = ctx.typeClause();
+    Map<TSDataType, IFill> fillTypes = new EnumMap<>(TSDataType.class);
+    for (TypeClauseContext typeClause : list) {
+      // group by fill doesn't support linear fill
+      if (typeClause.linearClause() != null) {
+        throw new SQLParserException("group by fill doesn't support linear fill");
+      }
+      // all type use the same fill way
+      if (SQLConstant.ALL.equals(typeClause.dataType().getText().toLowerCase())) {
+        IFill fill;
+        if (typeClause.previousUntilLastClause() != null) {
+          fill = new PreviousFill(-1, true);
+        } else {
+          fill = new PreviousFill(-1);
+        }
+        for (TSDataType tsDataType : TSDataType.values()) {
+          fillTypes.put(tsDataType, fill.copy());
+        }
+        break;
+      } else {
+        parseTypeClause(typeClause, fillTypes);
       }
     }
+    queryOp.setFill(true);
+    queryOp.setFillTypes(fillTypes);
+  }
 
+  private void parseTimeInterval(TimeIntervalContext timeInterval) {
     long startTime;
     long endTime;
-    TimeIntervalContext timeInterval = ctx.timeInterval();
     if (timeInterval.timeValue(0).INT() != null) {
       startTime = Long.parseLong(timeInterval.timeValue(0).INT().getText());
     } else if (timeInterval.timeValue(0).dateExpression() != null) {
@@ -707,6 +620,26 @@ public class LogicalGenerator extends SqlBaseBaseListener {
   }
 
   @Override
+  public void enterGroupByClause(GroupByClauseContext ctx) {
+    super.enterGroupByClause(ctx);
+    queryOp.setGroupBy(true);
+
+    // parse timeUnit
+    queryOp.setUnit(parseDuration(ctx.DURATION(0).getText()));
+    queryOp.setSlidingStep(queryOp.getUnit());
+    // parse sliding step
+    if (ctx.DURATION().size() == 2) {
+      queryOp.setSlidingStep(parseDuration(ctx.DURATION(1).getText()));
+      if (queryOp.getSlidingStep() < queryOp.getUnit()) {
+        throw new SQLParserException(
+            "The third parameter sliding step shouldn't be smaller than the second parameter time interval.");
+      }
+    }
+
+    parseTimeInterval(ctx.timeInterval());
+  }
+
+  @Override
   public void enterFillClause(FillClauseContext ctx) {
     super.enterFillClause(ctx);
     FilterOperator filterOperator = queryOp.getFilterOperator();
@@ -730,7 +663,8 @@ public class LogicalGenerator extends SqlBaseBaseListener {
     }
 
     int defaultFillInterval = IoTDBDescriptor.getInstance().getConfig().getDefaultFillInterval();
-    if (ctx.linearClause() != null) {
+
+    if (ctx.linearClause() != null) {  // linear
       if (ctx.linearClause().DURATION(0) != null) {
         long beforeRange = parseDuration(ctx.linearClause().DURATION(0).getText());
         long afterRange = parseDuration(ctx.linearClause().DURATION(1).getText());
@@ -738,13 +672,20 @@ public class LogicalGenerator extends SqlBaseBaseListener {
       } else {
         fillTypes.put(dataType, new LinearFill(defaultFillInterval, defaultFillInterval));
       }
-    } else {
+    } else if (ctx.previousClause() != null) { // previous
       if (ctx.previousClause().DURATION() != null) {
         long preRange = parseDuration(ctx.previousClause().DURATION().getText());
         fillTypes.put(dataType, new PreviousFill(preRange));
       } else {
         fillTypes.put(dataType, new PreviousFill(defaultFillInterval));
       }
+    } else { // previous until last
+      if (ctx.previousUntilLastClause().DURATION() != null) {
+        long preRange = parseDuration(ctx.previousClause().DURATION().getText());
+        fillTypes.put(dataType, new PreviousFill(preRange, true));
+      } else {
+        fillTypes.put(dataType, new PreviousFill(defaultFillInterval, true));
+      }
     }
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
index 378af12..96d7e8c 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
@@ -178,7 +178,21 @@ public class PhysicalGenerator {
       throws QueryProcessException {
     QueryPlan queryPlan;
 
-    if (queryOperator.isGroupBy()) {
+    if (queryOperator.isGroupBy() && queryOperator.isFill()) {
+      queryPlan = new GroupByFillPlan();
+      ((GroupByFillPlan) queryPlan).setInterval(queryOperator.getUnit());
+      ((GroupByFillPlan) queryPlan).setSlidingStep(queryOperator.getSlidingStep());
+      ((GroupByFillPlan) queryPlan).setStartTime(queryOperator.getStartTime());
+      ((GroupByFillPlan) queryPlan).setEndTime(queryOperator.getEndTime());
+      ((GroupByFillPlan) queryPlan)
+              .setAggregations(queryOperator.getSelectOperator().getAggregations());
+      for (String aggregation : queryPlan.getAggregations()) {
+        if (!SQLConstant.LAST_VALUE.equals(aggregation)) {
+          throw new QueryProcessException("Group By Fill only support last_value function");
+        }
+      }
+      ((GroupByFillPlan) queryPlan).setFillType(queryOperator.getFillTypes());
+    } else if (queryOperator.isGroupBy()) {
       queryPlan = new GroupByPlan();
       ((GroupByPlan) queryPlan).setInterval(queryOperator.getUnit());
       ((GroupByPlan) queryPlan).setSlidingStep(queryOperator.getSlidingStep());
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByEngineDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByEngineDataSet.java
index 4ca7ceb..a377597 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByEngineDataSet.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByEngineDataSet.java
@@ -21,9 +21,12 @@ package org.apache.iotdb.db.query.dataset.groupby;
 import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
 import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
 import org.apache.iotdb.tsfile.utils.Pair;
 
+import java.io.IOException;
+
 public abstract class GroupByEngineDataSet extends QueryDataSet {
 
   protected long queryId;
@@ -78,6 +81,13 @@ public abstract class GroupByEngineDataSet extends QueryDataSet {
     }
   }
 
+  @Override
+  protected abstract RowRecord nextWithoutConstraint() throws IOException;
+
+  public long getStartTime() {
+    return startTime;
+  }
+
   @TestOnly
   public Pair<Long, Long> nextTimePartition() {
     hasCachedTimeInterval = false;
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByFillDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByFillDataSet.java
new file mode 100644
index 0000000..7b73eb8
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByFillDataSet.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.query.dataset.groupby;
+
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.qp.physical.crud.GroupByFillPlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.executor.LastQueryExecutor;
+import org.apache.iotdb.db.query.fill.IFill;
+import org.apache.iotdb.db.query.fill.PreviousFill;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.TimeValuePair;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+public class GroupByFillDataSet extends QueryDataSet {
+
+  private GroupByEngineDataSet groupByEngineDataSet;
+  private Map<TSDataType, IFill> fillTypes;
+  // the first value for each time series
+  private Object[] previousValue;
+  // last timestamp for each time series
+  private long[] lastTimeArray;
+
+  public GroupByFillDataSet(List<Path> paths, List<TSDataType> dataTypes,
+      GroupByEngineDataSet groupByEngineDataSet,
+      Map<TSDataType, IFill> fillTypes, QueryContext context, GroupByFillPlan groupByFillPlan)
+      throws StorageEngineException, IOException, QueryProcessException {
+    super(paths, dataTypes);
+    this.groupByEngineDataSet = groupByEngineDataSet;
+    this.fillTypes = fillTypes;
+    initPreviousParis(context, groupByFillPlan);
+    initLastTimeArray(context);
+  }
+
+  private void initPreviousParis(QueryContext context, GroupByFillPlan groupByFillPlan)
+          throws StorageEngineException, IOException, QueryProcessException {
+    previousValue = new Object[paths.size()];
+    for (int i = 0; i < paths.size(); i++) {
+      Path path = paths.get(i);
+      TSDataType dataType = dataTypes.get(i);
+      IFill fill = new PreviousFill(dataType, groupByEngineDataSet.getStartTime(), -1L);
+      fill.configureFill(path, dataType, groupByEngineDataSet.getStartTime(), groupByFillPlan.getAllMeasurementsInDevice(path.getDevice()), context);
+
+      TimeValuePair timeValuePair = fill.getFillResult();
+      if (timeValuePair == null || timeValuePair.getValue() == null) {
+        previousValue[i] = null;
+      } else {
+        previousValue[i] = timeValuePair.getValue().getValue();
+      }
+    }
+  }
+
+  private void initLastTimeArray(QueryContext context)
+      throws IOException, StorageEngineException, QueryProcessException {
+    lastTimeArray = new long[paths.size()];
+    Arrays.fill(lastTimeArray, Long.MAX_VALUE);
+    for (int i = 0; i < paths.size(); i++) {
+      TimeValuePair lastTimeValuePair =
+          LastQueryExecutor.calculateLastPairForOneSeries(paths.get(i), dataTypes.get(i), context);
+      if (lastTimeValuePair.getValue() != null) {
+        lastTimeArray[i] = lastTimeValuePair.getTimestamp();
+      }
+    }
+  }
+
+  @Override
+  protected boolean hasNextWithoutConstraint() {
+    return groupByEngineDataSet.hasNextWithoutConstraint();
+  }
+
+  @Override
+  protected RowRecord nextWithoutConstraint() throws IOException {
+    RowRecord rowRecord = groupByEngineDataSet.nextWithoutConstraint();
+
+    for (int i = 0; i < paths.size(); i++) {
+      Field field = rowRecord.getFields().get(i);
+      // current group by result is null
+      if (field == null || field.getDataType() == null) {
+        // the previous value is not null and
+        // (fill type is not previous until last or now time is before last time)
+        if (previousValue[i] != null
+            && (!((PreviousFill) fillTypes.get(dataTypes.get(i))).isUntilLast()
+            || rowRecord.getTimestamp() <= lastTimeArray[i])) {
+          rowRecord.getFields().set(i, Field.getField(previousValue[i], dataTypes.get(i)));
+        }
+      } else {
+        // use now value update previous value
+        previousValue[i] = field.getObjectValue(field.getDataType());
+      }
+    }
+    return rowRecord;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/FillQueryExecutor.java b/server/src/main/java/org/apache/iotdb/db/query/executor/FillQueryExecutor.java
index 6b823c9..ae849cf 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/executor/FillQueryExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/executor/FillQueryExecutor.java
@@ -26,7 +26,6 @@ import org.apache.iotdb.db.qp.physical.crud.FillQueryPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.dataset.SingleDataSet;
 import org.apache.iotdb.db.query.fill.IFill;
-import org.apache.iotdb.db.query.fill.LinearFill;
 import org.apache.iotdb.db.query.fill.PreviousFill;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
@@ -38,7 +37,6 @@ import javax.activation.UnsupportedDataTypeException;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 public class FillQueryExecutor {
 
diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/IQueryRouter.java b/server/src/main/java/org/apache/iotdb/db/query/executor/IQueryRouter.java
index 129f12a..9c1f7ec 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/executor/IQueryRouter.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/executor/IQueryRouter.java
@@ -19,18 +19,15 @@
 package org.apache.iotdb.db.query.executor;
 
 
-import java.io.IOException;
 import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
-import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
-import org.apache.iotdb.db.qp.physical.crud.FillQueryPlan;
-import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
-import org.apache.iotdb.db.qp.physical.crud.RawDataQueryPlan;
-import org.apache.iotdb.db.qp.physical.crud.LastQueryPlan;
+import org.apache.iotdb.db.qp.physical.crud.*;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
 import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
 
+import java.io.IOException;
+
 public interface IQueryRouter {
 
   /**
@@ -59,6 +56,13 @@ public interface IQueryRouter {
       throws StorageEngineException, QueryProcessException, IOException;
 
   /**
+   * Execute group by fill query
+   */
+  QueryDataSet groupByFill(GroupByFillPlan groupByFillPlan, QueryContext context)
+      throws QueryFilterOptimizationException, StorageEngineException,
+      QueryProcessException, IOException;
+
+  /**
    * Execute last query
    */
   QueryDataSet lastQuery(LastQueryPlan lastQueryPlan, QueryContext context)
diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/LastQueryExecutor.java b/server/src/main/java/org/apache/iotdb/db/query/executor/LastQueryExecutor.java
index 2d4d555..b3d5a08 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/executor/LastQueryExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/executor/LastQueryExecutor.java
@@ -19,6 +19,7 @@
 
 package org.apache.iotdb.db.query.executor;
 
+
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_VALUE;
 import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_TIMESERIES;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
@@ -44,7 +45,11 @@ import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
 import org.apache.iotdb.tsfile.utils.Binary;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
 import java.io.IOException;
-import java.util.*;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_TIMESERIES;
+import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_VALUE;
 
 public class LastQueryExecutor {
   private List<Path> selectedSeries;
@@ -55,6 +60,11 @@ public class LastQueryExecutor {
     this.dataTypes = lastQueryPlan.getDeduplicatedDataTypes();
   }
 
+  public LastQueryExecutor(List<Path> selectedSeries, List<TSDataType> dataTypes) {
+    this.selectedSeries = selectedSeries;
+    this.dataTypes = dataTypes;
+  }
+
   /**
    * execute last function
    *
@@ -93,7 +103,7 @@ public class LastQueryExecutor {
    * @param context query context
    * @return TimeValuePair
    */
-  private TimeValuePair calculateLastPairForOneSeries(
+  public static TimeValuePair calculateLastPairForOneSeries(
       Path seriesPath, TSDataType tsDataType, QueryContext context)
       throws IOException, QueryProcessException, StorageEngineException {
 
@@ -153,7 +163,7 @@ public class LastQueryExecutor {
     return resultPair;
   }
 
-  private TimeValuePair constructLastPair(long timestamp, Object value, TSDataType dataType) {
+  private static TimeValuePair constructLastPair(long timestamp, Object value, TSDataType dataType) {
     return new TimeValuePair(timestamp, TsPrimitiveType.getByType(dataType, value));
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java b/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java
index 823cefb..9a4aaf2 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java
@@ -23,6 +23,8 @@ import org.apache.iotdb.db.exception.StorageEngineException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.physical.crud.*;
 import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.dataset.groupby.GroupByEngineDataSet;
+import org.apache.iotdb.db.query.dataset.groupby.GroupByFillDataSet;
 import org.apache.iotdb.db.query.dataset.groupby.GroupByWithValueFilterDataSet;
 import org.apache.iotdb.db.query.dataset.groupby.GroupByWithoutValueFilterDataSet;
 import org.apache.iotdb.db.query.fill.IFill;
@@ -172,6 +174,14 @@ public class QueryRouter implements IQueryRouter {
   }
 
   @Override
+  public QueryDataSet groupByFill(GroupByFillPlan groupByFillPlan, QueryContext context)
+          throws QueryFilterOptimizationException, StorageEngineException, QueryProcessException, IOException {
+    GroupByEngineDataSet groupByEngineDataSet = (GroupByEngineDataSet) groupBy(groupByFillPlan, context);
+    return new GroupByFillDataSet(groupByFillPlan.getDeduplicatedPaths(), groupByFillPlan.getDeduplicatedDataTypes(),
+            groupByEngineDataSet, groupByFillPlan.getFillType(), context, groupByFillPlan);
+  }
+
+  @Override
   public QueryDataSet lastQuery(LastQueryPlan lastQueryPlan, QueryContext context)
           throws StorageEngineException, QueryProcessException, IOException {
     LastQueryExecutor lastQueryExecutor = new LastQueryExecutor(lastQueryPlan);
diff --git a/server/src/main/java/org/apache/iotdb/db/query/fill/PreviousFill.java b/server/src/main/java/org/apache/iotdb/db/query/fill/PreviousFill.java
index 19b6584..b209c58 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/fill/PreviousFill.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/fill/PreviousFill.java
@@ -18,11 +18,6 @@
  */
 package org.apache.iotdb.db.query.fill;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.PriorityQueue;
-import java.util.Set;
 import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
 import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
 import org.apache.iotdb.db.exception.StorageEngineException;
@@ -40,11 +35,12 @@ import org.apache.iotdb.tsfile.read.common.Path;
 import org.apache.iotdb.tsfile.read.filter.TimeFilter;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.factory.FilterFactory;
-
-import java.io.IOException;
 import org.apache.iotdb.tsfile.read.reader.IPageReader;
 import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
 
+import java.io.IOException;
+import java.util.*;
+
 public class PreviousFill extends IFill {
 
   private Path seriesPath;
@@ -57,19 +53,35 @@ public class PreviousFill extends IFill {
 
   private List<TimeseriesMetadata> unseqTimeseriesMetadataList;
 
+  private boolean untilLast;
+
   public PreviousFill(TSDataType dataType, long queryTime, long beforeRange) {
-    super(dataType, queryTime);
-    this.beforeRange = beforeRange;
-    this.unseqTimeseriesMetadataList = new ArrayList<>();
+    this(dataType, queryTime, beforeRange, false);
   }
 
   public PreviousFill(long beforeRange) {
+    this(beforeRange, false);
+  }
+
+
+  public PreviousFill(long beforeRange, boolean untilLast) {
     this.beforeRange = beforeRange;
+    this.untilLast = untilLast;
   }
 
+
+  public PreviousFill(TSDataType dataType, long queryTime, long beforeRange, boolean untilLast) {
+    super(dataType, queryTime);
+    this.beforeRange = beforeRange;
+    this.unseqTimeseriesMetadataList = new ArrayList<>();
+    this.untilLast = untilLast;
+  }
+
+
+
   @Override
   public IFill copy() {
-    return new PreviousFill(dataType,  queryTime, beforeRange);
+    return new PreviousFill(dataType,  queryTime, beforeRange, untilLast);
   }
 
   @Override
@@ -269,4 +281,12 @@ public class PreviousFill extends IFill {
   private TimeValuePair constructLastPair(long timestamp, Object value, TSDataType dataType) {
     return new TimeValuePair(timestamp, TsPrimitiveType.getByType(dataType, value));
   }
+
+  public boolean isUntilLast() {
+    return untilLast;
+  }
+
+  public void setUntilLast(boolean untilLast) {
+    this.untilLast = untilLast;
+  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
index 762c26d..34a1f42 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
@@ -702,6 +702,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
         break;
       case AGGREGATION:
       case GROUPBY:
+      case GROUP_BY_FILL:
         List<String> aggregations = plan.getAggregations();
         if (aggregations.size() != paths.size()) {
           for (int i = 1; i < paths.size(); i++) {
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBGroupByFillIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBGroupByFillIT.java
new file mode 100644
index 0000000..94825fc
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBGroupByFillIT.java
@@ -0,0 +1,465 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.integration;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.apache.iotdb.jdbc.IoTDBSQLException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+import static org.apache.iotdb.db.constant.TestConstant.TIMESTAMP_STR;
+import static org.apache.iotdb.db.constant.TestConstant.last_value;
+import static org.junit.Assert.*;
+
+public class IoTDBGroupByFillIT {
+
+  private static String[] dataSet1 = new String[]{
+          "SET STORAGE GROUP TO root.ln.wf01.wt01",
+          "CREATE TIMESERIES root.ln.wf01.wt01.status WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
+          "CREATE TIMESERIES root.ln.wf01.wt01.temperature WITH DATATYPE=INT32, ENCODING=PLAIN",
+          "CREATE TIMESERIES root.ln.wf01.wt01.hardware WITH DATATYPE=DOUBLE, ENCODING=PLAIN",
+          "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+                  + "values(10, 21, false, 11.1)",
+          "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+                  + "values(12, 23, true, 22.3)",
+          "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+                  + "values(14, 25, false, 33.5)",
+          "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+                  + "values(29, 26, false, 33.2)",
+          "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+                  + "values(36, 29, false, 44.7)",
+          "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+                  + "values(37, 30, false, 55.8)",
+          "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+                  + "values(39, 40, false, 33.0)",
+          "flush"
+  };
+
+  @Before
+  public void setUp() throws Exception {
+    EnvironmentUtils.closeStatMonitor();
+    EnvironmentUtils.envSetUp();
+    IoTDBDescriptor.getInstance().getConfig().setPartitionInterval(1000);
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    prepareData();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    IoTDBDescriptor.getInstance().getConfig().setPartitionInterval(86400);
+    EnvironmentUtils.cleanEnv();
+  }
+
+  @Test
+  public void previousTest1() {
+    String[] retArray = new String[] {
+          "17,25",
+          "22,25",
+          "27,26",
+          "32,29",
+          "37,40",
+          "42,40",
+          "47,40",
+    };
+
+    try (Connection connection = DriverManager.
+            getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      boolean hasResultSet = statement.execute(
+              "select last_value(temperature) from "
+                      + "root.ln.wf01.wt01 "
+                      + "GROUP BY ([17, 48), 5ms) FILL(int32[previous])");
+
+      assertTrue(hasResultSet);
+      int cnt;
+      try (ResultSet resultSet = statement.getResultSet()) {
+        cnt = 0;
+        while (resultSet.next()) {
+          String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+                  .getString(last_value("root.ln.wf01.wt01.temperature"));
+          assertEquals(retArray[cnt], ans);
+          cnt++;
+        }
+        assertEquals(retArray.length, cnt);
+      }
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+
+  }
+
+  @Test
+  public void previousTest2() {
+    try (Connection connection = DriverManager.
+            getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      statement.execute(
+              "select count(temperature) from "
+                      + "root.ln.wf01.wt01 "
+                      + "GROUP BY ([17, 48), 5ms) FILL(int32[previous])");
+    } catch (IoTDBSQLException e) {
+      System.out.println("error message: " + e.getMessage());
+      assertTrue(e.getMessage().contains("Group By Fill only support last_value function"));
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+
+  }
+
+  @Test
+  public void previousTest3() {
+    String[] retArray = new String[] {
+            "2,null",
+            "7,21",
+            "12,25",
+            "17,25",
+            "22,25",
+            "27,26",
+            "32,29",
+            "37,40",
+            "42,40",
+            "47,40",
+    };
+
+    try (Connection connection = DriverManager.
+            getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      boolean hasResultSet = statement.execute(
+              "select last_value(temperature) from "
+                      + "root.ln.wf01.wt01 "
+                      + "GROUP BY ([2, 48), 5ms) FILL(int32[previous])");
+
+      assertTrue(hasResultSet);
+      int cnt;
+      try (ResultSet resultSet = statement.getResultSet()) {
+        cnt = 0;
+        while (resultSet.next()) {
+          String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+                  .getString(last_value("root.ln.wf01.wt01.temperature"));
+          assertEquals(retArray[cnt], ans);
+          cnt++;
+        }
+        assertEquals(retArray.length, cnt);
+      }
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void previousTest4() {
+    String[] retArray = new String[] {
+            "2,null,null",
+            "7,21,11.1",
+            "12,25,33.5",
+            "17,25,33.5",
+            "22,25,33.5",
+            "27,26,33.2",
+            "32,29,44.7",
+            "37,40,33.0",
+            "42,40,33.0",
+            "47,40,33.0",
+    };
+
+    try (Connection connection = DriverManager.
+            getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      boolean hasResultSet = statement.execute(
+              "select last_value(temperature), last_value(hardware) from "
+                      + "root.ln.wf01.wt01 "
+                      + "GROUP BY ([2, 48), 5ms) FILL(int32[previous], double[previous])");
+
+      assertTrue(hasResultSet);
+      int cnt;
+      try (ResultSet resultSet = statement.getResultSet()) {
+        cnt = 0;
+        while (resultSet.next()) {
+          String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+                  .getString(last_value("root.ln.wf01.wt01.temperature")) + "," +
+                          resultSet.getString(last_value("root.ln.wf01.wt01.hardware"));
+          assertEquals(retArray[cnt], ans);
+          cnt++;
+        }
+        assertEquals(retArray.length, cnt);
+      }
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void previousAllTest() {
+    String[] retArray = new String[] {
+            "2,null,null",
+            "7,21,11.1",
+            "12,25,33.5",
+            "17,25,33.5",
+            "22,25,33.5",
+            "27,26,33.2",
+            "32,29,44.7",
+            "37,40,33.0",
+            "42,40,33.0",
+            "47,40,33.0",
+    };
+
+    try (Connection connection = DriverManager.
+            getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      boolean hasResultSet = statement.execute(
+              "select last_value(temperature), last_value(hardware) from "
+                      + "root.ln.wf01.wt01 "
+                      + "GROUP BY ([2, 48), 5ms) FILL(ALL[previous])");
+
+      assertTrue(hasResultSet);
+      int cnt;
+      try (ResultSet resultSet = statement.getResultSet()) {
+        cnt = 0;
+        while (resultSet.next()) {
+          String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+                  .getString(last_value("root.ln.wf01.wt01.temperature")) + "," +
+                  resultSet.getString(last_value("root.ln.wf01.wt01.hardware"));
+          assertEquals(retArray[cnt], ans);
+          cnt++;
+        }
+        assertEquals(retArray.length, cnt);
+      }
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void previousUntilLastTest1() {
+    String[] retArray = new String[] {
+            "17,25",
+            "22,25",
+            "27,26",
+            "32,29",
+            "37,40",
+            "42,null",
+            "47,null",
+    };
+
+    try (Connection connection = DriverManager.
+            getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      boolean hasResultSet = statement.execute(
+              "select last_value(temperature) from "
+                      + "root.ln.wf01.wt01 "
+                      + "GROUP BY ([17, 48), 5ms) FILL(int32[previousUntilLast])");
+
+      assertTrue(hasResultSet);
+      int cnt;
+      try (ResultSet resultSet = statement.getResultSet()) {
+        cnt = 0;
+        while (resultSet.next()) {
+          String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+                  .getString(last_value("root.ln.wf01.wt01.temperature"));
+          assertEquals(retArray[cnt], ans);
+          cnt++;
+        }
+        assertEquals(retArray.length, cnt);
+      }
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+
+  }
+
+  @Test
+  public void previousUntilLastTest2() {
+    try (Connection connection = DriverManager.
+            getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      statement.execute(
+              "select count(temperature) from "
+                      + "root.ln.wf01.wt01 "
+                      + "GROUP BY ([17, 48), 5ms) FILL(int32[previousUntilLast])");
+    } catch (IoTDBSQLException e) {
+      System.out.println(e.getMessage());
+      assertTrue(e.getMessage().contains("Group By Fill only support last_value function"));
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+
+  }
+
+  @Test
+  public void previousUntilLastTest3() {
+    String[] retArray = new String[] {
+            "2,null",
+            "7,21",
+            "12,25",
+            "17,25",
+            "22,25",
+            "27,26",
+            "32,29",
+            "37,40",
+            "42,null",
+            "47,null",
+    };
+
+    try (Connection connection = DriverManager.
+            getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      boolean hasResultSet = statement.execute(
+              "select last_value(temperature) from "
+                      + "root.ln.wf01.wt01 "
+                      + "GROUP BY ([2, 48), 5ms) FILL(int32[previousUntilLast])");
+
+      assertTrue(hasResultSet);
+      int cnt;
+      try (ResultSet resultSet = statement.getResultSet()) {
+        cnt = 0;
+        while (resultSet.next()) {
+          String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+                  .getString(last_value("root.ln.wf01.wt01.temperature"));
+          assertEquals(retArray[cnt], ans);
+          cnt++;
+        }
+        assertEquals(retArray.length, cnt);
+      }
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void previousUntilLastTest4() {
+    String[] retArray = new String[] {
+            "2,null,null",
+            "7,21,11.1",
+            "12,25,33.5",
+            "17,25,33.5",
+            "22,25,33.5",
+            "27,26,33.2",
+            "32,29,44.7",
+            "37,40,33.0",
+            "42,null,null",
+            "47,null,null",
+    };
+
+    try (Connection connection = DriverManager.
+            getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      boolean hasResultSet = statement.execute(
+              "select last_value(temperature), last_value(hardware) from "
+                      + "root.ln.wf01.wt01 "
+                      + "GROUP BY ([2, 48), 5ms) FILL(int32[previousUntilLast], double[previousUntilLast])");
+
+      assertTrue(hasResultSet);
+      int cnt;
+      try (ResultSet resultSet = statement.getResultSet()) {
+        cnt = 0;
+        while (resultSet.next()) {
+          String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+                  .getString(last_value("root.ln.wf01.wt01.temperature")) + "," +
+                  resultSet.getString(last_value("root.ln.wf01.wt01.hardware"));
+          assertEquals(retArray[cnt], ans);
+          cnt++;
+        }
+        assertEquals(retArray.length, cnt);
+      }
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
+  public void previousUntilLastAllTest() {
+    String[] retArray = new String[] {
+            "2,null,null",
+            "7,21,11.1",
+            "12,25,33.5",
+            "17,25,33.5",
+            "22,25,33.5",
+            "27,26,33.2",
+            "32,29,44.7",
+            "37,40,33.0",
+            "42,null,null",
+            "47,null,null",
+    };
+
+    try (Connection connection = DriverManager.
+            getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+         Statement statement = connection.createStatement()) {
+      boolean hasResultSet = statement.execute(
+              "select last_value(temperature), last_value(hardware) from "
+                      + "root.ln.wf01.wt01 "
+                      + "GROUP BY ([2, 48), 5ms) FILL(ALL[previousUntilLast])");
+
+      assertTrue(hasResultSet);
+      int cnt;
+      try (ResultSet resultSet = statement.getResultSet()) {
+        cnt = 0;
+        while (resultSet.next()) {
+          String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+                  .getString(last_value("root.ln.wf01.wt01.temperature")) + "," +
+                  resultSet.getString(last_value("root.ln.wf01.wt01.hardware"));
+          assertEquals(retArray[cnt], ans);
+          cnt++;
+        }
+        assertEquals(retArray.length, cnt);
+      }
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+
+  private void prepareData() {
+    try (Connection connection = DriverManager
+            .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root",
+                    "root");
+         Statement statement = connection.createStatement();) {
+
+      for (String sql : dataSet1) {
+        statement.execute(sql);
+      }
+
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+}
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLastIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLastIT.java
index af697ad..a30311d 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLastIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBLastIT.java
@@ -18,26 +18,20 @@
  */
 package org.apache.iotdb.db.integration;
 
-import static org.junit.Assert.fail;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import org.apache.iotdb.db.exception.metadata.MetadataException;
-import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.metadata.mnode.LeafMNode;
 import org.apache.iotdb.db.metadata.mnode.MNode;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
-import org.apache.iotdb.db.utils.TestOnly;
 import org.apache.iotdb.jdbc.Config;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.sql.*;
+
+import static org.junit.Assert.fail;
+
 public class IoTDBLastIT {
 
   private static String[] dataSet1 = new String[]{
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
index 1730212..a11cb5d 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
@@ -18,33 +18,17 @@
  */
 package org.apache.iotdb.db.qp.plan;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
+import org.antlr.v4.runtime.misc.ParseCancellationException;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.exception.runtime.SQLParserException;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.qp.Planner;
 import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
-import org.apache.iotdb.db.qp.physical.crud.FillQueryPlan;
-import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
-import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
-import org.apache.iotdb.db.qp.physical.crud.RawDataQueryPlan;
-import org.apache.iotdb.db.qp.physical.crud.LastQueryPlan;
-import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
-import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
-import org.apache.iotdb.db.qp.physical.sys.DataAuthPlan;
-import org.apache.iotdb.db.qp.physical.sys.LoadConfigurationPlan;
-import org.apache.iotdb.db.qp.physical.sys.OperateFilePlan;
-import org.apache.iotdb.db.qp.physical.sys.ShowPlan;
+import org.apache.iotdb.db.qp.physical.crud.*;
+import org.apache.iotdb.db.qp.physical.sys.*;
 import org.apache.iotdb.db.query.fill.LinearFill;
 import org.apache.iotdb.db.query.fill.PreviousFill;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
@@ -64,6 +48,13 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.io.File;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.junit.Assert.*;
+
 public class PhysicalPlanTest {
 
   private Planner processor = new Planner();
@@ -212,6 +203,7 @@ public class PhysicalPlanTest {
     String sqlStr = "SELECT s1 FROM root.vehicle.d1 WHERE time = 5000 Fill(int32[linear, 5m], boolean[previous])";
     try {
       processor.parseSQLToPhysicalPlan(sqlStr);
+      fail();
     } catch (Exception e) {
       assertTrue(true);
     }
@@ -222,12 +214,157 @@ public class PhysicalPlanTest {
     String sqlStr = "SELECT s1 FROM root.vehicle.d1 WHERE time > 5000 Fill(int32[linear], boolean[previous])";
     try {
       processor.parseSQLToPhysicalPlan(sqlStr);
+      fail();
     } catch (Exception e) {
       assertEquals("Only \"=\" can be used in fill function", e.getMessage());
     }
   }
 
   @Test
+  public void testGroupByFill1() {
+    String sqlStr =
+            "select last_value(s1) " + " from root.vehicle.d1 "
+                    + "group by([8,737), 3ms) fill(int32[previous])";
+    try {
+      PhysicalPlan plan = processor.parseSQLToPhysicalPlan(sqlStr);
+      if (!plan.isQuery()) {
+        fail();
+      }
+      if (!(plan instanceof GroupByFillPlan)) {
+        fail();
+      }
+      GroupByFillPlan groupByFillPlan = (GroupByFillPlan) plan;
+      assertEquals(3L, groupByFillPlan.getInterval());
+      assertEquals(3L, groupByFillPlan.getSlidingStep());
+      assertEquals(8L, groupByFillPlan.getStartTime());
+      assertEquals(737L, groupByFillPlan.getEndTime());
+      assertEquals(1, groupByFillPlan.getFillType().size());
+      assertTrue(groupByFillPlan.getFillType().containsKey(TSDataType.INT32));
+      assertTrue(groupByFillPlan.getFillType().get(TSDataType.INT32) instanceof PreviousFill);
+      PreviousFill previousFill = (PreviousFill) groupByFillPlan.getFillType().get(TSDataType.INT32);
+      assertFalse(previousFill.isUntilLast());
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail();
+    }
+  }
+
+  @Test
+  public void testGroupByFill2() {
+    String sqlStr =
+            "select last_value(s1) " + " from root.vehicle.d1 "
+                    + "group by([8,737), 3ms) fill(ALL[previousuntillast])";
+    try {
+      PhysicalPlan plan = processor.parseSQLToPhysicalPlan(sqlStr);
+      if (!plan.isQuery()) {
+        fail();
+      }
+      if (!(plan instanceof GroupByFillPlan)) {
+        fail();
+      }
+      GroupByFillPlan groupByFillPlan = (GroupByFillPlan) plan;
+      assertEquals(3L, groupByFillPlan.getInterval());
+      assertEquals(3L, groupByFillPlan.getSlidingStep());
+      assertEquals(8L, groupByFillPlan.getStartTime());
+      assertEquals(737L, groupByFillPlan.getEndTime());
+      assertEquals(TSDataType.values().length, groupByFillPlan.getFillType().size());
+      for (TSDataType tsDataType : TSDataType.values()) {
+        assertTrue(groupByFillPlan.getFillType().containsKey(tsDataType));
+        assertTrue(groupByFillPlan.getFillType().get(tsDataType) instanceof PreviousFill);
+        PreviousFill previousFill = (PreviousFill) groupByFillPlan.getFillType().get(tsDataType);
+        assertTrue(previousFill.isUntilLast());
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail();
+    }
+  }
+
+  @Test
+  public void testGroupByFill3() {
+    String sqlStr =
+            "select last_value(d1.s1), last_value(d2.s1)" + " from root.vehicle "
+                    + "group by([8,737), 3ms) fill(int32[previousuntillast], int64[previous])";
+    try {
+      PhysicalPlan plan = processor.parseSQLToPhysicalPlan(sqlStr);
+      if (!plan.isQuery()) {
+        fail();
+      }
+      if (!(plan instanceof GroupByFillPlan)) {
+        fail();
+      }
+      GroupByFillPlan groupByFillPlan = (GroupByFillPlan) plan;
+      assertEquals(3L, groupByFillPlan.getInterval());
+      assertEquals(3L, groupByFillPlan.getSlidingStep());
+      assertEquals(8L, groupByFillPlan.getStartTime());
+      assertEquals(737L, groupByFillPlan.getEndTime());
+      assertEquals(2, groupByFillPlan.getDeduplicatedPaths().size());
+      assertEquals(2, groupByFillPlan.getFillType().size());
+
+      assertTrue(groupByFillPlan.getFillType().containsKey(TSDataType.INT32));
+      assertTrue(groupByFillPlan.getFillType().get(TSDataType.INT32) instanceof PreviousFill);
+      PreviousFill previousFill = (PreviousFill) groupByFillPlan.getFillType().get(TSDataType.INT32);
+      assertTrue(previousFill.isUntilLast());
+
+      assertTrue(groupByFillPlan.getFillType().containsKey(TSDataType.INT64));
+      assertTrue(groupByFillPlan.getFillType().get(TSDataType.INT64) instanceof PreviousFill);
+      previousFill = (PreviousFill) groupByFillPlan.getFillType().get(TSDataType.INT64);
+      assertFalse(previousFill.isUntilLast());
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail();
+    }
+  }
+
+  @Test
+  public void testGroupByFill4() {
+    String sqlStr =
+            "select last_value(d1.s1), last_value(d2.s1)" + " from root.vehicle "
+                    + "group by([8,737), 3ms) fill(int32[linear])";
+    try {
+      processor.parseSQLToPhysicalPlan(sqlStr);
+      fail();
+    } catch (SQLParserException e) {
+      assertEquals("group by fill doesn't support linear fill", e.getMessage());
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail();
+    }
+  }
+
+  @Test
+  public void testGroupByFill5() {
+    String sqlStr =
+            "select last_value(d1.s1), count(d2.s1)" + " from root.vehicle "
+                    + "group by([8,737), 3ms) fill(int32[previous])";
+    try {
+      processor.parseSQLToPhysicalPlan(sqlStr);
+      fail();
+    } catch (QueryProcessException e) {
+      assertEquals("Group By Fill only support last_value function", e.getMessage());
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail();
+    }
+  }
+
+  @Test
+  public void testGroupByFill6() {
+    String sqlStr =
+            "select count(s1)" + "from root.vehicle.d1 "
+                    + "group by([8,737), 3ms, 5ms) fill(int32[previous])";
+    try {
+      processor.parseSQLToPhysicalPlan(sqlStr);
+      fail();
+    } catch (ParseCancellationException e) {
+      assertTrue(e.getMessage().contains("mismatched input 'fill'"));
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail();
+    }
+  }
+
+  @Test
   public void testQuery1() throws QueryProcessException {
     String sqlStr = "SELECT s1 FROM root.vehicle.d1 WHERE time > 5000";
     PhysicalPlan plan = processor.parseSQLToPhysicalPlan(sqlStr);
diff --git a/site/src/main/.vuepress/config.js b/site/src/main/.vuepress/config.js
index 538fa9f..8ca3505 100644
--- a/site/src/main/.vuepress/config.js
+++ b/site/src/main/.vuepress/config.js
@@ -441,7 +441,8 @@ var config = {
 							['5-DataQuery/6-GroupByQuery','GroupByQuery'],
 							['5-DataQuery/7-LastQuery','LastQuery'],
 							['5-DataQuery/8-AlignByDeviceQuery','AlignByDeviceQuery'],
-							['5-DataQuery/9-FillFunction','FillFunction']
+							['5-DataQuery/9-FillFunction','FillFunction'],
+	                        ['5-DataQuery/10-GroupByFillQuery', 'GroupByFillQuery']
 						]
 					},
 					{
@@ -851,7 +852,8 @@ var config = {
 							['5-DataQuery/6-GroupByQuery','降采样查询'],
 							['5-DataQuery/7-LastQuery','最近时间戳 Last 查询'],
 							['5-DataQuery/8-AlignByDeviceQuery','按设备对齐查询'],
-							['5-DataQuery/9-FillFunction','空值填充']
+							['5-DataQuery/9-FillFunction','空值填充'],
+		                    ['5-DataQuery/10-GroupByFillQuery', '降采样补空值查询']
 						]
 					},
 					{