You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2022/03/17 12:01:48 UTC

[GitHub] [iotdb] flashzxi opened a new pull request #5273: [IOTDB-2679] Support logical operators in select clauses

flashzxi opened a new pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273


   ## Description
   Add some functions to support select logical expression in select clauses.
   
   ### Details
   Now you can select logical expressions like a>b, fore example
   
   ```sql
   select a, a>10, !(a<20) && a<30 from root.test;
   ```
   **output**:
   ```
   IoTDB> select a, a>10, !(a<20) && a<30 from root.test;
   +-----------------------------+-----------+----------------+--------------------------------------+
   |                         Time|root.test.a|root.test.a > 10|!root.test.a < 20 & (root.test.a < 30)|
   +-----------------------------+-----------+----------------+--------------------------------------+
   |1970-01-01T08:00:00.001+08:00|         23|            true|                                  true|
   |1970-01-01T08:00:00.002+08:00|         33|            true|                                 false|
   |1970-01-01T08:00:00.004+08:00|         13|            true|                                 false|
   |1970-01-01T08:00:00.005+08:00|         26|            true|                                  true|
   |1970-01-01T08:00:00.008+08:00|          1|           false|                                 false|
   |1970-01-01T08:00:00.010+08:00|         23|            true|                                  true|
   +-----------------------------+-----------+----------------+--------------------------------------+
   ```
   ```
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] ericpai commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
ericpai commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r835706750



##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/BinaryTransformer.java
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+import java.io.IOException;
+
+public abstract class BinaryTransformer extends Transformer {
+
+  private final LayerPointReader leftPointReader;
+  private final LayerPointReader rightPointReader;
+
+  protected BinaryTransformer(LayerPointReader leftPointReader, LayerPointReader rightPointReader) {
+    this.leftPointReader = leftPointReader;
+    this.rightPointReader = rightPointReader;
+  }
+
+  @Override
+  public boolean isConstantPointReader() {
+    return leftPointReader.isConstantPointReader() && rightPointReader.isConstantPointReader();
+  }
+
+  @Override
+  protected boolean cacheValue() throws QueryProcessException, IOException {
+    if (!leftPointReader.next() || !rightPointReader.next()) {
+      return false;
+    }
+    if (!cacheTime()) {
+      return false;
+    }
+    if (leftPointReader.isCurrentNull() || rightPointReader.isCurrentNull()) {
+      currentNull = true;
+    } else {
+      switch (getDataType()) {
+        case DOUBLE:
+          cachedDouble =
+              evaluateDouble(
+                  castCurrentValueToDoubleOperand(leftPointReader),
+                  castCurrentValueToDoubleOperand(rightPointReader));
+          break;
+        case BOOLEAN:
+          cachedBoolean =
+              evaluateBoolean(
+                  castCurrentValueToDoubleOperand(leftPointReader),
+                  castCurrentValueToDoubleOperand(rightPointReader));
+      }
+    }
+    leftPointReader.readyForNext();
+    rightPointReader.readyForNext();
+    return true;
+  }
+
+  /**
+   * finds the smallest, unconsumed timestamp that exists in both {@code leftPointReader} and {@code
+   * rightPointReader} and then caches the timestamp in {@code cachedTime}.
+   *
+   * @return true if there has a timestamp that meets the requirements
+   */
+  private boolean cacheTime() throws IOException, QueryProcessException {
+    if (leftPointReader.isConstantPointReader() && rightPointReader.isConstantPointReader()) {
+      return true;
+    }
+    if (leftPointReader.isConstantPointReader()) {
+      cachedTime = rightPointReader.currentTime();
+      return true;
+    }
+    if (rightPointReader.isConstantPointReader()) {
+      cachedTime = leftPointReader.currentTime();
+      return true;
+    }
+
+    long leftTime = leftPointReader.currentTime();
+    long rightTime = rightPointReader.currentTime();
+
+    while (leftTime != rightTime) {
+      if (leftTime < rightTime) {
+        leftPointReader.readyForNext();
+        if (!leftPointReader.next()) {
+          return false;
+        }
+        leftTime = leftPointReader.currentTime();
+      } else {
+        rightPointReader.readyForNext();
+        if (!rightPointReader.next()) {
+          return false;
+        }
+        rightTime = rightPointReader.currentTime();
+      }
+    }
+
+    // leftTime == rightTime
+    cachedTime = leftTime;
+    return true;
+  }
+
+  protected double evaluateDouble(double leftOperand, double rightOperand) {
+    return 0.0;
+  }
+
+  protected boolean evaluateBoolean(double leftOperand, double rightOperand) {

Review comment:
       So is this defined in CompareBinaryTransformer better?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#issuecomment-1073401511


   Hi @cornmonster @ericpai, PTAL :D


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r830707486



##########
File path: integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSelectCompareExpressionIT.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.integration.env.EnvFactory;
+import org.apache.iotdb.itbase.category.ClusterTest;
+import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.*;

Review comment:
       We don't use `*` in import.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] flashzxi commented on pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
flashzxi commented on pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#issuecomment-1078906762


   > @flashzxi why the file mod of .bat and .sh are changed to 100644? They should be excutable.
    I haven't modified them. Maybe caused by the difference of \n and \r\n on win and mac platform? 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] flashzxi commented on pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
flashzxi commented on pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#issuecomment-1078761336


   @ericpai  Please take a look


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] ericpai commented on pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
ericpai commented on pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#issuecomment-1079667558


   LGTM. Thanks for the excellent feature!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] ericpai commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
ericpai commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r835743609



##########
File path: docs/zh/UserGuide/Query-Data/Select-Expression.md
##########
@@ -106,6 +110,55 @@ Total line number = 5
 It costs 0.014s
 ```
 
+## 逻辑运算查询
+#### 一元逻辑运算符
+支持运算符 `!`
+
+输入数据类型:`BOOLEAN`
+
+输出数据类型:`BOOLEAN`
+
+注意:`!`的优先级很高,记得使用括号调整优先级
+
+#### 二元比较运算符
+
+支持运算符 `>`, `>=`, `<`, `<=`, `==`, `!=`
+
+输入数据类型: `INT32`, `INT64`, `FLOAT`, `DOUBLE`
+
+会将所有数据转换为`DOUBLE`类型后进行比较。
+
+返回类型:`BOOLEAN`
+
+#### 二元逻辑运算符
+
+支持运算符 AND:`and`,`&`, `&&`; OR:`or`,`|`,`||`
+
+输入数据类型:`BOOLEAN`
+
+返回类型 `BOOLEAN`
+
+注意:当某个时间戳下左操作数和右操作数都为BOOLEAN类型时,二元逻辑操作才会有输出结果

