You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ja...@apache.org on 2020/02/26 07:38:22 UTC

[incubator-iotdb] branch groupbyfill updated (0914138 -> 3eb4608)

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

jackietien pushed a change to branch groupbyfill
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git.


    from 0914138  need to wait for last
     new 5965334  add ut and it
     new 3eb4608  add previousUntilLast IT

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../org/apache/iotdb/db/qp/strategy/SqlBase.g4     |   4 +-
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |   3 +-
 .../iotdb/db/qp/logical/crud/SelectOperator.java   |   5 +-
 .../iotdb/db/qp/strategy/LogicalGenerator.java     |  18 +-
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |   5 +
 .../query/dataset/groupby/GroupByFillDataSet.java  |   6 +-
 .../apache/iotdb/db/query/fill/PreviousFill.java   |   6 +-
 .../org/apache/iotdb/db/service/TSServiceImpl.java |   1 +
 .../iotdb/db/integration/IoTDBGroupByFillIT.java   | 447 +++++++++++++++++++++
 .../apache/iotdb/db/qp/plan/PhysicalPlanTest.java  | 176 +++++++-
 10 files changed, 637 insertions(+), 34 deletions(-)
 create mode 100644 server/src/test/java/org/apache/iotdb/db/integration/IoTDBGroupByFillIT.java


[incubator-iotdb] 02/02: add previousUntilLast IT

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 3eb46083bff70ee5d0e6ffadd9112eeea5559ad6
Author: JackieTien97 <Ja...@foxmail.com>
AuthorDate: Wed Feb 26 15:38:00 2020 +0800

    add previousUntilLast IT
---
 .../iotdb/db/integration/IoTDBGroupByFillIT.java   | 186 +++++++++++++++++++++
 1 file changed, 186 insertions(+)

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
index 328d094..6efbd23 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBGroupByFillIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBGroupByFillIT.java
@@ -244,6 +244,192 @@ public class IoTDBGroupByFillIT {
     }
   }
 
+  @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])");
+
+      Assert.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) {
+      assertEquals("Statement format is not right: Group By Fill only support last_value function", e.getMessage());
+    } 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])");
+
+      Assert.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])");
+
+      Assert.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])");
+
+      Assert.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",


[incubator-iotdb] 01/02: add ut and it

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 596533477fa622b6eb106c466d31b7a94881bde8
Author: JackieTien97 <Ja...@foxmail.com>
AuthorDate: Wed Feb 26 15:04:18 2020 +0800

    add ut and it
---
 .../org/apache/iotdb/db/qp/strategy/SqlBase.g4     |   4 +-
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |   3 +-
 .../iotdb/db/qp/logical/crud/SelectOperator.java   |   5 +-
 .../iotdb/db/qp/strategy/LogicalGenerator.java     |  18 +-
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |   5 +
 .../query/dataset/groupby/GroupByFillDataSet.java  |   6 +-
 .../apache/iotdb/db/query/fill/PreviousFill.java   |   6 +-
 .../org/apache/iotdb/db/service/TSServiceImpl.java |   1 +
 .../iotdb/db/integration/IoTDBGroupByFillIT.java   | 261 +++++++++++++++++++++
 .../apache/iotdb/db/qp/plan/PhysicalPlanTest.java  | 176 ++++++++++++--
 10 files changed, 451 insertions(+), 34 deletions(-)

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 2434075..9f0ad4e 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
@@ -210,7 +210,7 @@ groupByFillClause
       timeInterval
       COMMA DURATION
       RR_BRACKET
-      fillClause
+      FILL LR_BRACKET typeClause (COMMA typeClause)* RR_BRACKET
      ;
 
 typeClause
@@ -228,7 +228,7 @@ previousClause
     ;
 
 previousUntilLastClause
-    : PREVIOUSUNTILLAST
+    : PREVIOUSUNTILLAST (COMMA DURATION)?
     ;
 
 indexWithClause
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 e968e7f..30733fd 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
@@ -221,9 +221,10 @@ public class PlanExecutor implements IPlanExecutor {
     } 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.groupByFill(groupByPlan, context);
+        return queryRouter.groupBy(groupByPlan, context);
       } else if (queryPlan instanceof AggregationPlan) {
         AggregationPlan aggregationPlan = (AggregationPlan) queryPlan;
         queryDataSet = queryRouter.aggregate(aggregationPlan, context);
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 902dc38..9308ef1 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/strategy/LogicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
index 0180540..f52e546 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
@@ -570,9 +570,7 @@ public class LogicalGenerator extends SqlBaseBaseListener {
 
     parseTimeInterval(ctx.timeInterval());
 
-    FillClauseContext fillClauseContext = ctx.fillClause();
-    super.enterFillClause(fillClauseContext);
-    List<TypeClauseContext> list = fillClauseContext.typeClause();
+    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
@@ -580,8 +578,16 @@ public class LogicalGenerator extends SqlBaseBaseListener {
         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())) {
-
+      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);
@@ -670,7 +676,7 @@ public class LogicalGenerator extends SqlBaseBaseListener {
         fillTypes.put(dataType, new PreviousFill(defaultFillInterval));
       }
     } else { // previous until last
-      if (ctx.previousClause().DURATION() != null) {
+      if (ctx.previousUntilLastClause().DURATION() != null) {
         long preRange = parseDuration(ctx.previousClause().DURATION().getText());
         fillTypes.put(dataType, new PreviousFill(preRange, true));
       } else {
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 3e22199..e585222 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
@@ -176,6 +176,11 @@ public class PhysicalGenerator {
       ((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();
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
index 1bfc43b..5c7c381 100644
--- 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
@@ -72,14 +72,14 @@ public class GroupByFillDataSet extends QueryDataSet {
       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(field.getDataType())).isUntilLast() || rowRecord.getTimestamp() <= lastTimeArray[i])) {
-          rowRecord.getFields().set(i, Field.getField(previousValue, field.getDataType()));
+                && (!((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 null;
+    return rowRecord;
   }
 }
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 441075b..b95c49b 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
@@ -55,7 +55,7 @@ public class PreviousFill extends IFill {
 
   @Override
   public IFill copy() {
-    return new PreviousFill(dataType, queryTime, beforeRange);
+    return new PreviousFill(dataType, queryTime, beforeRange, untilLast);
   }
 
   @Override
@@ -98,4 +98,8 @@ public class PreviousFill extends IFill {
   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 b2a8f71..c44a3a5 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
@@ -784,6 +784,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..328d094
--- /dev/null
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBGroupByFillIT.java
@@ -0,0 +1,261 @@
+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.Assert;
+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.integration.Constant.TIMESTAMP_STR;
+import static org.apache.iotdb.db.integration.Constant.last_value;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+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])");
+
+      Assert.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) {
+      assertEquals("Statement format is not right: Group By Fill only support last_value function", e.getMessage());
+    } 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])");
+
+      Assert.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])");
+
+      Assert.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])");
+
+      Assert.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/qp/plan/PhysicalPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
index c2371c9..4185ed3 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,32 +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.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;
@@ -63,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();
@@ -213,6 +205,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);
     }
@@ -223,12 +216,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);