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 2021/08/12 07:21:52 UTC

[iotdb] branch master updated: [IOTDB-1561]Support fill by specific value (#3726)

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/iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 1f1dff2  [IOTDB-1561]Support fill by specific value (#3726)
1f1dff2 is described below

commit 1f1dff23d553eede5e3d6a0b71d4587857f46b3c
Author: Hang Ji <55...@users.noreply.github.com>
AuthorDate: Thu Aug 12 15:21:29 2021 +0800

    [IOTDB-1561]Support fill by specific value (#3726)
---
 .../antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4   |   9 ++
 .../DML-Data-Manipulation-Language.md              |  52 +++++++++-
 .../DML-Data-Manipulation-Language.md              |  60 +++++++++--
 .../apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java    |   7 ++
 .../iotdb/db/query/executor/fill/ValueFill.java    | 110 +++++++++++++++++++++
 .../apache/iotdb/db/integration/IoTDBFillIT.java   |  32 ++++++
 6 files changed, 258 insertions(+), 12 deletions(-)

diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
index 0943224..d122f24 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
@@ -311,6 +311,7 @@ groupByLevelClause
 typeClause
     : (dataType | ALL) LS_BRACKET linearClause RS_BRACKET
     | (dataType | ALL) LS_BRACKET previousClause RS_BRACKET
+    | (dataType | ALL) LS_BRACKET valueClause RS_BRACKET
     | (dataType | ALL) LS_BRACKET previousUntilLastClause RS_BRACKET
     ;
 
@@ -322,6 +323,10 @@ previousClause
     : PREVIOUS (COMMA DURATION)?
     ;
 
+valueClause
+    : VALUE (COMMA constant)?
+    ;
+
 previousUntilLastClause
     : PREVIOUSUNTILLAST (COMMA DURATION)?
     ;
@@ -817,6 +822,10 @@ LINEAR
     : L I N E A R
     ;
 
+VALUE
+    : V A L U E
+    ;
+
 PREVIOUS
     : P R E V I O U S
     ;
diff --git a/docs/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md b/docs/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
index 068b74f..8548cd0 100644
--- a/docs/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
+++ b/docs/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
@@ -1032,6 +1032,48 @@ Total line number = 1
 It costs 0.017s
 ```
 
+* Value Method
+
+When the value of the queried timestamp is null, given fill value is used to fill the blank. The formalized value method is as follows:
+
+```
+select <path> from <prefixPath> where time = <T> fill(<data_type>[value, constant]…)
+```
+Detailed descriptions of all parameters are given in Table 3-6.
+
+<center>**Table 3-6 Specific value fill paramter list**
+
+|Parameter name (case insensitive)|Interpretation|
+|:---|:---|
+|path, prefixPath|query path; mandatory field|
+|T|query timestamp (only one can be specified); mandatory field|
+|data_type|the type of data used by the fill method. Optional values are int32, int64, float, double, boolean, text; optional field|
+|constant|represents given fill value|
+</center>
+
+**Note** if the timeseries has a valid value at query timestamp T, this value will be used as the specific fill value.
+
+Here we give an example of filling null values using the value method. The SQL statement is as follows:
+
+```
+select temperature from root.sgcc.wf03.wt01 where time = 2017-11-01T16:37:50.000 fill(float [value, 2.0])
+```
+which means:
+
+Because the timeseries root.sgcc.wf03.wt01.temperature is null at 2017-11-01T16:37:50.000, the system uses given specific value 2.0 to fill
+
+On the [sample data](https://github.com/thulab/iotdb/files/4438687/OtherMaterial-Sample.Data.txt), the execution result of this statement is shown below:
+
+```
++-----------------------------+-------------------------------+
+|                         Time|root.sgcc.wf03.wt01.temperature|
++-----------------------------+-------------------------------+
+|2017-11-01T16:37:50.000+08:00|                            2.0|
++-----------------------------+-------------------------------+
+Total line number = 1
+It costs 0.007s
+```
+
 #### Correspondence between Data Type and Fill Method
 
 Data types and the supported fill methods are shown in Table 3-6.
@@ -1040,11 +1082,11 @@ Data types and the supported fill methods are shown in Table 3-6.
 
 |Data Type|Supported Fill Methods|
 |:---|:---|
-|boolean|previous|
-|int32|previous, linear|
-|int64|previous, linear|
-|float|previous, linear|
-|double|previous, linear|
+|boolean|previous, value|
+|int32|previous, linear, value|
+|int64|previous, linear, value|
+|float|previous, linear, value|
+|double|previous, linear, value|
 |text|previous|
 </center>
 
diff --git a/docs/zh/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md b/docs/zh/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
index e7af0cd..8a8bae7 100644
--- a/docs/zh/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
+++ b/docs/zh/UserGuide/IoTDB-SQL-Language/DML-Data-Manipulation-Language.md
@@ -810,6 +810,52 @@ Total line number = 1
 It costs 0.017s
 ```
 
+- Value方法
+
+当查询的时间戳值为空时,将使用给定的值来填充空白。 特定值填充方法如下:
+
+```
+select <path> from <prefixPath> where time = <T> fill(<data_type>[value, constant]…)
+```
+
+表3-6中给出了所有参数的详细说明。
+
+<center>**表3-6特定值填充参数列表**
+
+| 参数名称(不区分大小写)    | 解释                                                         |
+| :-------------------------- | :----------------------------------------------------------- |
+| path, prefixPath            | 查询路径; 必填项                                            |
+| T                           | 查询时间戳(只能指定一个); 必填项                          |
+| data_type                   | 填充方法使用的数据类型。 可选值是int32,int64,float,double,boolean,text; 可选字段 |
+| constant                    | 给定的填充值                                          |
+
+</center>
+
+需要注意的是一旦时间序列在查询时间戳T时刻存在有效值,特定值填充就会使用这个值作为结果返回。
+
+
+在这里,我们举一个使用特定值方法填充空值的示例。 SQL语句如下:
+
+```
+select temperature from root.sgcc.wf03.wt01 where time = 2017-11-01T16:37:50.000 fill(float [value, 2.0])
+```
+
+意思是:
+
+由于时间根目录root.sgcc.wf03.wt01.temperature在2017-11-01T16:37:50.000为空,因此使用给定的值2.0进行填充:
+
+在 [样例数据](https://github.com/thulab/iotdb/files/4438687/OtherMaterial-Sample.Data.txt), 该语句的执行结果如下所示:
+
+```
++-----------------------------+-------------------------------+
+|                         Time|root.sgcc.wf03.wt01.temperature|
++-----------------------------+-------------------------------+
+|2017-11-01T16:37:50.000+08:00|                           2.0 |
++-----------------------------+-------------------------------+
+Total line number = 1
+It costs 0.007s
+```
+
 数据类型和填充方法之间的对应关系
 
 数据类型和支持的填充方法如表 3-6 所示。
@@ -817,13 +863,13 @@ It costs 0.017s
 <center>**表 3-6 数据类型和支持的填充方法**
 
 | 数据类型 | 支持的填充方法   |
-| :------- | :--------------- |
-| boolean  | previous         |
-| int32    | previous, linear |
-| int64    | previous, linear |
-| float    | previous, linear |
-| double   | previous, linear |
-| text     | previous         |
+| :------- | :---------------------  |
+| boolean  | previous, value         |
+| int32    | previous, linear, value |
+| int64    | previous, linear, value |
+| float    | previous, linear, value |
+| double   | previous, linear, value |
+| text     | previous, value         |
 
 </center>
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java b/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
index 6da499b..ef09d82 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
@@ -242,6 +242,7 @@ import org.apache.iotdb.db.qp.utils.DatetimeUtils;
 import org.apache.iotdb.db.query.executor.fill.IFill;
 import org.apache.iotdb.db.query.executor.fill.LinearFill;
 import org.apache.iotdb.db.query.executor.fill.PreviousFill;
+import org.apache.iotdb.db.query.executor.fill.ValueFill;
 import org.apache.iotdb.db.query.expression.Expression;
 import org.apache.iotdb.db.query.expression.ResultColumn;
 import org.apache.iotdb.db.query.expression.binary.AdditionExpression;
@@ -1816,6 +1817,12 @@ public class IoTDBSqlVisitor extends SqlBaseBaseVisitor<Operator> {
       } else {
         fillTypes.put(dataType, new PreviousFill(defaultFillInterval));
       }
+    } else if (ctx.valueClause() != null) {
+      if (ctx.valueClause().constant() != null) {
+        fillTypes.put(dataType, new ValueFill(ctx.valueClause().constant().getText(), dataType));
+      } else {
+        throw new SQLParserException("fill value cannot be null");
+      }
     } else { // previous until last
       if (ctx.previousUntilLastClause().DURATION() != null) {
         long preRange =
diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/fill/ValueFill.java b/server/src/main/java/org/apache/iotdb/db/query/executor/fill/ValueFill.java
new file mode 100644
index 0000000..356e21b
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/executor/fill/ValueFill.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.query.executor.fill;
+
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.TimeValuePair;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+
+import java.util.Set;
+
+public class ValueFill extends IFill implements Cloneable {
+
+  private String value;
+
+  private TsPrimitiveType tsPrimitiveType;
+
+  public ValueFill(String value, TSDataType dataType) {
+    this.value = value;
+    this.dataType = dataType;
+    parseTsPrimitiveType();
+  }
+
+  @Override
+  public IFill copy() {
+    return (IFill) clone();
+  }
+
+  @Override
+  public Object clone() {
+    ValueFill valueFill = null;
+    try {
+      valueFill = (ValueFill) super.clone();
+    } catch (CloneNotSupportedException ignored) {
+    }
+    return valueFill;
+  }
+
+  @Override
+  public void configureFill(
+      PartialPath path,
+      TSDataType dataType,
+      long queryTime,
+      Set<String> deviceMeasurements,
+      QueryContext context) {
+    this.queryTime = queryTime;
+  }
+
+  @Override
+  public TimeValuePair getFillResult() {
+    switch (dataType) {
+      case BOOLEAN:
+      case INT32:
+      case INT64:
+      case FLOAT:
+      case DOUBLE:
+      case TEXT:
+        return new TimeValuePair(queryTime, tsPrimitiveType);
+      default:
+        throw new UnSupportedDataTypeException("Unsupported data type:" + dataType);
+    }
+  }
+
+  @Override
+  void constructFilter() {}
+
+  private void parseTsPrimitiveType() {
+    switch (dataType) {
+      case BOOLEAN:
+        this.tsPrimitiveType = new TsPrimitiveType.TsBoolean(Boolean.parseBoolean(value));
+        break;
+      case INT32:
+        this.tsPrimitiveType = new TsPrimitiveType.TsInt(Integer.parseInt(value));
+        break;
+      case INT64:
+        this.tsPrimitiveType = new TsPrimitiveType.TsLong(Long.parseLong(value));
+        break;
+      case FLOAT:
+        this.tsPrimitiveType = new TsPrimitiveType.TsFloat(Float.parseFloat(value));
+        break;
+      case DOUBLE:
+        this.tsPrimitiveType = new TsPrimitiveType.TsDouble(Double.parseDouble(value));
+        break;
+      case TEXT:
+        this.tsPrimitiveType = new TsPrimitiveType.TsBinary(Binary.valueOf(value));
+        break;
+      default:
+        throw new UnSupportedDataTypeException("Unsupported data type:" + dataType);
+    }
+  }
+}
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFillIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFillIT.java
index 1647f29..45aabaa 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFillIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBFillIT.java
@@ -312,6 +312,38 @@ public class IoTDBFillIT {
   }
 
   @Test
+  public void ValueFillTest() {
+    String res = "7,7.0,true,7";
+    try (Connection connection =
+            DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      boolean hasResultSet =
+          statement.execute(
+              "select temperature,status, hardware "
+                  + "from root.ln.wf01.wt01 where time = 7 "
+                  + "Fill(int32[value, 7], double[value, 7], boolean[value, true])");
+
+      Assert.assertTrue(hasResultSet);
+      ResultSet resultSet = statement.getResultSet();
+      while (resultSet.next()) {
+        String ans =
+            resultSet.getString(TIMESTAMP_STR)
+                + ","
+                + resultSet.getString(TEMPERATURE_STR_1)
+                + ","
+                + resultSet.getString(STATUS_STR_1)
+                + ","
+                + resultSet.getString(HARDWARE_STR);
+        Assert.assertEquals(res, ans);
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+
+  @Test
   public void PreviousFillTest() {
     String[] retArray1 = new String[] {"3,3.3,false,33", "70,50.5,false,550", "70,null,null,null"};
     try (Connection connection =