Review comment:
       It seems that the `&` and `&&` have been  already defined with the same meaning. There will be a break change if we modify it. :(
   Just add a summary of the priorites of all the operators. Thank you~




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] ericpai commented on pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
ericpai commented on pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#issuecomment-1079668469


   @SteveYurongSu Are there any other comments?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] flashzxi commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
flashzxi commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r831283266



##########
File path: integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSelectCompareExpressionIT.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.integration.env.EnvFactory;
+import org.apache.iotdb.itbase.category.ClusterTest;
+import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.*;
+
+@Category({LocalStandaloneTest.class, ClusterTest.class})
+public class IoTDBSelectCompareExpressionIT {
+
+  private static String[] INSERTION_SQLS;
+  private static List<Long> time = new ArrayList<>(0);
+  private static List<Integer> _int = new ArrayList<>(0);
+  private static List<Long> _long = new ArrayList<>(0);
+  private static List<Float> _float = new ArrayList<>(0);
+  private static List<Double> _double = new ArrayList<>(0);
+  private static List<Boolean> _bool = new ArrayList<>(0);
+
+  private static void generateInsertionSQLS() {
+    INSERTION_SQLS = new String[50];
+    Random random = new Random();
+    for (int j = 0; j < 50; ++j) {
+      _int.add(random.nextInt(10));
+      _long.add((long) random.nextInt(10));
+      _float.add((float) (random.nextInt(100) / 10.0));
+      _double.add(random.nextInt(100) / 10.0);
+      _bool.add(random.nextBoolean());
+      INSERTION_SQLS[j] =
+          generateInsertionSQL(
+              (long) j,
+              _int.get(_int.size() - 1),
+              _long.get(_long.size() - 1),
+              _float.get(_float.size() - 1),
+              _double.get(_double.size() - 1),
+              _bool.get(_bool.size() - 1),
+              "\"magic_words\"");
+    }
+  }
+
+  private static String generateInsertionSQL(
+      long time,
+      int _int32,
+      long _int64,
+      float _float,
+      double _double,
+      boolean _bool,
+      String _text) {
+    return String.format(
+        "insert into root.sg.d1(time, s1, s2, s3, s4, s5, s6) values (%d, %d, %d, %f, %f, %s, %s);",
+        time, _int32, _int64, _float, _double, _bool ? "true" : "false", _text);
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    EnvFactory.getEnv().initBeforeClass();
+    createTimeSeries();
+    generateData();
+  }
+
+  private static void generateData() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      generateInsertionSQLS();
+      for (String dataGenerationSql : INSERTION_SQLS) {
+        statement.execute(dataGenerationSql);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+  }
+
+  private static void createTimeSeries() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      statement.execute("SET STORAGE GROUP TO root.sg");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s1 with datatype=INT32,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s2 with datatype=INT64,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s3 with datatype=FLOAT,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s4 with datatype=DOUBLE,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s5 with datatype=BOOLEAN,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s6 with datatype=TEXT,encoding=PLAIN");
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    EnvFactory.getEnv().cleanAfterClass();
+  }
+
+  /*
+   * Test compare expressions between different TSDataType
+   * */
+  @Test
+  public void testCompareWithConstant() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s1>=5, s1<=5, s1>5, s1<5, s1==5, s1!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_int.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_int.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_int.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_int.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_int.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_int.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s2>=5, s2<=5, s2>5, s2<5, s2==5, s2!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_long.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_long.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_long.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_long.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_long.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_long.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s3>=5, s3<=5, s3>5, s3<5, s3==5, s3!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_float.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_float.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_float.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_float.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_float.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_float.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s4>=5, s4<=5, s4>5, s4<5, s4==5, s4!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_double.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_double.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_double.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_double.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_double.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_double.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s5==true, s5!=true, s5==false, s5!=false from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 4, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_bool.get(i) == true, bool);

Review comment:
       Boolean.parseBoolean(resultSet.getString(2)) is not always true. It's identical to _bool.get(i)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] ericpai commented on pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
ericpai commented on pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#issuecomment-1079672793


   The tests are failure caused by other modules, please rebase and sync with the master code. @flashzxi 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] ericpai commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
ericpai commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r835706660



##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/LogicOrTransformer.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+
+public class LogicOrTransformer extends CompareBinaryTransformer {
+
+  public LogicOrTransformer(LayerPointReader leftPointReader, LayerPointReader rightPointReader) {
+    super(leftPointReader, rightPointReader);
+  }
+
+  @Override
+  protected boolean evaluateBoolean(double leftOperand, double rightOperand) {

Review comment:
       If the operator is `|` or `&`, the operands in both right and left should be boolean, not double. IMO the `|` or `&` operation on two numeric values doesn't make any sense. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] flashzxi commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
flashzxi commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r830747437



##########
File path: antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4
##########
@@ -827,7 +827,8 @@ MOD : '%';
 
 // Operators. Comparation
 
-OPERATOR_EQ : '=' | '==';
+OPERATOR_DEQ : '==';

Review comment:
       if just use OPERATOR_EQ, parser can't tell the difference between "compare equal to" and "valuation".
   for example:
   
   select ts=1 and udf(ts,"id"="value") from root.test;
   
   parser don't know whether "id"="value" means valuation or means a string equals to another string




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r830708248



##########
File path: integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSelectCompareExpressionIT.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.integration.env.EnvFactory;
+import org.apache.iotdb.itbase.category.ClusterTest;
+import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.*;
+
+@Category({LocalStandaloneTest.class, ClusterTest.class})
+public class IoTDBSelectCompareExpressionIT {
+
+  private static String[] INSERTION_SQLS;
+  private static List<Long> time = new ArrayList<>(0);
+  private static List<Integer> _int = new ArrayList<>(0);
+  private static List<Long> _long = new ArrayList<>(0);
+  private static List<Float> _float = new ArrayList<>(0);
+  private static List<Double> _double = new ArrayList<>(0);
+  private static List<Boolean> _bool = new ArrayList<>(0);
+
+  private static void generateInsertionSQLS() {
+    INSERTION_SQLS = new String[50];
+    Random random = new Random();
+    for (int j = 0; j < 50; ++j) {
+      _int.add(random.nextInt(10));
+      _long.add((long) random.nextInt(10));
+      _float.add((float) (random.nextInt(100) / 10.0));
+      _double.add(random.nextInt(100) / 10.0);
+      _bool.add(random.nextBoolean());
+      INSERTION_SQLS[j] =
+          generateInsertionSQL(
+              (long) j,
+              _int.get(_int.size() - 1),
+              _long.get(_long.size() - 1),
+              _float.get(_float.size() - 1),
+              _double.get(_double.size() - 1),
+              _bool.get(_bool.size() - 1),
+              "\"magic_words\"");
+    }
+  }
+
+  private static String generateInsertionSQL(
+      long time,
+      int _int32,
+      long _int64,
+      float _float,
+      double _double,
+      boolean _bool,
+      String _text) {
+    return String.format(
+        "insert into root.sg.d1(time, s1, s2, s3, s4, s5, s6) values (%d, %d, %d, %f, %f, %s, %s);",
+        time, _int32, _int64, _float, _double, _bool ? "true" : "false", _text);
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    EnvFactory.getEnv().initBeforeClass();
+    createTimeSeries();
+    generateData();
+  }
+
+  private static void generateData() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      generateInsertionSQLS();
+      for (String dataGenerationSql : INSERTION_SQLS) {
+        statement.execute(dataGenerationSql);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+  }
+
+  private static void createTimeSeries() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      statement.execute("SET STORAGE GROUP TO root.sg");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s1 with datatype=INT32,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s2 with datatype=INT64,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s3 with datatype=FLOAT,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s4 with datatype=DOUBLE,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s5 with datatype=BOOLEAN,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s6 with datatype=TEXT,encoding=PLAIN");
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    EnvFactory.getEnv().cleanAfterClass();
+  }
+
+  /*
+   * Test compare expressions between different TSDataType
+   * */
+  @Test
+  public void testCompareWithConstant() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s1>=5, s1<=5, s1>5, s1<5, s1==5, s1!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_int.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_int.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_int.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_int.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_int.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_int.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s2>=5, s2<=5, s2>5, s2<5, s2==5, s2!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_long.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_long.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_long.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_long.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_long.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_long.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s3>=5, s3<=5, s3>5, s3<5, s3==5, s3!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_float.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_float.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_float.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_float.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_float.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_float.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s4>=5, s4<=5, s4>5, s4<5, s4==5, s4!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_double.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_double.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_double.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_double.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_double.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_double.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s5==true, s5!=true, s5==false, s5!=false from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 4, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_bool.get(i) == true, bool);

Review comment:
       ```suggestion
           assertTrue(Boolean.parseBoolean(resultSet.getString(2)));
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls commented on pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
coveralls commented on pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#issuecomment-1070963693


   
   [![Coverage Status](https://coveralls.io/builds/47460087/badge)](https://coveralls.io/builds/47460087)
   
   Coverage decreased (-0.007%) to 65.427% when pulling **473eb33476caf8bfc002218bcee3a98f3ec335d8 on flashzxi:compareExpression** into **31108de8039e2563a8c83c5ab0549b0c0eb6d1fa on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r831694768



##########
File path: integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSelectCompareExpressionIT.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.integration.env.EnvFactory;
+import org.apache.iotdb.itbase.category.ClusterTest;
+import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.*;
+
+@Category({LocalStandaloneTest.class, ClusterTest.class})
+public class IoTDBSelectCompareExpressionIT {
+
+  private static String[] INSERTION_SQLS;
+  private static List<Long> time = new ArrayList<>(0);
+  private static List<Integer> _int = new ArrayList<>(0);
+  private static List<Long> _long = new ArrayList<>(0);
+  private static List<Float> _float = new ArrayList<>(0);
+  private static List<Double> _double = new ArrayList<>(0);
+  private static List<Boolean> _bool = new ArrayList<>(0);
+
+  private static void generateInsertionSQLS() {
+    INSERTION_SQLS = new String[50];
+    Random random = new Random();
+    for (int j = 0; j < 50; ++j) {
+      _int.add(random.nextInt(10));
+      _long.add((long) random.nextInt(10));
+      _float.add((float) (random.nextInt(100) / 10.0));
+      _double.add(random.nextInt(100) / 10.0);
+      _bool.add(random.nextBoolean());
+      INSERTION_SQLS[j] =
+          generateInsertionSQL(
+              (long) j,
+              _int.get(_int.size() - 1),
+              _long.get(_long.size() - 1),
+              _float.get(_float.size() - 1),
+              _double.get(_double.size() - 1),
+              _bool.get(_bool.size() - 1),
+              "\"magic_words\"");
+    }
+  }
+
+  private static String generateInsertionSQL(
+      long time,
+      int _int32,
+      long _int64,
+      float _float,
+      double _double,
+      boolean _bool,
+      String _text) {
+    return String.format(
+        "insert into root.sg.d1(time, s1, s2, s3, s4, s5, s6) values (%d, %d, %d, %f, %f, %s, %s);",
+        time, _int32, _int64, _float, _double, _bool ? "true" : "false", _text);
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    EnvFactory.getEnv().initBeforeClass();
+    createTimeSeries();
+    generateData();
+  }
+
+  private static void generateData() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      generateInsertionSQLS();
+      for (String dataGenerationSql : INSERTION_SQLS) {
+        statement.execute(dataGenerationSql);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+  }
+
+  private static void createTimeSeries() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      statement.execute("SET STORAGE GROUP TO root.sg");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s1 with datatype=INT32,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s2 with datatype=INT64,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s3 with datatype=FLOAT,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s4 with datatype=DOUBLE,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s5 with datatype=BOOLEAN,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s6 with datatype=TEXT,encoding=PLAIN");
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    EnvFactory.getEnv().cleanAfterClass();
+  }
+
+  /*
+   * Test compare expressions between different TSDataType
+   * */
+  @Test
+  public void testCompareWithConstant() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s1>=5, s1<=5, s1>5, s1<5, s1==5, s1!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_int.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_int.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_int.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_int.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_int.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_int.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s2>=5, s2<=5, s2>5, s2<5, s2==5, s2!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_long.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_long.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_long.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_long.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_long.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_long.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s3>=5, s3<=5, s3>5, s3<5, s3==5, s3!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_float.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_float.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_float.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_float.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_float.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_float.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s4>=5, s4<=5, s4>5, s4<5, s4==5, s4!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_double.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_double.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_double.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_double.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_double.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_double.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s5==true, s5!=true, s5==false, s5!=false from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 4, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_bool.get(i) == true, bool);

Review comment:
       Got it :D




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu merged pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu merged pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] flashzxi commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
flashzxi commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r835747722



##########
File path: docs/zh/UserGuide/Query-Data/Select-Expression.md
##########
@@ -106,6 +110,55 @@ Total line number = 5
 It costs 0.014s
 ```
 
+## 逻辑运算查询
+#### 一元逻辑运算符
+支持运算符 `!`
+
+输入数据类型:`BOOLEAN`
+
+输出数据类型:`BOOLEAN`
+
+注意:`!`的优先级很高,记得使用括号调整优先级
+
+#### 二元比较运算符
+
+支持运算符 `>`, `>=`, `<`, `<=`, `==`, `!=`
+
+输入数据类型: `INT32`, `INT64`, `FLOAT`, `DOUBLE`
+
+会将所有数据转换为`DOUBLE`类型后进行比较。
+
+返回类型:`BOOLEAN`
+
+#### 二元逻辑运算符
+
+支持运算符 AND:`and`,`&`, `&&`; OR:`or`,`|`,`||`
+
+输入数据类型:`BOOLEAN`
+
+返回类型 `BOOLEAN`
+
+注意:当某个时间戳下左操作数和右操作数都为BOOLEAN类型时,二元逻辑操作才会有输出结果

Review comment:
       Got it. ok




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#issuecomment-1079893511


   @ericpai Thanks for the reviewing :D


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] ericpai commented on pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
ericpai commented on pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#issuecomment-1078786221


   @flashzxi why the file mod of .bat and .sh are changed to 100644? They should be excutable.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] ericpai commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
ericpai commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r835742518



##########
File path: docs/zh/UserGuide/Query-Data/Select-Expression.md
##########
@@ -106,6 +110,55 @@ Total line number = 5
 It costs 0.014s
 ```
 
+## 逻辑运算查询
+#### 一元逻辑运算符
+支持运算符 `!`
+
+输入数据类型:`BOOLEAN`
+
+输出数据类型:`BOOLEAN`
+
+注意:`!`的优先级很高,记得使用括号调整优先级
+
+#### 二元比较运算符
+
+支持运算符 `>`, `>=`, `<`, `<=`, `==`, `!=`
+
+输入数据类型: `INT32`, `INT64`, `FLOAT`, `DOUBLE`
+
+会将所有数据转换为`DOUBLE`类型后进行比较。
+
+返回类型:`BOOLEAN`
+
+#### 二元逻辑运算符
+
+支持运算符 AND:`and`,`&`, `&&`; OR:`or`,`|`,`||`
+
+输入数据类型:`BOOLEAN`
+
+返回类型 `BOOLEAN`
+
+注意:当某个时间戳下左操作数和右操作数都为BOOLEAN类型时,二元逻辑操作才会有输出结果

Review comment:
       BOOLEAN -> `BOOLEAN`
   It's better to add a summary to indicate the priority of all the operators. Now the `&` and `&&` are both the same with the meaning `boolean and`. Is it `&` for `bitwise and` and `&&` for `boolean and` better? What do you think? @SteveYurongSu @flashzxi 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls commented on pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
coveralls commented on pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#issuecomment-1070963693


   
   [![Coverage Status](https://coveralls.io/builds/47460087/badge)](https://coveralls.io/builds/47460087)
   
   Coverage decreased (-0.007%) to 65.427% when pulling **473eb33476caf8bfc002218bcee3a98f3ec335d8 on flashzxi:compareExpression** into **31108de8039e2563a8c83c5ab0549b0c0eb6d1fa on apache:master**.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] SteveYurongSu commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
SteveYurongSu commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r830708628



##########
File path: integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSelectCompareExpressionIT.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.integration.env.EnvFactory;
+import org.apache.iotdb.itbase.category.ClusterTest;
+import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.*;
+
+@Category({LocalStandaloneTest.class, ClusterTest.class})
+public class IoTDBSelectCompareExpressionIT {
+
+  private static String[] INSERTION_SQLS;
+  private static List<Long> time = new ArrayList<>(0);
+  private static List<Integer> _int = new ArrayList<>(0);

Review comment:
       Generally, we don't use `_` in variables' names. Try `intValues` instead.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] ericpai commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
ericpai commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r830723311



##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/CompareOperatorTransformer.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public abstract class CompareOperatorTransformer extends ArithmeticBinaryTransformer {

Review comment:
       As `CompareOperatorTransformer` always accepts two inputs and output a boolean result. It's not good to inherit `ArithmeticBinaryTransformer` directly as it returns a double. Maybe we can define a new base class called `LogicBinaryTransformer` which returns boolean directly?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] ericpai commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
ericpai commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r835694344



##########
File path: server/src/main/java/org/apache/iotdb/db/qp/sql/IoTDBSqlVisitor.java
##########
@@ -92,15 +92,9 @@
 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;
-import org.apache.iotdb.db.query.expression.binary.DivisionExpression;
-import org.apache.iotdb.db.query.expression.binary.ModuloExpression;
-import org.apache.iotdb.db.query.expression.binary.MultiplicationExpression;
-import org.apache.iotdb.db.query.expression.binary.SubtractionExpression;
-import org.apache.iotdb.db.query.expression.unary.ConstantOperand;
-import org.apache.iotdb.db.query.expression.unary.FunctionExpression;
-import org.apache.iotdb.db.query.expression.unary.NegationExpression;
-import org.apache.iotdb.db.query.expression.unary.TimeSeriesOperand;
+import org.apache.iotdb.db.query.expression.binary.*;

Review comment:
       Please do not use wildcard imports.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/expression/unary/LogicNotExpression.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.expression.unary;
+
+import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.exception.sql.StatementAnalyzeException;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.qp.physical.crud.UDTFPlan;
+import org.apache.iotdb.db.qp.utils.WildcardsRemover;
+import org.apache.iotdb.db.query.expression.Expression;
+import org.apache.iotdb.db.query.udf.core.executor.UDTFExecutor;
+import org.apache.iotdb.db.query.udf.core.layer.*;

Review comment:
       Please do not use wildcard imports.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/BinaryTransformer.java
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+import java.io.IOException;
+
+public abstract class BinaryTransformer extends Transformer {
+
+  private final LayerPointReader leftPointReader;
+  private final LayerPointReader rightPointReader;
+
+  protected BinaryTransformer(LayerPointReader leftPointReader, LayerPointReader rightPointReader) {
+    this.leftPointReader = leftPointReader;
+    this.rightPointReader = rightPointReader;
+  }
+
+  @Override
+  public boolean isConstantPointReader() {
+    return leftPointReader.isConstantPointReader() && rightPointReader.isConstantPointReader();
+  }
+
+  @Override
+  protected boolean cacheValue() throws QueryProcessException, IOException {
+    if (!leftPointReader.next() || !rightPointReader.next()) {
+      return false;
+    }
+    if (!cacheTime()) {
+      return false;
+    }
+    if (leftPointReader.isCurrentNull() || rightPointReader.isCurrentNull()) {
+      currentNull = true;
+    } else {
+      switch (getDataType()) {
+        case DOUBLE:
+          cachedDouble =
+              evaluateDouble(
+                  castCurrentValueToDoubleOperand(leftPointReader),
+                  castCurrentValueToDoubleOperand(rightPointReader));
+          break;
+        case BOOLEAN:
+          cachedBoolean =
+              evaluateBoolean(
+                  castCurrentValueToDoubleOperand(leftPointReader),
+                  castCurrentValueToDoubleOperand(rightPointReader));
+      }
+    }
+    leftPointReader.readyForNext();
+    rightPointReader.readyForNext();
+    return true;
+  }
+
+  /**
+   * finds the smallest, unconsumed timestamp that exists in both {@code leftPointReader} and {@code
+   * rightPointReader} and then caches the timestamp in {@code cachedTime}.
+   *
+   * @return true if there has a timestamp that meets the requirements
+   */
+  private boolean cacheTime() throws IOException, QueryProcessException {
+    if (leftPointReader.isConstantPointReader() && rightPointReader.isConstantPointReader()) {
+      return true;
+    }
+    if (leftPointReader.isConstantPointReader()) {
+      cachedTime = rightPointReader.currentTime();
+      return true;
+    }
+    if (rightPointReader.isConstantPointReader()) {
+      cachedTime = leftPointReader.currentTime();
+      return true;
+    }
+
+    long leftTime = leftPointReader.currentTime();
+    long rightTime = rightPointReader.currentTime();
+
+    while (leftTime != rightTime) {
+      if (leftTime < rightTime) {
+        leftPointReader.readyForNext();
+        if (!leftPointReader.next()) {
+          return false;
+        }
+        leftTime = leftPointReader.currentTime();
+      } else {
+        rightPointReader.readyForNext();
+        if (!rightPointReader.next()) {
+          return false;
+        }
+        rightTime = rightPointReader.currentTime();
+      }
+    }
+
+    // leftTime == rightTime
+    cachedTime = leftTime;
+    return true;
+  }
+
+  protected double evaluateDouble(double leftOperand, double rightOperand) {
+    return 0.0;
+  }
+
+  protected boolean evaluateBoolean(double leftOperand, double rightOperand) {
+    return false;
+  }
+
+  private static double castCurrentValueToDoubleOperand(LayerPointReader layerPointReader)
+      throws IOException, QueryProcessException {
+    switch (layerPointReader.getDataType()) {
+      case INT32:
+        return layerPointReader.currentInt();
+      case INT64:
+        return layerPointReader.currentLong();
+      case FLOAT:
+        return layerPointReader.currentFloat();
+      case DOUBLE:
+        return layerPointReader.currentDouble();
+      case BOOLEAN:
+        return layerPointReader.currentBoolean() ? 1.0 : 0.0;
+      default:
+        throw new QueryProcessException(
+            "Unsupported data type: " + layerPointReader.getDataType().toString());
+    }
+  }
+
+  @Override
+  public TSDataType getDataType() {

Review comment:
       I think this one should also be abstract as the sub-classes may return `DOUBLE` or `BOOLEAN`.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/BinaryTransformer.java
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+import java.io.IOException;
+
+public abstract class BinaryTransformer extends Transformer {
+
+  private final LayerPointReader leftPointReader;
+  private final LayerPointReader rightPointReader;
+
+  protected BinaryTransformer(LayerPointReader leftPointReader, LayerPointReader rightPointReader) {
+    this.leftPointReader = leftPointReader;
+    this.rightPointReader = rightPointReader;
+  }
+
+  @Override
+  public boolean isConstantPointReader() {
+    return leftPointReader.isConstantPointReader() && rightPointReader.isConstantPointReader();
+  }
+
+  @Override
+  protected boolean cacheValue() throws QueryProcessException, IOException {
+    if (!leftPointReader.next() || !rightPointReader.next()) {
+      return false;
+    }
+    if (!cacheTime()) {
+      return false;
+    }
+    if (leftPointReader.isCurrentNull() || rightPointReader.isCurrentNull()) {
+      currentNull = true;
+    } else {
+      switch (getDataType()) {
+        case DOUBLE:
+          cachedDouble =
+              evaluateDouble(
+                  castCurrentValueToDoubleOperand(leftPointReader),
+                  castCurrentValueToDoubleOperand(rightPointReader));
+          break;
+        case BOOLEAN:
+          cachedBoolean =
+              evaluateBoolean(
+                  castCurrentValueToDoubleOperand(leftPointReader),
+                  castCurrentValueToDoubleOperand(rightPointReader));
+      }
+    }
+    leftPointReader.readyForNext();
+    rightPointReader.readyForNext();
+    return true;
+  }
+
+  /**
+   * finds the smallest, unconsumed timestamp that exists in both {@code leftPointReader} and {@code
+   * rightPointReader} and then caches the timestamp in {@code cachedTime}.
+   *
+   * @return true if there has a timestamp that meets the requirements
+   */
+  private boolean cacheTime() throws IOException, QueryProcessException {
+    if (leftPointReader.isConstantPointReader() && rightPointReader.isConstantPointReader()) {
+      return true;
+    }
+    if (leftPointReader.isConstantPointReader()) {
+      cachedTime = rightPointReader.currentTime();
+      return true;
+    }
+    if (rightPointReader.isConstantPointReader()) {
+      cachedTime = leftPointReader.currentTime();
+      return true;
+    }
+
+    long leftTime = leftPointReader.currentTime();
+    long rightTime = rightPointReader.currentTime();
+
+    while (leftTime != rightTime) {
+      if (leftTime < rightTime) {
+        leftPointReader.readyForNext();
+        if (!leftPointReader.next()) {
+          return false;
+        }
+        leftTime = leftPointReader.currentTime();
+      } else {
+        rightPointReader.readyForNext();
+        if (!rightPointReader.next()) {
+          return false;
+        }
+        rightTime = rightPointReader.currentTime();
+      }
+    }
+
+    // leftTime == rightTime
+    cachedTime = leftTime;
+    return true;
+  }
+
+  protected double evaluateDouble(double leftOperand, double rightOperand) {
+    return 0.0;
+  }
+
+  protected boolean evaluateBoolean(double leftOperand, double rightOperand) {

Review comment:
       Could `evaluateBoolean` accept two boolean parameters instead of double?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] flashzxi commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
flashzxi commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r835702894



##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/BinaryTransformer.java
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+import java.io.IOException;
+
+public abstract class BinaryTransformer extends Transformer {
+
+  private final LayerPointReader leftPointReader;
+  private final LayerPointReader rightPointReader;
+
+  protected BinaryTransformer(LayerPointReader leftPointReader, LayerPointReader rightPointReader) {
+    this.leftPointReader = leftPointReader;
+    this.rightPointReader = rightPointReader;
+  }
+
+  @Override
+  public boolean isConstantPointReader() {
+    return leftPointReader.isConstantPointReader() && rightPointReader.isConstantPointReader();
+  }
+
+  @Override
+  protected boolean cacheValue() throws QueryProcessException, IOException {
+    if (!leftPointReader.next() || !rightPointReader.next()) {
+      return false;
+    }
+    if (!cacheTime()) {
+      return false;
+    }
+    if (leftPointReader.isCurrentNull() || rightPointReader.isCurrentNull()) {
+      currentNull = true;
+    } else {
+      switch (getDataType()) {
+        case DOUBLE:
+          cachedDouble =
+              evaluateDouble(
+                  castCurrentValueToDoubleOperand(leftPointReader),
+                  castCurrentValueToDoubleOperand(rightPointReader));
+          break;
+        case BOOLEAN:
+          cachedBoolean =
+              evaluateBoolean(
+                  castCurrentValueToDoubleOperand(leftPointReader),
+                  castCurrentValueToDoubleOperand(rightPointReader));
+      }
+    }
+    leftPointReader.readyForNext();
+    rightPointReader.readyForNext();
+    return true;
+  }
+
+  /**
+   * finds the smallest, unconsumed timestamp that exists in both {@code leftPointReader} and {@code
+   * rightPointReader} and then caches the timestamp in {@code cachedTime}.
+   *
+   * @return true if there has a timestamp that meets the requirements
+   */
+  private boolean cacheTime() throws IOException, QueryProcessException {
+    if (leftPointReader.isConstantPointReader() && rightPointReader.isConstantPointReader()) {
+      return true;
+    }
+    if (leftPointReader.isConstantPointReader()) {
+      cachedTime = rightPointReader.currentTime();
+      return true;
+    }
+    if (rightPointReader.isConstantPointReader()) {
+      cachedTime = leftPointReader.currentTime();
+      return true;
+    }
+
+    long leftTime = leftPointReader.currentTime();
+    long rightTime = rightPointReader.currentTime();
+
+    while (leftTime != rightTime) {
+      if (leftTime < rightTime) {
+        leftPointReader.readyForNext();
+        if (!leftPointReader.next()) {
+          return false;
+        }
+        leftTime = leftPointReader.currentTime();
+      } else {
+        rightPointReader.readyForNext();
+        if (!rightPointReader.next()) {
+          return false;
+        }
+        rightTime = rightPointReader.currentTime();
+      }
+    }
+
+    // leftTime == rightTime
+    cachedTime = leftTime;
+    return true;
+  }
+
+  protected double evaluateDouble(double leftOperand, double rightOperand) {
+    return 0.0;
+  }
+
+  protected boolean evaluateBoolean(double leftOperand, double rightOperand) {

Review comment:
       for compare expression like `a>b`, `a` `b` are both double. Only in logic `and` and `or`, rightOperand and rightOperand are boolean




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] ericpai commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
ericpai commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r830722121



##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/CompareGreaterEqualTransformer.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+
+public class CompareGreaterEqualTransformer extends CompareOperatorTransformer {
+
+  public CompareGreaterEqualTransformer(
+      LayerPointReader leftPointReader, LayerPointReader rightPointReader) {
+    super(leftPointReader, rightPointReader);
+  }
+
+  @Override
+  protected double evaluate(double leftOperand, double rightOperand) {
+    return leftOperand >= rightOperand ? 1.0 : 0.0;

Review comment:
       It's not suggested that use `>=` to compare double directly, as there may be precision loss. 
   Please reference the review comment of `CompareOperatorTransformer` and change like this
   
   ```suggestion
   protected boolean evaluate(boolean leftOperand, boolean rightOperand) {
         return Double.compare(leftOperand, rightOperand) >= 0;
   }

##########
File path: antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4
##########
@@ -827,7 +827,8 @@ MOD : '%';
 
 // Operators. Comparation
 
-OPERATOR_EQ : '=' | '==';
+OPERATOR_DEQ : '==';

Review comment:
       What's the real difference between `==` and `=` ? Are they alias mutually?

##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/LogicNotTransformer.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+import java.io.IOException;
+
+public class LogicNotTransformer extends Transformer {
+  private final LayerPointReader layerPointReader;
+
+  public LogicNotTransformer(LayerPointReader layerPointReader) {
+    this.layerPointReader = layerPointReader;
+  }
+
+  @Override
+  public boolean isConstantPointReader() {
+    return layerPointReader.isConstantPointReader();
+  }
+
+  @Override
+  protected boolean cacheValue() throws QueryProcessException, IOException {
+    if (!layerPointReader.next()) {
+      return false;
+    }
+    cachedTime = layerPointReader.currentTime();
+    if (layerPointReader.isCurrentNull()) {
+      currentNull = true;
+    } else {
+      switch (layerPointReader.getDataType()) {
+        case INT32:
+          cachedBoolean = layerPointReader.currentInt() == 0;
+          break;
+        case INT64:
+          cachedBoolean = layerPointReader.currentLong() == 0L;
+          break;
+        case FLOAT:
+          cachedBoolean = layerPointReader.currentFloat() == 0.0f;
+          break;
+        case DOUBLE:
+          cachedBoolean = layerPointReader.currentDouble() == 0.0;

Review comment:
       Use `Double.compare` instead.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/LogicNotTransformer.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.exception.query.QueryProcessException;
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+import java.io.IOException;
+
+public class LogicNotTransformer extends Transformer {
+  private final LayerPointReader layerPointReader;
+
+  public LogicNotTransformer(LayerPointReader layerPointReader) {
+    this.layerPointReader = layerPointReader;
+  }
+
+  @Override
+  public boolean isConstantPointReader() {
+    return layerPointReader.isConstantPointReader();
+  }
+
+  @Override
+  protected boolean cacheValue() throws QueryProcessException, IOException {
+    if (!layerPointReader.next()) {
+      return false;
+    }
+    cachedTime = layerPointReader.currentTime();
+    if (layerPointReader.isCurrentNull()) {
+      currentNull = true;
+    } else {
+      switch (layerPointReader.getDataType()) {
+        case INT32:
+          cachedBoolean = layerPointReader.currentInt() == 0;
+          break;
+        case INT64:
+          cachedBoolean = layerPointReader.currentLong() == 0L;
+          break;
+        case FLOAT:
+          cachedBoolean = layerPointReader.currentFloat() == 0.0f;

Review comment:
       Use `Float.compare` instead.

##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/CompareEqualToTransformer.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+
+public class CompareEqualToTransformer extends CompareOperatorTransformer {
+
+  public CompareEqualToTransformer(
+      LayerPointReader leftPointReader, LayerPointReader rightPointReader) {
+    super(leftPointReader, rightPointReader);
+  }
+
+  @Override
+  protected double evaluate(double leftOperand, double rightOperand) {
+    return leftOperand == rightOperand ? 1.0 : 0.0;

Review comment:
       It's not suggested that use `==` to compare double directly, as there may be precision loss. 
   Please reference the review comment of `CompareOperatorTransformer` and change like this
   
   ```suggestion
   protected boolean evaluate(boolean leftOperand, boolean rightOperand) {
         return Double.compare(leftOperand, rightOperand) == 0;
   }

##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/CompareGreaterThanTransformer.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+
+public class CompareGreaterThanTransformer extends CompareOperatorTransformer {
+
+  public CompareGreaterThanTransformer(
+      LayerPointReader leftPointReader, LayerPointReader rightPointReader) {
+    super(leftPointReader, rightPointReader);
+  }
+
+  @Override
+  protected double evaluate(double leftOperand, double rightOperand) {
+    return leftOperand > rightOperand ? 1.0 : 0.0;

Review comment:
       Same as above.
   
   ```suggestion
   protected boolean evaluate(boolean leftOperand, boolean rightOperand) {
         return Double.compare(leftOperand, rightOperand) == 1;
   }
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/CompareNonEqualTransformer.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+
+public class CompareNonEqualTransformer extends CompareOperatorTransformer {
+
+  public CompareNonEqualTransformer(
+      LayerPointReader leftPointReader, LayerPointReader rightPointReader) {
+    super(leftPointReader, rightPointReader);
+  }
+
+  @Override
+  protected double evaluate(double leftOperand, double rightOperand) {
+    return leftOperand != rightOperand ? 1.0 : 0.0;

Review comment:
       Same as above.
   
   ```suggestion
   protected boolean evaluate(boolean leftOperand, boolean rightOperand) {
         return Double.compare(leftOperand, rightOperand) != 0;
   }
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/CompareLessEqualTransformer.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+
+public class CompareLessEqualTransformer extends CompareOperatorTransformer {
+
+  public CompareLessEqualTransformer(
+      LayerPointReader leftPointReader, LayerPointReader rightPointReader) {
+    super(leftPointReader, rightPointReader);
+  }
+
+  @Override
+  protected double evaluate(double leftOperand, double rightOperand) {
+    return leftOperand <= rightOperand ? 1.0 : 0.0;

Review comment:
       Same as above.
   
   ```suggestion
   protected boolean evaluate(boolean leftOperand, boolean rightOperand) {
         return Double.compare(leftOperand, rightOperand)  <= 0;
   }
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/CompareOperatorTransformer.java
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public abstract class CompareOperatorTransformer extends ArithmeticBinaryTransformer {

Review comment:
       As `CompareOperatorTransformer` always accepts two inputs and output a boolean result. It's not good to inherit `ArithmeticBinaryTransformer` directly as it's return double. Maybe we can define a new base class called `LogicBinaryTransformer` which returns boolean directly?

##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/CompareLessThanTransformer.java
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+
+public class CompareLessThanTransformer extends CompareOperatorTransformer {
+
+  public CompareLessThanTransformer(
+      LayerPointReader leftPointReader, LayerPointReader rightPointReader) {
+    super(leftPointReader, rightPointReader);
+  }
+
+  @Override
+  protected double evaluate(double leftOperand, double rightOperand) {
+    return leftOperand < rightOperand ? 1.0 : 0.0;

Review comment:
       Same as above.
   
   ```suggestion
   protected boolean evaluate(boolean leftOperand, boolean rightOperand) {
         return Double.compare(leftOperand, rightOperand) < 0;
   }
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/LogicAndTransformer.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+
+public class LogicAndTransformer extends CompareOperatorTransformer {
+
+  public LogicAndTransformer(LayerPointReader leftPointReader, LayerPointReader rightPointReader) {
+    super(leftPointReader, rightPointReader);
+  }
+
+  @Override
+  protected double evaluate(double leftOperand, double rightOperand) {
+    return (leftOperand == 1.0) && (rightOperand == 1.0) ? 1.0 : 0.0;

Review comment:
       Please reference the review comment of `CompareOperatorTransformer` and change like this
   
   ```suggestion
   protected boolean evaluate(boolean leftOperand, boolean rightOperand) {
         return leftOperand && rightOperand;
   }
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/query/udf/core/transformer/LogicOrTransformer.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.udf.core.transformer;
+
+import org.apache.iotdb.db.query.udf.core.reader.LayerPointReader;
+
+public class LogicOrTransformer extends CompareOperatorTransformer {
+
+  public LogicOrTransformer(LayerPointReader leftPointReader, LayerPointReader rightPointReader) {
+    super(leftPointReader, rightPointReader);
+  }
+
+  @Override
+  protected double evaluate(double leftOperand, double rightOperand) {

Review comment:
       Please reference the review comment of `CompareOperatorTransformer` and change like this
   
   ```suggestion
   protected boolean evaluate(boolean leftOperand, boolean rightOperand) {
         return leftOperand || rightOperand;
   }




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] ericpai commented on a change in pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
ericpai commented on a change in pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#discussion_r830748559



##########
File path: integration/src/test/java/org/apache/iotdb/db/integration/IoTDBSelectCompareExpressionIT.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.integration.env.EnvFactory;
+import org.apache.iotdb.itbase.category.ClusterTest;
+import org.apache.iotdb.itbase.category.LocalStandaloneTest;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import static org.junit.Assert.*;
+
+@Category({LocalStandaloneTest.class, ClusterTest.class})
+public class IoTDBSelectCompareExpressionIT {
+
+  private static String[] INSERTION_SQLS;
+  private static List<Long> time = new ArrayList<>(0);
+  private static List<Integer> _int = new ArrayList<>(0);
+  private static List<Long> _long = new ArrayList<>(0);
+  private static List<Float> _float = new ArrayList<>(0);
+  private static List<Double> _double = new ArrayList<>(0);
+  private static List<Boolean> _bool = new ArrayList<>(0);
+
+  private static void generateInsertionSQLS() {
+    INSERTION_SQLS = new String[50];
+    Random random = new Random();
+    for (int j = 0; j < 50; ++j) {
+      _int.add(random.nextInt(10));
+      _long.add((long) random.nextInt(10));
+      _float.add((float) (random.nextInt(100) / 10.0));
+      _double.add(random.nextInt(100) / 10.0);
+      _bool.add(random.nextBoolean());
+      INSERTION_SQLS[j] =
+          generateInsertionSQL(
+              (long) j,
+              _int.get(_int.size() - 1),
+              _long.get(_long.size() - 1),
+              _float.get(_float.size() - 1),
+              _double.get(_double.size() - 1),
+              _bool.get(_bool.size() - 1),
+              "\"magic_words\"");
+    }
+  }
+
+  private static String generateInsertionSQL(
+      long time,
+      int _int32,
+      long _int64,
+      float _float,
+      double _double,
+      boolean _bool,
+      String _text) {
+    return String.format(
+        "insert into root.sg.d1(time, s1, s2, s3, s4, s5, s6) values (%d, %d, %d, %f, %f, %s, %s);",
+        time, _int32, _int64, _float, _double, _bool ? "true" : "false", _text);
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    EnvFactory.getEnv().initBeforeClass();
+    createTimeSeries();
+    generateData();
+  }
+
+  private static void generateData() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      generateInsertionSQLS();
+      for (String dataGenerationSql : INSERTION_SQLS) {
+        statement.execute(dataGenerationSql);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+  }
+
+  private static void createTimeSeries() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      statement.execute("SET STORAGE GROUP TO root.sg");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s1 with datatype=INT32,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s2 with datatype=INT64,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s3 with datatype=FLOAT,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s4 with datatype=DOUBLE,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s5 with datatype=BOOLEAN,encoding=PLAIN");
+      statement.execute("CREATE TIMESERIES root.sg.d1.s6 with datatype=TEXT,encoding=PLAIN");
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    EnvFactory.getEnv().cleanAfterClass();
+  }
+
+  /*
+   * Test compare expressions between different TSDataType
+   * */
+  @Test
+  public void testCompareWithConstant() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s1>=5, s1<=5, s1>5, s1<5, s1==5, s1!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_int.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_int.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_int.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_int.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_int.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_int.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s2>=5, s2<=5, s2>5, s2<5, s2==5, s2!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_long.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_long.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_long.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_long.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_long.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_long.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s3>=5, s3<=5, s3>5, s3<5, s3==5, s3!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_float.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_float.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_float.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_float.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_float.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_float.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s4>=5, s4<=5, s4>5, s4<5, s4==5, s4!=5 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_double.get(i) >= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_double.get(i) <= 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_double.get(i) > 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_double.get(i) < 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals(_double.get(i) == 5, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals(_double.get(i) != 5, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery("select s5==true, s5!=true, s5==false, s5!=false from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 4, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_bool.get(i) == true, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_bool.get(i) != true, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_bool.get(i) == false, bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_bool.get(i) != false, bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+  }
+
+  @Test
+  public void testCompareDifferentType() {
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery(
+              "select s1>=s2, s1<=s2, s1>s3, s1<s3, s1==s4, s1!=s4 from root.sg.d1");
+      int columnCount = resultSet.getMetaData().getColumnCount();
+      assertEquals(1 + 6, columnCount);
+
+      boolean bool;
+      for (int i = 0; i < time.size(); ++i) {
+        resultSet.next();
+
+        bool = Boolean.parseBoolean(resultSet.getString(2));
+        assertEquals(_int.get(i) >= _long.get(i), bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(3));
+        assertEquals(_int.get(i) <= _long.get(i), bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(4));
+        assertEquals(_int.get(i) > _float.get(i), bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(5));
+        assertEquals(_int.get(i) < _float.get(i), bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(6));
+        assertEquals((double) _int.get(i) == _double.get(i), bool);
+
+        bool = Boolean.parseBoolean(resultSet.getString(7));
+        assertEquals((double) _int.get(i) != _double.get(i), bool);
+      }
+    } catch (SQLException throwable) {
+      fail(throwable.getMessage());
+    }
+
+    try (Connection connection = EnvFactory.getEnv().getConnection();
+        Statement statement = connection.createStatement()) {
+      ResultSet resultSet =
+          statement.executeQuery(
+              "select s2>=s3, s2<=s3, s2>s4, s2<s4, s2==s1, s2!=s1 from root.sg.d1");

Review comment:
       Could you add a more complex test case like 'select s1 * 2 <= s2 + 1 * 10' and so on?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] ericpai commented on pull request #5273: [IOTDB-2679] Support logical operators in select clauses

Posted by GitBox <gi...@apache.org>.
ericpai commented on pull request #5273:
URL: https://github.com/apache/iotdb/pull/5273#issuecomment-1073466311


   > ## Description
   > Add some functions to support select logical expression in select clauses.
   > 
   > ### Details
   > Now you can select logical expressions like a>b, fore example
   > 
   > ```sql
   > select a, a>10, !(a<20) && a<30 from root.test;
   > ```
   > 
   > **output**:
   > 
   > ```
   > IoTDB> select a, a>10, !(a<20) && a<30 from root.test;
   > +-----------------------------+-----------+----------------+--------------------------------------+
   > |                         Time|root.test.a|root.test.a > 10|!root.test.a < 20 & (root.test.a < 30)|
   > +-----------------------------+-----------+----------------+--------------------------------------+
   > |1970-01-01T08:00:00.001+08:00|         23|            true|                                  true|
   > |1970-01-01T08:00:00.002+08:00|         33|            true|                                 false|
   > |1970-01-01T08:00:00.004+08:00|         13|            true|                                 false|
   > |1970-01-01T08:00:00.005+08:00|         26|            true|                                  true|
   > |1970-01-01T08:00:00.008+08:00|          1|           false|                                 false|
   > |1970-01-01T08:00:00.010+08:00|         23|            true|                                  true|
   > +-----------------------------+-----------+----------------+--------------------------------------+
   > ```
   > 
   > ```
   > 
   > ```
   
   @flashzxi Could you add some docs about this excellent feature in English and Chinese :)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org