You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by lt...@apache.org on 2020/02/05 08:20:05 UTC

[incubator-iotdb] branch master updated: [IOTDB-448]Add IN operation (#763)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new db7e8eb  [IOTDB-448]Add IN operation (#763)
db7e8eb is described below

commit db7e8eb4bd31fa93f7989b20111eb50eb272eafa
Author: Tianan Li <li...@163.com>
AuthorDate: Wed Feb 5 16:19:58 2020 +0800

    [IOTDB-448]Add IN operation (#763)
    
    * add IN operation
    
    * fix sonar issues
    
    * add ut test
    
    * modify anltr4
    
    * remove valueToString
    
    * remove useless codes
    
    * remove useless import
    
    * fix sonar issues
    
    * fix sonar issues
---
 .../org/apache/iotdb/db/qp/strategy/SqlBase.g4     |   6 +
 .../apache/iotdb/db/qp/constant/SQLConstant.java   |   2 +
 .../db/qp/executor/IQueryProcessExecutor.java      |   2 -
 .../org/apache/iotdb/db/qp/logical/Operator.java   |   2 +-
 .../db/qp/logical/crud/BasicFunctionOperator.java  |  62 ++-----
 .../iotdb/db/qp/logical/crud/FilterOperator.java   |  11 +-
 .../iotdb/db/qp/logical/crud/FunctionOperator.java |   7 +
 .../iotdb/db/qp/logical/crud/InOperator.java       | 200 +++++++++++++++++++++
 .../iotdb/db/qp/strategy/LogicalGenerator.java     |  83 ++++++---
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    |   2 +-
 .../qp/strategy/optimizer/ConcatPathOptimizer.java |   6 +-
 .../qp/strategy/optimizer/DnfFilterOptimizer.java  |   2 +-
 .../db/qp/strategy/optimizer/IFilterOptimizer.java |   3 +-
 .../qp/strategy/optimizer/RemoveNotOptimizer.java  |  15 +-
 .../groupby/GroupByWithValueFilterDataSet.java     |   3 +-
 .../groupby/GroupByWithoutValueFilterDataSet.java  |   1 -
 .../receiver/recover/ISyncReceiverLogAnalyzer.java |   1 -
 .../iotdb/db/integration/IoTDBQueryDemoIT.java     | 134 +++++++++++++-
 .../apache/iotdb/db/qp/plan/PhysicalPlanTest.java  |  40 +++++
 .../read/expression/impl/GlobalTimeExpression.java |   2 +-
 .../expression/impl/SingleSeriesExpression.java    |   2 +-
 .../iotdb/tsfile/read/filter/GroupByFilter.java    |   2 +-
 .../iotdb/tsfile/read/filter/TimeFilter.java       |  16 ++
 .../iotdb/tsfile/read/filter/ValueFilter.java      |  16 ++
 .../tsfile/read/filter/basic/BinaryFilter.java     |   2 +-
 .../iotdb/tsfile/read/filter/basic/Filter.java     |   2 +-
 .../tsfile/read/filter/basic/UnaryFilter.java      |   3 +-
 .../tsfile/read/filter/factory/FilterFactory.java  |   2 -
 .../read/filter/factory/FilterSerializeId.java     |   2 +-
 .../tsfile/read/filter/operator/AndFilter.java     |   4 +-
 .../iotdb/tsfile/read/filter/operator/Eq.java      |   6 +-
 .../iotdb/tsfile/read/filter/operator/Gt.java      |  13 +-
 .../iotdb/tsfile/read/filter/operator/GtEq.java    |  13 +-
 .../filter/operator/{NotFilter.java => In.java}    |  92 +++++-----
 .../iotdb/tsfile/read/filter/operator/Lt.java      |  13 +-
 .../iotdb/tsfile/read/filter/operator/LtEq.java    |  13 +-
 .../iotdb/tsfile/read/filter/operator/NotEq.java   |  13 +-
 .../tsfile/read/filter/operator/NotFilter.java     |   4 +-
 .../tsfile/read/filter/operator/OrFilter.java      |   4 +-
 39 files changed, 585 insertions(+), 221 deletions(-)

diff --git a/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4 b/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
index 24fa59a..ba925a5 100644
--- a/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
+++ b/server/src/main/antlr4/org/apache/iotdb/db/qp/strategy/SqlBase.g4
@@ -137,9 +137,13 @@ andExpression
 
 predicate
     : (TIME | TIMESTAMP | suffixPath | prefixPath) comparisonOperator constant
+    | (TIME | TIMESTAMP | suffixPath | prefixPath) inClause
     | OPERATOR_NOT? LR_BRACKET orExpression RR_BRACKET
     ;
 
+inClause
+    : OPERATOR_NOT? OPERATOR_IN LR_BRACKET constant (COMMA constant)* RR_BRACKET
+    ;
 
 fromClause
     : FROM prefixPath (COMMA prefixPath)*
@@ -750,6 +754,8 @@ OPERATOR_LTE : '<=';
 
 OPERATOR_NEQ : '!=' | '<>';
 
+OPERATOR_IN : I N;
+
 OPERATOR_AND
     : A N D
     | '&'
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
index 9a89fcc..f62b719 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
@@ -78,6 +78,7 @@ public class SQLConstant {
   public static final int GREATERTHANOREQUALTO = SqlBaseLexer.OPERATOR_GTE;
   public static final int GREATERTHAN = SqlBaseLexer.OPERATOR_GT;
   public static final int EQUAL_NS = SqlBaseLexer.OPERATOR_NEQ;
+  public static final int IN = SqlBaseLexer.OPERATOR_IN;
 
   public static final int TOK_SELECT = 21;
   public static final int TOK_FROM = 22;
@@ -165,6 +166,7 @@ public class SQLConstant {
     tokenNames.put(LESSTHAN, "lessthan");
     tokenNames.put(GREATERTHANOREQUALTO, "greaterthan_or_equalto");
     tokenNames.put(GREATERTHAN, "greaterthan");
+    tokenNames.put(IN, "in");
 
     tokenNames.put(TOK_SELECT, "TOK_SELECT");
     tokenNames.put(TOK_FROM, "TOK_FROM");
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/IQueryProcessExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/IQueryProcessExecutor.java
index 461498e..0893596 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/IQueryProcessExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/IQueryProcessExecutor.java
@@ -22,11 +22,9 @@ package org.apache.iotdb.db.qp.executor;
 import java.io.IOException;
 import java.sql.SQLException;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.StorageEngineException;
-import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.path.PathException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
index 50d87bc..2dfd9ad 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
@@ -67,7 +67,7 @@ public abstract class Operator {
    */
   public enum OperatorType {
     SFW, JOIN, UNION, FILTER, GROUPBY, ORDERBY, LIMIT, SELECT, SEQTABLESCAN, HASHTABLESCAN,
-    MERGEJOIN, FILEREAD, NULL, TABLESCAN, UPDATE, INSERT, BATCHINSERT, DELETE, BASIC_FUNC, QUERY, MERGEQUERY,
+    MERGEJOIN, FILEREAD, NULL, TABLESCAN, UPDATE, INSERT, BATCHINSERT, DELETE, BASIC_FUNC, IN, QUERY, MERGEQUERY,
     AGGREGATION, AUTHOR, FROM, FUNC, LOADDATA, METADATA, PROPERTY, INDEX, INDEXQUERY, FILL,
     SET_STORAGE_GROUP, CREATE_TIMESERIES, DELETE_TIMESERIES, CREATE_USER, DELETE_USER, MODIFY_PASSWORD,
     GRANT_USER_PRIVILEGE, REVOKE_USER_PRIVILEGE, GRANT_USER_ROLE, REVOKE_USER_ROLE, CREATE_ROLE,
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/BasicFunctionOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/BasicFunctionOperator.java
index 2fce08a..8895897 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/BasicFunctionOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/BasicFunctionOperator.java
@@ -39,7 +39,6 @@ import org.slf4j.LoggerFactory;
  */
 public class BasicFunctionOperator extends FunctionOperator {
 
-  protected Path path;
   protected String value;
   private Logger logger = LoggerFactory.getLogger(BasicFunctionOperator.class);
   private BasicOperatorType funcToken;
@@ -57,74 +56,51 @@ public class BasicFunctionOperator extends FunctionOperator {
     super(tokenIntType);
     operatorType = Operator.OperatorType.BASIC_FUNC;
     funcToken = BasicOperatorType.getBasicOpBySymbol(tokenIntType);
-    this.path = this.singlePath = path;
+    this.singlePath = path;
     this.value = value;
     isLeaf = true;
     isSingle = true;
   }
 
-  /**
-   * get path.
-   *
-   * @return path
-   */
-  public String getPath() {
-    return path.toString();
-  }
-
   public String getValue() {
     return value;
   }
 
-  /**
-   * set reversed token.
-   *
-   * @throws LogicalOperatorException Logical Operator Exception
-   */
-  public void setReversedTokenIntType() throws LogicalOperatorException {
+  @Override
+  public void reverseFunc() {
     int intType = SQLConstant.reverseWords.get(tokenIntType);
     setTokenIntType(intType);
     funcToken = BasicOperatorType.getBasicOpBySymbol(intType);
   }
 
   @Override
-  public Path getSinglePath() {
-    return singlePath;
-  }
-
-  @Override
-  public void setSinglePath(Path singlePath) {
-    this.path = this.singlePath = singlePath;
-  }
-
-  @Override
   protected Pair<IUnaryExpression, String> transformToSingleQueryFilter(
       IQueryProcessExecutor executor) throws LogicalOperatorException, PathException {
-    TSDataType type = executor.getSeriesType(path);
+    TSDataType type = executor.getSeriesType(singlePath);
     if (type == null) {
       throw new PathException(
-          "given seriesPath:{" + path.getFullPath() + "} don't exist in metadata");
+          "given seriesPath:{" + singlePath.getFullPath() + "} don't exist in metadata");
     }
     IUnaryExpression ret;
 
     switch (type) {
       case INT32:
-        ret = funcToken.getUnaryExpression(path, Integer.valueOf(value));
+        ret = funcToken.getUnaryExpression(singlePath, Integer.valueOf(value));
         break;
       case INT64:
-        ret = funcToken.getUnaryExpression(path, Long.valueOf(value));
+        ret = funcToken.getUnaryExpression(singlePath, Long.valueOf(value));
         break;
       case BOOLEAN:
-        ret = funcToken.getUnaryExpression(path, Boolean.valueOf(value));
+        ret = funcToken.getUnaryExpression(singlePath, Boolean.valueOf(value));
         break;
       case FLOAT:
-        ret = funcToken.getUnaryExpression(path, Float.valueOf(value));
+        ret = funcToken.getUnaryExpression(singlePath, Float.valueOf(value));
         break;
       case DOUBLE:
-        ret = funcToken.getUnaryExpression(path, Double.valueOf(value));
+        ret = funcToken.getUnaryExpression(singlePath, Double.valueOf(value));
         break;
       case TEXT:
-        ret = funcToken.getUnaryExpression(path,
+        ret = funcToken.getUnaryExpression(singlePath,
             (value.startsWith("'") && value.endsWith("'")) || (value.startsWith("\"") && value
                 .endsWith("\""))
                 ? new Binary(value.substring(1, value.length() - 1)) : new Binary(value));
@@ -133,7 +109,7 @@ public class BasicFunctionOperator extends FunctionOperator {
         throw new LogicalOperatorException(type.toString(), "");
     }
 
-    return new Pair<>(ret, path.getFullPath());
+    return new Pair<>(ret, singlePath.getFullPath());
   }
 
   @Override
@@ -142,17 +118,17 @@ public class BasicFunctionOperator extends FunctionOperator {
     for (int i = 0; i < spaceNum; i++) {
       sc.addTail("  ");
     }
-    sc.addTail(path.toString(), this.tokenSymbol, value, ", single\n");
+    sc.addTail(singlePath.toString(), this.tokenSymbol, value, ", single\n");
     return sc.toString();
   }
 
   @Override
-  public BasicFunctionOperator clone() {
+  public BasicFunctionOperator copy() {
     BasicFunctionOperator ret;
     try {
-      ret = new BasicFunctionOperator(this.tokenIntType, path.clone(), value);
+      ret = new BasicFunctionOperator(this.tokenIntType, singlePath.clone(), value);
     } catch (SQLParserException e) {
-      logger.error("error clone:", e);
+      logger.error("error copy:", e);
       return null;
     }
     ret.tokenSymbol = tokenSymbol;
@@ -163,7 +139,7 @@ public class BasicFunctionOperator extends FunctionOperator {
 
   @Override
   public String toString() {
-    return "[" + path.getFullPath() + tokenSymbol + value + "]";
+    return "[" + singlePath.getFullPath() + tokenSymbol + value + "]";
   }
 
   @Override
@@ -175,13 +151,13 @@ public class BasicFunctionOperator extends FunctionOperator {
       return false;
     }
     BasicFunctionOperator that = (BasicFunctionOperator) o;
-    return Objects.equals(path, that.path) &&
+    return Objects.equals(singlePath, that.singlePath) &&
         Objects.equals(value, that.value) &&
         funcToken == that.funcToken;
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(super.hashCode(), path, value, funcToken);
+    return Objects.hash(super.hashCode(), singlePath, value, funcToken);
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FilterOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FilterOperator.java
index b1f1903..cd63d74 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FilterOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FilterOperator.java
@@ -98,8 +98,8 @@ public class FilterOperator extends Operator implements Comparable<FilterOperato
     return singlePath;
   }
 
-  public void setSinglePath(Path path) {
-    this.singlePath = path;
+  public void setSinglePath(Path singlePath) {
+    this.singlePath = singlePath;
   }
 
   public boolean addChildOperator(FilterOperator op) {
@@ -114,7 +114,7 @@ public class FilterOperator extends Operator implements Comparable<FilterOperato
    * @return QueryFilter in TsFile
    */
   public IExpression transformToExpression(IQueryProcessExecutor executor)
-      throws QueryProcessException, LogicalOperatorException {
+      throws QueryProcessException {
     if (isSingle) {
       Pair<IUnaryExpression, String> ret = transformToSingleQueryFilter(executor);
       return ret.left;
@@ -263,8 +263,7 @@ public class FilterOperator extends Operator implements Comparable<FilterOperato
     return sc.toString();
   }
 
-  @Override
-  public FilterOperator clone() {
+  public FilterOperator copy() {
     FilterOperator ret = new FilterOperator(this.tokenIntType);
     ret.tokenSymbol = tokenSymbol;
     ret.isLeaf = isLeaf;
@@ -273,7 +272,7 @@ public class FilterOperator extends Operator implements Comparable<FilterOperato
       ret.singlePath = singlePath.clone();
     }
     for (FilterOperator filterOperator : this.childOperators) {
-      ret.addChildOperator(filterOperator.clone());
+      ret.addChildOperator(filterOperator.copy());
     }
     return ret;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FunctionOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FunctionOperator.java
index 64039bd..e1e6d8f 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FunctionOperator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FunctionOperator.java
@@ -37,6 +37,13 @@ public class FunctionOperator extends FilterOperator {
     operatorType = OperatorType.FUNC;
   }
 
+  /**
+   * reverse func.
+   */
+  public void reverseFunc() {
+    // Implemented by subclass
+  }
+
   @Override
   public boolean addChildOperator(FilterOperator op) {
     logger.error("cannot add child to leaf FilterOperator, now it's FunctionOperator");
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InOperator.java
new file mode 100644
index 0000000..bd52fb1
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/InOperator.java
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.qp.logical.crud;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.iotdb.db.exception.path.PathException;
+import org.apache.iotdb.db.exception.query.LogicalOperatorException;
+import org.apache.iotdb.db.qp.executor.IQueryProcessExecutor;
+import org.apache.iotdb.db.qp.logical.Operator;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.expression.IUnaryExpression;
+import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
+import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
+import org.apache.iotdb.tsfile.read.filter.TimeFilter;
+import org.apache.iotdb.tsfile.read.filter.ValueFilter;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.Pair;
+import org.apache.iotdb.tsfile.utils.StringContainer;
+
+/**
+ * operator 'in' 'not in'
+ */
+public class InOperator extends FunctionOperator {
+
+  private boolean not;
+  protected Set<String> values;
+
+  /**
+   * In Operator Constructor.
+   *
+   * @param tokenIntType token in Int Type
+   * @param path path
+   * @param values values
+   */
+  public InOperator(int tokenIntType, Path path, boolean not, Set<String> values) {
+    super(tokenIntType);
+    operatorType = Operator.OperatorType.IN;
+    this.singlePath = path;
+    this.values = values;
+    this.not = not;
+    isLeaf = true;
+    isSingle = true;
+  }
+
+  public Set<String> getValues() {
+    return values;
+  }
+
+  @Override
+  public void reverseFunc() {
+    not = !not;
+  }
+
+  @Override
+  protected Pair<IUnaryExpression, String> transformToSingleQueryFilter(
+      IQueryProcessExecutor executor) throws LogicalOperatorException, PathException {
+    TSDataType type = executor.getSeriesType(singlePath);
+    if (type == null) {
+      throw new PathException(
+          "given seriesPath:{" + singlePath.getFullPath() + "} don't exist in metadata");
+    }
+    IUnaryExpression ret;
+
+    switch (type) {
+      case INT32:
+        Set<Integer> integerValues = new HashSet<>();
+        for (String val : values) {
+          integerValues.add(Integer.valueOf(val));
+        }
+        ret = In.getUnaryExpression(singlePath, integerValues, not);
+        break;
+      case INT64:
+        Set<Long> longValues = new HashSet<>();
+        for (String val : values) {
+          longValues.add(Long.valueOf(val));
+        }
+        ret = In.getUnaryExpression(singlePath, longValues, not);
+        break;
+      case BOOLEAN:
+        Set<Boolean> booleanValues = new HashSet<>();
+        for (String val : values) {
+          booleanValues.add(Boolean.valueOf(val));
+        }
+        ret = In.getUnaryExpression(singlePath, booleanValues, not);
+        break;
+      case FLOAT:
+        Set<Float> floatValues = new HashSet<>();
+        for (String val : values) {
+          floatValues.add(Float.parseFloat(val));
+        }
+        ret = In.getUnaryExpression(singlePath, floatValues, not);
+        break;
+      case DOUBLE:
+        Set<Double> doubleValues = new HashSet<>();
+        for (String val : values) {
+          doubleValues.add(Double.parseDouble(val));
+        }
+        ret = In.getUnaryExpression(singlePath, doubleValues, not);
+        break;
+      case TEXT:
+        Set<Binary> binaryValues = new HashSet<>();
+        for (String val : values) {
+          binaryValues.add(
+              (val.startsWith("'") && val.endsWith("'")) || (val.startsWith("\"") && val
+                  .endsWith("\"")) ? new Binary(val.substring(1, val.length() - 1))
+                  : new Binary(val));
+        }
+        ret = In.getUnaryExpression(singlePath, binaryValues, not);
+        break;
+      default:
+        throw new LogicalOperatorException(type.toString(), "");
+    }
+
+    return new Pair<>(ret, singlePath.getFullPath());
+  }
+
+  @Override
+  public String showTree(int spaceNum) {
+    StringContainer sc = new StringContainer();
+    for (int i = 0; i < spaceNum; i++) {
+      sc.addTail("  ");
+    }
+    sc.addTail(singlePath.toString(), this.tokenSymbol, not, values, ", single\n");
+    return sc.toString();
+  }
+
+  @Override
+  public InOperator copy() {
+    InOperator ret;
+    ret = new InOperator(this.tokenIntType, singlePath.clone(), not, new HashSet<>(values));
+    ret.tokenSymbol = tokenSymbol;
+    ret.isLeaf = isLeaf;
+    ret.isSingle = isSingle;
+    return ret;
+  }
+
+  @Override
+  public String toString() {
+    List<String> valuesList = new ArrayList<>(values);
+    Collections.sort(valuesList);
+    return "[" + singlePath.getFullPath() + tokenSymbol + not + valuesList + "]";
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    InOperator that = (InOperator) o;
+    return Objects.equals(singlePath, that.singlePath) && values.containsAll(that.values)
+        && values.size() == that.values.size() && not == that.not;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode(), singlePath, not, values);
+  }
+
+  private static class In {
+
+    public static <T extends Comparable<T>> IUnaryExpression getUnaryExpression(Path path,
+        Set<T> values, boolean not) {
+      if (path.equals("time")) {
+        return new GlobalTimeExpression(TimeFilter.in((Set<Long>) values, not));
+      } else {
+        return new SingleSeriesExpression(path, ValueFilter.in(values, not));
+      }
+    }
+
+    public <T extends Comparable<T>> Filter getValueFilter(T value) {
+      return ValueFilter.notEq(value);
+    }
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
index 6c93df5..3757b33 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
@@ -23,8 +23,10 @@ import java.time.ZoneId;
 import java.util.ArrayList;
 import java.util.EnumMap;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import org.antlr.v4.runtime.tree.TerminalNode;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.runtime.SQLParserException;
@@ -35,6 +37,7 @@ import org.apache.iotdb.db.qp.logical.crud.BasicFunctionOperator;
 import org.apache.iotdb.db.qp.logical.crud.DeleteDataOperator;
 import org.apache.iotdb.db.qp.logical.crud.FilterOperator;
 import org.apache.iotdb.db.qp.logical.crud.FromOperator;
+import org.apache.iotdb.db.qp.logical.crud.InOperator;
 import org.apache.iotdb.db.qp.logical.crud.InsertOperator;
 import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
 import org.apache.iotdb.db.qp.logical.crud.SelectOperator;
@@ -88,6 +91,7 @@ import org.apache.iotdb.db.qp.strategy.SqlBaseParser.GrantUserContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.GrantWatermarkEmbeddingContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.GroupByClauseContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.GroupByDeviceClauseContext;
+import org.apache.iotdb.db.qp.strategy.SqlBaseParser.InClauseContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.InsertColumnSpecContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.InsertStatementContext;
 import org.apache.iotdb.db.qp.strategy.SqlBaseParser.InsertValuesSpecContext;
@@ -165,9 +169,6 @@ public class LogicalGenerator extends SqlBaseBaseListener {
   private SelectOperator selectOp;
   private UpdateOperator updateOp;
   private QueryOperator queryOp;
-  private boolean isAndWhereClause = false;
-  private boolean isOrWhereClause = false;
-  private boolean isNotWhereClause = false;
   private DeleteDataOperator deleteDataOp;
 
   LogicalGenerator(ZoneId zoneId) {
@@ -181,7 +182,7 @@ public class LogicalGenerator extends SqlBaseBaseListener {
   @Override
   public void enterCountTimeseries(CountTimeseriesContext ctx) {
     super.enterCountTimeseries(ctx);
-    if(ctx.INT() != null) {
+    if (ctx.INT() != null) {
       initializedOperator = new CountOperator(SQLConstant.TOK_COUNT_NODE_TIMESERIES,
           parsePrefixPath(ctx.prefixPath()), Integer.parseInt(ctx.INT().getText()));
     } else {
@@ -200,20 +201,24 @@ public class LogicalGenerator extends SqlBaseBaseListener {
   @Override
   public void enterShowDevices(ShowDevicesContext ctx) {
     super.enterShowDevices(ctx);
-    if(ctx.prefixPath() != null) {
-      initializedOperator = new ShowDevicesOperator(SQLConstant.TOK_DEVICES, parsePrefixPath(ctx.prefixPath()));
+    if (ctx.prefixPath() != null) {
+      initializedOperator = new ShowDevicesOperator(SQLConstant.TOK_DEVICES,
+          parsePrefixPath(ctx.prefixPath()));
     } else {
-      initializedOperator = new ShowDevicesOperator(SQLConstant.TOK_DEVICES, new Path(SQLConstant.ROOT));
+      initializedOperator = new ShowDevicesOperator(SQLConstant.TOK_DEVICES,
+          new Path(SQLConstant.ROOT));
     }
   }
 
   @Override
   public void enterShowChildPaths(ShowChildPathsContext ctx) {
     super.enterShowChildPaths(ctx);
-    if(ctx.prefixPath()!= null) {
-      initializedOperator = new ShowChildPathsOperator(SQLConstant.TOK_CHILD_PATHS, parsePrefixPath(ctx.prefixPath()));
+    if (ctx.prefixPath() != null) {
+      initializedOperator = new ShowChildPathsOperator(SQLConstant.TOK_CHILD_PATHS,
+          parsePrefixPath(ctx.prefixPath()));
     } else {
-      initializedOperator = new ShowChildPathsOperator(SQLConstant.TOK_CHILD_PATHS, new Path(SQLConstant.ROOT));
+      initializedOperator = new ShowChildPathsOperator(SQLConstant.TOK_CHILD_PATHS,
+          new Path(SQLConstant.ROOT));
     }
   }
 
@@ -284,11 +289,12 @@ public class LogicalGenerator extends SqlBaseBaseListener {
   @Override
   public void enterShowTimeseries(ShowTimeseriesContext ctx) {
     super.enterShowTimeseries(ctx);
-    if(ctx.prefixPath() != null) {
+    if (ctx.prefixPath() != null) {
       initializedOperator = new ShowTimeSeriesOperator(SQLConstant.TOK_TIMESERIES,
           parsePrefixPath(ctx.prefixPath()));
     } else {
-      initializedOperator = new ShowTimeSeriesOperator(SQLConstant.TOK_TIMESERIES, new Path("root"));
+      initializedOperator = new ShowTimeSeriesOperator(SQLConstant.TOK_TIMESERIES,
+          new Path("root"));
     }
   }
 
@@ -384,7 +390,7 @@ public class LogicalGenerator extends SqlBaseBaseListener {
     super.enterAlterUser(ctx);
     AuthorOperator authorOperator = new AuthorOperator(SQLConstant.TOK_AUTHOR_UPDATE_USER,
         AuthorOperator.AuthorType.UPDATE_USER);
-    if(ctx.ID() != null) {
+    if (ctx.ID() != null) {
       authorOperator.setUserName(ctx.ID().getText());
     } else {
       authorOperator.setUserName(ctx.ROOT().getText());
@@ -1110,10 +1116,8 @@ public class LogicalGenerator extends SqlBaseBaseListener {
 
   private FilterOperator parseOrExpression(OrExpressionContext ctx) {
     if (ctx.andExpression().size() == 1) {
-      isOrWhereClause = false;
       return parseAndExpression(ctx.andExpression(0));
     }
-    isOrWhereClause = true;
     FilterOperator binaryOp = new FilterOperator(SQLConstant.KW_OR);
     if (ctx.andExpression().size() > 2) {
       binaryOp.addChildOperator(parseAndExpression(ctx.andExpression(0)));
@@ -1134,10 +1138,8 @@ public class LogicalGenerator extends SqlBaseBaseListener {
 
   private FilterOperator parseAndExpression(AndExpressionContext ctx) {
     if (ctx.predicate().size() == 1) {
-      isAndWhereClause = false;
       return parsePredicate(ctx.predicate(0));
     }
-    isAndWhereClause = true;
     FilterOperator binaryOp = new FilterOperator(SQLConstant.KW_AND);
     int size = ctx.predicate().size();
     if (size > 2) {
@@ -1159,7 +1161,6 @@ public class LogicalGenerator extends SqlBaseBaseListener {
 
   private FilterOperator parsePredicate(PredicateContext ctx) {
     if (ctx.OPERATOR_NOT() != null) {
-      isNotWhereClause = true;
       FilterOperator notOp = new FilterOperator(SQLConstant.KW_NOT);
       notOp.addChildOperator(parseOrExpression(ctx.orExpression()));
       return notOp;
@@ -1167,31 +1168,55 @@ public class LogicalGenerator extends SqlBaseBaseListener {
       return parseOrExpression(ctx.orExpression());
     } else {
       Path path = null;
-      BasicFunctionOperator basic;
+      if (ctx.TIME() != null || ctx.TIMESTAMP() != null) {
+        path = new Path(SQLConstant.RESERVED_TIME);
+      }
       if (ctx.prefixPath() != null) {
         path = parsePrefixPath(ctx.prefixPath());
       }
       if (ctx.suffixPath() != null) {
         path = parseSuffixPath(ctx.suffixPath());
       }
-      if(ctx.TIME() != null || ctx.TIMESTAMP() != null) {
-        path = new Path(SQLConstant.RESERVED_TIME);
+      if (path == null) {
+        throw new SQLParserException("Path is null, please check the sql.");
+      }
+      if (ctx.inClause() != null) {
+        return parseInOperator(ctx.inClause(), path);
+      } else {
+        return parseBasicFunctionOperator(ctx, path);
       }
-      if (ctx.constant().dateExpression() != null) {
+    }
+  }
+
+  private FilterOperator parseInOperator(InClauseContext ctx, Path path) {
+    Set<String> values = new HashSet<>();
+    boolean not = ctx.OPERATOR_NOT() != null;
+    for (ConstantContext constant : ctx.constant()) {
+      if (constant.dateExpression() != null) {
         if (!path.equals(SQLConstant.RESERVED_TIME)) {
           throw new SQLParserException(path.toString(), "Date can only be used to time");
         }
-        basic = new BasicFunctionOperator(ctx.comparisonOperator().type.getType(), path,
-            Long.toString(parseDateExpression(ctx.constant().dateExpression())));
+        values.add(Long.toString(parseDateExpression(constant.dateExpression())));
       } else {
-        basic = new BasicFunctionOperator(ctx.comparisonOperator().type.getType(), path,
-            ctx.constant().getText());
+        values.add(constant.getText());
       }
-      if (!isNotWhereClause && !isAndWhereClause && !isOrWhereClause) {
-        return basic;
+    }
+    return new InOperator(ctx.OPERATOR_IN().getSymbol().getType(), path, not, values);
+  }
+
+  private FilterOperator parseBasicFunctionOperator(PredicateContext ctx, Path path) {
+    BasicFunctionOperator basic;
+    if (ctx.constant().dateExpression() != null) {
+      if (!path.equals(SQLConstant.RESERVED_TIME)) {
+        throw new SQLParserException(path.toString(), "Date can only be used to time");
       }
-      return basic;
+      basic = new BasicFunctionOperator(ctx.comparisonOperator().type.getType(), path,
+          Long.toString(parseDateExpression(ctx.constant().dateExpression())));
+    } else {
+      basic = new BasicFunctionOperator(ctx.comparisonOperator().type.getType(), path,
+          ctx.constant().getText());
     }
+    return basic;
   }
 
   private Path parseSuffixPath(SuffixPathContext ctx) {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
index 8b4d8f6..3162c30 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
@@ -353,7 +353,7 @@ public class PhysicalGenerator {
     // remove stars in fromPaths and get deviceId with deduplication
     List<String> noStarDevices = removeStarsInDeviceWithUnique(fromPaths);
     for (int i = 0; i < noStarDevices.size(); i++) {
-      FilterOperator newOperator = operator.clone();
+      FilterOperator newOperator = operator.copy();
       newOperator = concatFilterPath(noStarDevices.get(i), newOperator);
 
       deviceToFilterMap.put(noStarDevices.get(i), newOperator.transformToExpression(executor));
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
index 81e3041..a8b0f0c 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java
@@ -216,8 +216,8 @@ public class ConcatPathOptimizer implements ILogicalOptimizer {
       operator.setChildren(newFilterList);
       return operator;
     }
-    BasicFunctionOperator basicOperator = (BasicFunctionOperator) operator;
-    Path filterPath = basicOperator.getSinglePath();
+    FunctionOperator functionOperator = (FunctionOperator) operator;
+    Path filterPath = functionOperator.getSinglePath();
     // do nothing in the cases of "where time > 5" or "where root.d1.s1 > 5"
     if (SQLConstant.isReservedPath(filterPath) || filterPath.startWith(SQLConstant.ROOT)) {
       return operator;
@@ -228,7 +228,7 @@ public class ConcatPathOptimizer implements ILogicalOptimizer {
     if (noStarPaths.size() == 1) {
       // Transform "select s1 from root.car.* where s1 > 10" to
       // "select s1 from root.car.* where root.car.*.s1 > 10"
-      basicOperator.setSinglePath(noStarPaths.get(0));
+      functionOperator.setSinglePath(noStarPaths.get(0));
       return operator;
     } else {
       // Transform "select s1 from root.car.d1, root.car.d2 where s1 > 10" to
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/DnfFilterOptimizer.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/DnfFilterOptimizer.java
index b3a8c6e..160e97a 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/DnfFilterOptimizer.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/DnfFilterOptimizer.java
@@ -45,7 +45,7 @@ public class DnfFilterOptimizer implements IFilterOptimizer {
       List<FilterOperator> newChildrenList) throws LogicalOptimizeException {
     for (FilterOperator leftAndChild : leftAndChildren) {
       for (FilterOperator rightAndChild : rightAndChildren) {
-        FilterOperator r = mergeToConjunction(leftAndChild.clone(), rightAndChild.clone());
+        FilterOperator r = mergeToConjunction(leftAndChild.copy(), rightAndChild.copy());
         newChildrenList.add(r);
       }
     }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/IFilterOptimizer.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/IFilterOptimizer.java
index f38581f..c87ab37 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/IFilterOptimizer.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/IFilterOptimizer.java
@@ -18,7 +18,6 @@
  */
 package org.apache.iotdb.db.qp.strategy.optimizer;
 
-import org.apache.iotdb.db.exception.query.LogicalOperatorException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
 import org.apache.iotdb.db.qp.logical.crud.FilterOperator;
 
@@ -29,5 +28,5 @@ import org.apache.iotdb.db.qp.logical.crud.FilterOperator;
 public interface IFilterOptimizer {
 
   FilterOperator optimize(FilterOperator filter)
-      throws QueryProcessException, LogicalOperatorException;
+      throws QueryProcessException;
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/RemoveNotOptimizer.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/RemoveNotOptimizer.java
index 9b1fde7..4d31fd6 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/RemoveNotOptimizer.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/RemoveNotOptimizer.java
@@ -26,15 +26,11 @@ import java.util.List;
 import org.apache.iotdb.db.exception.query.LogicalOperatorException;
 import org.apache.iotdb.db.exception.query.LogicalOptimizeException;
 import org.apache.iotdb.db.qp.constant.SQLConstant;
-import org.apache.iotdb.db.qp.logical.crud.BasicFunctionOperator;
 import org.apache.iotdb.db.qp.logical.crud.FilterOperator;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.iotdb.db.qp.logical.crud.FunctionOperator;
 
 public class RemoveNotOptimizer implements IFilterOptimizer {
 
-  private static final Logger logger = LoggerFactory.getLogger(RemoveNotOptimizer.class);
-
   /**
    * get DNF(disjunctive normal form) for this filter operator tree. Before getDNF, this op tree
    * must be binary, in another word, each non-leaf node has exactly two children.
@@ -77,14 +73,7 @@ public class RemoveNotOptimizer implements IFilterOptimizer {
   private FilterOperator reverseFilter(FilterOperator filter) throws LogicalOperatorException {
     int tokenInt = filter.getTokenIntType();
     if (filter.isLeaf()) {
-      try {
-        ((BasicFunctionOperator) filter).setReversedTokenIntType();
-      } catch (LogicalOperatorException e) {
-        logger.error("meet error while converting BasicFunction.", e);
-        throw new LogicalOperatorException(
-            "Convert BasicFuntion to reserved meet failed: previous token: "
-                + tokenInt + "tokenName: " + SQLConstant.tokenNames.get(tokenInt));
-      }
+      ((FunctionOperator)filter).reverseFunc();
       return filter;
     }
     switch (tokenInt) {
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java
index a6c8262..b5a8510 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java
@@ -64,8 +64,7 @@ public class GroupByWithValueFilterDataSet extends GroupByEngineDataSet {
     initGroupBy(context, groupByPlan);
   }
 
-  public GroupByWithValueFilterDataSet(long queryId, GroupByPlan groupByPlan)
-      throws PathException, IOException, StorageEngineException {
+  public GroupByWithValueFilterDataSet(long queryId, GroupByPlan groupByPlan) {
     super(new QueryContext(queryId), groupByPlan);
     this.allDataReaderList = new ArrayList<>();
     this.timeStampFetchSize = IoTDBDescriptor.getInstance().getConfig().getBatchSize();
diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithoutValueFilterDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithoutValueFilterDataSet.java
index 9d9ed28..cd420a0 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithoutValueFilterDataSet.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithoutValueFilterDataSet.java
@@ -32,7 +32,6 @@ import org.apache.iotdb.db.query.reader.IPointReader;
 import org.apache.iotdb.db.query.reader.resourceRelated.OldUnseqResourceMergeReader;
 import org.apache.iotdb.db.query.reader.resourceRelated.SeqResourceIterateReader;
 import org.apache.iotdb.tsfile.file.header.PageHeader;
-import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.*;
 import org.apache.iotdb.tsfile.read.expression.IExpression;
 import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
index ffa0e8d..1bed56a 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
@@ -20,7 +20,6 @@ package org.apache.iotdb.db.sync.receiver.recover;
 
 import java.io.File;
 import java.io.IOException;
-import org.apache.iotdb.db.sync.receiver.load.FileLoader;
 import org.apache.iotdb.db.sync.receiver.load.IFileLoader;
 
 public interface ISyncReceiverLogAnalyzer {
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBQueryDemoIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBQueryDemoIT.java
index 77191a6..ee758a7 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBQueryDemoIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBQueryDemoIT.java
@@ -233,7 +233,6 @@ public class IoTDBQueryDemoIT {
           for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
             builder.append(resultSet.getString(i)).append(",");
           }
-          System.out.println(builder.toString());
           Assert.assertEquals(retArray[cnt], builder.toString());
           cnt++;
         }
@@ -251,7 +250,6 @@ public class IoTDBQueryDemoIT {
         for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
           header.append(resultSetMetaData.getColumnName(i)).append(",");
         }
-        System.out.println(header.toString());
         Assert.assertEquals(
             "Time,root.ln.wf01.wt01.status,root.ln.wf01.wt01.temperature,"
                 + "root.ln.wf02.wt02.hardware,root.ln.wf02.wt02.status,root.sgcc.wf03.wt01.status,"
@@ -270,7 +268,6 @@ public class IoTDBQueryDemoIT {
           for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
             builder.append(resultSet.getString(i)).append(",");
           }
-          System.out.println(builder.toString());
           Assert.assertEquals(retArray[cnt], builder.toString());
           cnt++;
         }
@@ -282,5 +279,136 @@ public class IoTDBQueryDemoIT {
     }
   }
 
+  @Test
+  public void InTest() throws ClassNotFoundException {
+    String[] retArray = new String[]{
+        "1509465780000,false,20.18,v1,false,false,20.18,",
+        "1509465840000,false,21.13,v1,false,false,21.13,",
+        "1509465900000,false,22.72,v1,false,false,22.72,",
+        "1509465960000,false,20.71,v1,false,false,20.71,",
+        "1509466020000,false,21.45,v1,false,false,21.45,",
+    };
+
+    Class.forName(Config.JDBC_DRIVER_NAME);
+    try (Connection connection = DriverManager
+        .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+
+      // test 1: fetchSize < limitNumber
+      statement.setFetchSize(4);
+      Assert.assertEquals(4, statement.getFetchSize());
+      boolean hasResultSet = statement.execute("select * from root where time in (1509465780000, 1509465840000, 1509465900000, 1509465960000, 1509466020000)");
+      Assert.assertTrue(hasResultSet);
+      try (ResultSet resultSet = statement.getResultSet()) {
+        ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+        StringBuilder header = new StringBuilder();
+        for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
+          header.append(resultSetMetaData.getColumnName(i)).append(",");
+        }
+        Assert.assertEquals(
+            "Time,root.ln.wf01.wt01.status,root.ln.wf01.wt01.temperature,"
+                + "root.ln.wf02.wt02.hardware,root.ln.wf02.wt02.status,root.sgcc.wf03.wt01.status,"
+                + "root.sgcc.wf03.wt01.temperature,", header.toString());
+        Assert.assertEquals(Types.TIMESTAMP, resultSetMetaData.getColumnType(1));
+        Assert.assertEquals(Types.BOOLEAN, resultSetMetaData.getColumnType(2));
+        Assert.assertEquals(Types.FLOAT, resultSetMetaData.getColumnType(3));
+        Assert.assertEquals(Types.VARCHAR, resultSetMetaData.getColumnType(4));
+        Assert.assertEquals(Types.BOOLEAN, resultSetMetaData.getColumnType(5));
+        Assert.assertEquals(Types.BOOLEAN, resultSetMetaData.getColumnType(6));
+        Assert.assertEquals(Types.FLOAT, resultSetMetaData.getColumnType(7));
+
+        int cnt = 0;
+        while (resultSet.next()) {
+          StringBuilder builder = new StringBuilder();
+          for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
+            builder.append(resultSet.getString(i)).append(",");
+          }
+          Assert.assertEquals(retArray[cnt], builder.toString());
+          cnt++;
+        }
+        Assert.assertEquals(5, cnt);
+      }
+
+      retArray = new String[]{
+          "1509465600000,true,25.96,v2,true,true,25.96,",
+          "1509465660000,true,24.36,v2,true,true,24.36,",
+          "1509465720000,false,20.09,v1,false,false,20.09,",
+          "1509466080000,false,22.58,v1,false,false,22.58,",
+          "1509466140000,false,20.98,v1,false,false,20.98,",
+      };
+      hasResultSet = statement.execute("select * from root where time not in (1509465780000, 1509465840000, 1509465900000, 1509465960000, 1509466020000)");
+      Assert.assertTrue(hasResultSet);
+      try (ResultSet resultSet = statement.getResultSet()) {
+        ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+        StringBuilder header = new StringBuilder();
+        for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
+          header.append(resultSetMetaData.getColumnName(i)).append(",");
+        }
+        Assert.assertEquals(
+            "Time,root.ln.wf01.wt01.status,root.ln.wf01.wt01.temperature,"
+                + "root.ln.wf02.wt02.hardware,root.ln.wf02.wt02.status,root.sgcc.wf03.wt01.status,"
+                + "root.sgcc.wf03.wt01.temperature,", header.toString());
+        Assert.assertEquals(Types.TIMESTAMP, resultSetMetaData.getColumnType(1));
+        Assert.assertEquals(Types.BOOLEAN, resultSetMetaData.getColumnType(2));
+        Assert.assertEquals(Types.FLOAT, resultSetMetaData.getColumnType(3));
+        Assert.assertEquals(Types.VARCHAR, resultSetMetaData.getColumnType(4));
+        Assert.assertEquals(Types.BOOLEAN, resultSetMetaData.getColumnType(5));
+        Assert.assertEquals(Types.BOOLEAN, resultSetMetaData.getColumnType(6));
+        Assert.assertEquals(Types.FLOAT, resultSetMetaData.getColumnType(7));
+
+        int cnt = 0;
+        while (resultSet.next()) {
+          StringBuilder builder = new StringBuilder();
+          for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
+            builder.append(resultSet.getString(i)).append(",");
+          }
+          Assert.assertEquals(retArray[cnt], builder.toString());
+          cnt++;
+        }
+        Assert.assertEquals(5, cnt);
+      }
+
+      retArray = new String[]{
+          "1509465780000,false,20.18,v1,false,false,20.18,",
+          "1509465960000,false,20.71,v1,false,false,20.71,",
+          "1509466080000,false,22.58,v1,false,false,22.58,",
+      };
+      hasResultSet = statement.execute("select * from root where ln.wf01.wt01.temperature in (20.18, 20.71, 22.58)");
+      Assert.assertTrue(hasResultSet);
+      try (ResultSet resultSet = statement.getResultSet()) {
+        ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
+        StringBuilder header = new StringBuilder();
+        for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
+          header.append(resultSetMetaData.getColumnName(i)).append(",");
+        }
+        Assert.assertEquals(
+            "Time,root.ln.wf01.wt01.status,root.ln.wf01.wt01.temperature,"
+                + "root.ln.wf02.wt02.hardware,root.ln.wf02.wt02.status,root.sgcc.wf03.wt01.status,"
+                + "root.sgcc.wf03.wt01.temperature,", header.toString());
+        Assert.assertEquals(Types.TIMESTAMP, resultSetMetaData.getColumnType(1));
+        Assert.assertEquals(Types.BOOLEAN, resultSetMetaData.getColumnType(2));
+        Assert.assertEquals(Types.FLOAT, resultSetMetaData.getColumnType(3));
+        Assert.assertEquals(Types.VARCHAR, resultSetMetaData.getColumnType(4));
+        Assert.assertEquals(Types.BOOLEAN, resultSetMetaData.getColumnType(5));
+        Assert.assertEquals(Types.BOOLEAN, resultSetMetaData.getColumnType(6));
+        Assert.assertEquals(Types.FLOAT, resultSetMetaData.getColumnType(7));
+
+        int cnt = 0;
+        while (resultSet.next()) {
+          StringBuilder builder = new StringBuilder();
+          for (int i = 1; i <= resultSetMetaData.getColumnCount(); i++) {
+            builder.append(resultSet.getString(i)).append(",");
+          }
+          Assert.assertEquals(retArray[cnt], builder.toString());
+          cnt++;
+        }
+        Assert.assertEquals(3, cnt);
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+      fail(e.getMessage());
+    }
+  }
+
 
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
index 7eb34b5..450fabd 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
@@ -23,6 +23,11 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.File;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.exception.metadata.MetadataException;
 import org.apache.iotdb.db.exception.query.QueryProcessException;
@@ -510,6 +515,41 @@ public class PhysicalPlanTest {
   }
 
   @Test
+  public void testInOperator() throws QueryProcessException {
+    String sqlStr = "SELECT s1 FROM root.vehicle.d1 WHERE s1 in (25, 30, 40)";
+    PhysicalPlan plan = processor.parseSQLToPhysicalPlan(sqlStr);
+    IExpression queryFilter = ((QueryPlan) plan).getExpression();
+    Set<Float> values = new HashSet<>();
+    values.add(25.0f);
+    values.add(30.0f);
+    values.add(40.0f);
+    IExpression expect = new SingleSeriesExpression(new Path("root.vehicle.d1.s1"),
+        ValueFilter.in(values, false));
+    assertEquals(expect.toString(), queryFilter.toString());
+  }
+
+  @Test
+  public void testNotInOperator() throws QueryProcessException {
+    String sqlStr = "SELECT s1 FROM root.vehicle.d1 WHERE s1 not in (25, 30, 40)";
+    PhysicalPlan plan = processor.parseSQLToPhysicalPlan(sqlStr);
+    IExpression queryFilter = ((QueryPlan) plan).getExpression();
+    Set<Float> values = new HashSet<>();
+    values.add(25.0f);
+    values.add(30.0f);
+    values.add(40.0f);
+    IExpression expect = new SingleSeriesExpression(new Path("root.vehicle.d1.s1"),
+        ValueFilter.in(values, true));
+    assertEquals(expect.toString(), queryFilter.toString());
+
+    sqlStr = "SELECT s1 FROM root.vehicle.d1 WHERE not(s1 not in (25, 30, 40))";
+    plan = processor.parseSQLToPhysicalPlan(sqlStr);
+    queryFilter = ((QueryPlan) plan).getExpression();
+    expect = new SingleSeriesExpression(new Path("root.vehicle.d1.s1"),
+        ValueFilter.in(values, false));
+    assertEquals(expect.toString(), queryFilter.toString());
+  }
+
+  @Test
   public void testGrantWatermarkEmbedding()
       throws QueryProcessException {
     String sqlStr = "GRANT WATERMARK_EMBEDDING to a,b";
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/GlobalTimeExpression.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/GlobalTimeExpression.java
index 9989ec2..8926a0e 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/GlobalTimeExpression.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/GlobalTimeExpression.java
@@ -50,7 +50,7 @@ public class GlobalTimeExpression implements IUnaryExpression, Serializable {
 
   @Override
   public IExpression clone() {
-    return new GlobalTimeExpression(filter.clone());
+    return new GlobalTimeExpression(filter.copy());
   }
 
   @Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/SingleSeriesExpression.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/SingleSeriesExpression.java
index 905eebb..7ebcaef 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/SingleSeriesExpression.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/SingleSeriesExpression.java
@@ -43,7 +43,7 @@ public class SingleSeriesExpression implements IUnaryExpression, Serializable {
 
   @Override
   public IExpression clone() {
-    return new SingleSeriesExpression(seriesPath.clone(), filter.clone());
+    return new SingleSeriesExpression(seriesPath.clone(), filter.copy());
   }
 
   @Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java
index a5a7a25..b1cf764 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java
@@ -95,7 +95,7 @@ public class GroupByFilter implements Filter, Serializable {
   }
 
   @Override
-  public Filter clone() {
+  public Filter copy() {
     return new GroupByFilter(unit, slidingStep, startTime, endTime);
   }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/TimeFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/TimeFilter.java
index d2a663a..307c694 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/TimeFilter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/TimeFilter.java
@@ -18,11 +18,13 @@
  */
 package org.apache.iotdb.tsfile.read.filter;
 
+import java.util.Set;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.factory.FilterType;
 import org.apache.iotdb.tsfile.read.filter.operator.Eq;
 import org.apache.iotdb.tsfile.read.filter.operator.Gt;
 import org.apache.iotdb.tsfile.read.filter.operator.GtEq;
+import org.apache.iotdb.tsfile.read.filter.operator.In;
 import org.apache.iotdb.tsfile.read.filter.operator.Lt;
 import org.apache.iotdb.tsfile.read.filter.operator.LtEq;
 import org.apache.iotdb.tsfile.read.filter.operator.NotEq;
@@ -30,6 +32,9 @@ import org.apache.iotdb.tsfile.read.filter.operator.NotFilter;
 
 public class TimeFilter {
 
+  private TimeFilter() {
+  }
+
   public static TimeEq eq(long value) {
     return new TimeEq(value);
   }
@@ -58,6 +63,17 @@ public class TimeFilter {
     return new TimeNotEq(value);
   }
 
+  public static TimeIn in(Set<Long> values, boolean not) {
+    return new TimeIn(values, not);
+  }
+
+  public static class TimeIn extends In {
+
+    private TimeIn(Set<Long> values, boolean not) {
+      super(values, FilterType.TIME_FILTER,not);
+    }
+  }
+
   public static class TimeEq extends Eq {
 
     private TimeEq(long value) {
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/ValueFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/ValueFilter.java
index 18b1285..5919be5 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/ValueFilter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/ValueFilter.java
@@ -18,11 +18,13 @@
  */
 package org.apache.iotdb.tsfile.read.filter;
 
+import java.util.Set;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 import org.apache.iotdb.tsfile.read.filter.factory.FilterType;
 import org.apache.iotdb.tsfile.read.filter.operator.Eq;
 import org.apache.iotdb.tsfile.read.filter.operator.Gt;
 import org.apache.iotdb.tsfile.read.filter.operator.GtEq;
+import org.apache.iotdb.tsfile.read.filter.operator.In;
 import org.apache.iotdb.tsfile.read.filter.operator.Lt;
 import org.apache.iotdb.tsfile.read.filter.operator.LtEq;
 import org.apache.iotdb.tsfile.read.filter.operator.NotEq;
@@ -30,6 +32,9 @@ import org.apache.iotdb.tsfile.read.filter.operator.NotFilter;
 
 public class ValueFilter {
 
+  private ValueFilter() {
+  }
+
   public static <T extends Comparable<T>> ValueEq<T> eq(T value) {
     return new ValueEq(value);
   }
@@ -50,6 +55,10 @@ public class ValueFilter {
     return new ValueLtEq(value);
   }
 
+  public static <T extends  Comparable<T>> ValueIn<T> in(Set<T> values, boolean not){
+    return new ValueIn(values, not);
+  }
+
   public static ValueNotFilter not(Filter filter) {
     return new ValueNotFilter(filter);
   }
@@ -58,6 +67,13 @@ public class ValueFilter {
     return new ValueNotEq(value);
   }
 
+  public static class ValueIn<T extends Comparable<T>> extends In<T> {
+
+    private ValueIn(Set<T> values, boolean not) {
+      super(values, FilterType.VALUE_FILTER, not);
+    }
+  }
+
   public static class ValueEq<T extends Comparable<T>> extends Eq<T> {
 
     private ValueEq(T value) {
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/basic/BinaryFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/basic/BinaryFilter.java
index 4099372..7320f3d 100755
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/basic/BinaryFilter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/basic/BinaryFilter.java
@@ -57,7 +57,7 @@ public abstract class BinaryFilter implements Filter, Serializable {
   }
 
   @Override
-  public abstract Filter clone();
+  public abstract Filter copy();
 
   @Override
   public void serialize(DataOutputStream outputStream) {
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/basic/Filter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/basic/Filter.java
index cfc7478..549bbe0 100755
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/basic/Filter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/basic/Filter.java
@@ -65,7 +65,7 @@ public interface Filter {
    */
   boolean containStartEndTime(long startTime, long endTime);
 
-  Filter clone();
+  Filter copy();
 
   void serialize(DataOutputStream outputStream);
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/basic/UnaryFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/basic/UnaryFilter.java
index d1f8146..48f9222 100755
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/basic/UnaryFilter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/basic/UnaryFilter.java
@@ -24,7 +24,6 @@ import java.io.Serializable;
 import java.nio.ByteBuffer;
 import java.util.Objects;
 import org.apache.iotdb.tsfile.read.filter.factory.FilterType;
-import org.apache.iotdb.tsfile.read.filter.operator.Eq;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 /**
@@ -59,7 +58,7 @@ public abstract class UnaryFilter<T extends Comparable<T>> implements Filter, Se
   public abstract String toString();
 
   @Override
-  public abstract Filter clone();
+  public abstract Filter copy();
 
   @Override
   public void serialize(DataOutputStream outputStream) {
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/factory/FilterFactory.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/factory/FilterFactory.java
index 7781e42..07f849a 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/factory/FilterFactory.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/factory/FilterFactory.java
@@ -30,8 +30,6 @@ import org.apache.iotdb.tsfile.read.filter.operator.LtEq;
 import org.apache.iotdb.tsfile.read.filter.operator.NotEq;
 import org.apache.iotdb.tsfile.read.filter.operator.NotFilter;
 import org.apache.iotdb.tsfile.read.filter.operator.OrFilter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class FilterFactory {
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/factory/FilterSerializeId.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/factory/FilterSerializeId.java
index 1b20251..a76f1fa 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/factory/FilterSerializeId.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/factory/FilterSerializeId.java
@@ -20,5 +20,5 @@
 package org.apache.iotdb.tsfile.read.filter.factory;
 
 public enum FilterSerializeId {
-  AND, EQ, GROUP_BY, GT, GTEQ, LT, LTEQ, NEQ, NOT, OR
+  AND, EQ, GROUP_BY, GT, GTEQ, LT, LTEQ, NEQ, NOT, OR, IN
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/AndFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/AndFilter.java
index 1b3c021..10ca630 100755
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/AndFilter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/AndFilter.java
@@ -65,8 +65,8 @@ public class AndFilter extends BinaryFilter {
   }
 
   @Override
-  public Filter clone() {
-    return new AndFilter(left.clone(), right.clone());
+  public Filter copy() {
+    return new AndFilter(left.copy(), right.copy());
   }
 
   @Override
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/Eq.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/Eq.java
index 9a6cc0e..75bb91f 100755
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/Eq.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/Eq.java
@@ -18,7 +18,6 @@
  */
 package org.apache.iotdb.tsfile.read.filter.operator;
 
-import java.util.Objects;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
@@ -45,7 +44,8 @@ public class Eq<T extends Comparable<T>> extends UnaryFilter<T> {
   @Override
   public boolean satisfy(Statistics statistics) {
     if (filterType == FilterType.TIME_FILTER) {
-      return ((Long) value) >= statistics.getStartTime() && ((Long) value) <= statistics.getEndTime();
+      return ((Long) value) >= statistics.getStartTime() && ((Long) value) <= statistics
+          .getEndTime();
     } else {
       if (statistics.getType() == TSDataType.TEXT || statistics.getType() == TSDataType.BOOLEAN) {
         return true;
@@ -82,7 +82,7 @@ public class Eq<T extends Comparable<T>> extends UnaryFilter<T> {
   }
 
   @Override
-  public Filter clone() {
+  public Filter copy() {
     return new Eq(value, filterType);
   }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/Gt.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/Gt.java
index 9034956..5401af9 100755
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/Gt.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/Gt.java
@@ -63,10 +63,7 @@ public class Gt<T extends Comparable<T>> extends UnaryFilter<T> {
   public boolean satisfyStartEndTime(long startTime, long endTime) {
     if (filterType == FilterType.TIME_FILTER) {
       long time = (Long) value;
-      if (time >= endTime) {
-        return false;
-      }
-      return true;
+      return time < endTime;
     } else {
       return true;
     }
@@ -76,18 +73,14 @@ public class Gt<T extends Comparable<T>> extends UnaryFilter<T> {
   public boolean containStartEndTime(long startTime, long endTime) {
     if (filterType == FilterType.TIME_FILTER) {
       long time = (Long) value;
-      if (startTime > time) {
-        return true;
-      } else {
-        return false;
-      }
+      return startTime > time;
     } else {
       return true;
     }
   }
 
   @Override
-  public Filter clone() {
+  public Filter copy() {
     return new Gt(value, filterType);
   }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/GtEq.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/GtEq.java
index 47ead82..7c3c448 100755
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/GtEq.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/GtEq.java
@@ -64,10 +64,7 @@ public class GtEq<T extends Comparable<T>> extends UnaryFilter<T> {
   public boolean satisfyStartEndTime(long startTime, long endTime) {
     if (filterType == FilterType.TIME_FILTER) {
       long time = (Long) value;
-      if (time > endTime) {
-        return false;
-      }
-      return true;
+      return time <= endTime;
     } else {
       return true;
     }
@@ -77,18 +74,14 @@ public class GtEq<T extends Comparable<T>> extends UnaryFilter<T> {
   public boolean containStartEndTime(long startTime, long endTime) {
     if (filterType == FilterType.TIME_FILTER) {
       long time = (Long) value;
-      if (startTime >= time) {
-        return true;
-      } else {
-        return false;
-      }
+      return startTime >= time;
     } else {
       return true;
     }
   }
 
   @Override
-  public Filter clone() {
+  public Filter copy() {
     return new GtEq(value, filterType);
   }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/NotFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/In.java
old mode 100755
new mode 100644
similarity index 54%
copy from tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/NotFilter.java
copy to tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/In.java
index 21993bb..1bb76d4
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/NotFilter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/In.java
@@ -20,72 +20,78 @@ package org.apache.iotdb.tsfile.read.filter.operator;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.io.Serializable;
 import java.nio.ByteBuffer;
-import java.util.Objects;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
 import org.apache.iotdb.tsfile.read.filter.basic.Filter;
-import org.apache.iotdb.tsfile.read.filter.factory.FilterFactory;
 import org.apache.iotdb.tsfile.read.filter.factory.FilterSerializeId;
+import org.apache.iotdb.tsfile.read.filter.factory.FilterType;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 /**
- * NotFilter necessary. Use InvertExpressionVisitor
+ * in clause.
+ *
+ * @param <T> comparable data type
  */
-public class NotFilter implements Filter, Serializable {
+public class In<T extends Comparable<T>> implements Filter {
+
+  private static final long serialVersionUID = 8572705136773595399L;
+
+  private Set<T> values;
 
-  private static final long serialVersionUID = 584860326604020881L;
-  private Filter that;
+  private boolean not;
 
-  public NotFilter() {
+  private FilterType filterType;
+
+  public In() {
   }
 
-  public NotFilter(Filter that) {
-    this.that = that;
+  public In(Set<T> values, FilterType filterType, boolean not) {
+    this.values = values;
+    this.filterType = filterType;
+    this.not = not;
   }
 
   @Override
   public boolean satisfy(Statistics statistics) {
-    return !that.satisfy(statistics);
+    return true;
   }
 
   @Override
   public boolean satisfy(long time, Object value) {
-    return !that.satisfy(time, value);
+    Object v = filterType == FilterType.TIME_FILTER ? time : value;
+    return this.values.contains(v) != not;
   }
 
-  /**
-   * Notice that, if the not filter only contains value filter, this method may return false, this
-   * may cause misunderstanding.
-   */
   @Override
   public boolean satisfyStartEndTime(long startTime, long endTime) {
-    return !that.satisfyStartEndTime(startTime, endTime);
+    return true;
   }
 
   @Override
   public boolean containStartEndTime(long startTime, long endTime) {
-    return !that.satisfyStartEndTime(startTime, endTime);
-  }
-
-  @Override
-  public Filter clone() {
-    return new NotFilter(that.clone());
-  }
-
-  public Filter getFilter() {
-    return this.that;
+    return true;
   }
 
   @Override
-  public String toString() {
-    return "NotFilter: " + that;
+  public Filter copy() {
+    return new In(new HashSet(values), filterType, not);
   }
 
   @Override
   public void serialize(DataOutputStream outputStream) {
     try {
       outputStream.write(getSerializeId().ordinal());
-      that.serialize(outputStream);
+      outputStream.write(filterType.ordinal());
+      ReadWriteIOUtils.write(not, outputStream);
+      outputStream.write(values.size());
+      for (T value : values) {
+        ReadWriteIOUtils.writeObject(value, outputStream);
+      }
     } catch (IOException ignored) {
       // ignored
     }
@@ -93,25 +99,23 @@ public class NotFilter implements Filter, Serializable {
 
   @Override
   public void deserialize(ByteBuffer buffer) {
-    that = FilterFactory.deserialize(buffer);
-  }
-
-  @Override
-  public FilterSerializeId getSerializeId() {
-    return FilterSerializeId.NOT;
+    filterType = FilterType.values()[buffer.get()];
+    not = ReadWriteIOUtils.readBool(buffer);
+    values = new HashSet<>();
+    for (int i = 0; i < buffer.get(); i++) {
+      values.add((T) ReadWriteIOUtils.readObject(buffer));
+    }
   }
 
   @Override
-  public boolean equals(Object obj) {
-    if (!(obj instanceof NotFilter)) {
-      return false;
-    }
-    NotFilter other = ((NotFilter) obj);
-    return this.that.equals(other.that);
+  public String toString() {
+    List<T> valueList = new ArrayList<>(values);
+    Collections.sort(valueList);
+    return filterType + " < " + "reverse: " + not + ", " + valueList;
   }
 
   @Override
-  public int hashCode() {
-    return Objects.hash(that);
+  public FilterSerializeId getSerializeId() {
+    return FilterSerializeId.IN;
   }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/Lt.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/Lt.java
index ff6ab89..6523570 100755
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/Lt.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/Lt.java
@@ -63,10 +63,7 @@ public class Lt<T extends Comparable<T>> extends UnaryFilter<T> {
   public boolean satisfyStartEndTime(long startTime, long endTime) {
     if (filterType == FilterType.TIME_FILTER) {
       long time = (Long) value;
-      if (time <= startTime) {
-        return false;
-      }
-      return true;
+      return time > startTime;
     } else {
       return true;
     }
@@ -76,18 +73,14 @@ public class Lt<T extends Comparable<T>> extends UnaryFilter<T> {
   public boolean containStartEndTime(long startTime, long endTime) {
     if (filterType == FilterType.TIME_FILTER) {
       long time = (Long) value;
-      if (endTime < time) {
-        return true;
-      } else {
-        return false;
-      }
+      return endTime < time;
     } else {
       return true;
     }
   }
 
   @Override
-  public Filter clone() {
+  public Filter copy() {
     return new Lt(value, filterType);
   }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/LtEq.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/LtEq.java
index ca37348..f36969d 100755
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/LtEq.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/LtEq.java
@@ -64,10 +64,7 @@ public class LtEq<T extends Comparable<T>> extends UnaryFilter<T> {
   public boolean satisfyStartEndTime(long startTime, long endTime) {
     if (filterType == FilterType.TIME_FILTER) {
       long time = (Long) value;
-      if (time < startTime) {
-        return false;
-      }
-      return true;
+      return time >= startTime;
     } else {
       return true;
     }
@@ -77,18 +74,14 @@ public class LtEq<T extends Comparable<T>> extends UnaryFilter<T> {
   public boolean containStartEndTime(long startTime, long endTime) {
     if (filterType == FilterType.TIME_FILTER) {
       long time = (Long) value;
-      if (endTime <= time) {
-        return true;
-      } else {
-        return false;
-      }
+      return endTime <= time;
     } else {
       return true;
     }
   }
 
   @Override
-  public Filter clone() {
+  public Filter copy() {
     return new LtEq(value, filterType);
   }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/NotEq.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/NotEq.java
index c58e788..affe96e 100755
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/NotEq.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/NotEq.java
@@ -64,10 +64,7 @@ public class NotEq<T extends Comparable<T>> extends UnaryFilter<T> {
   public boolean satisfyStartEndTime(long startTime, long endTime) {
     if (filterType == FilterType.TIME_FILTER) {
       long time = (Long) value;
-      if (time == startTime && time == endTime) {
-        return false;
-      }
-      return true;
+      return time != startTime || time != endTime;
     } else {
       return true;
     }
@@ -77,18 +74,14 @@ public class NotEq<T extends Comparable<T>> extends UnaryFilter<T> {
   public boolean containStartEndTime(long startTime, long endTime) {
     if (filterType == FilterType.TIME_FILTER) {
       long time = (Long) value;
-      if (time < startTime || time > endTime) {
-        return true;
-      } else {
-        return false;
-      }
+      return time < startTime || time > endTime;
     } else {
       return true;
     }
   }
 
   @Override
-  public Filter clone() {
+  public Filter copy() {
     return new NotEq(value, filterType);
   }
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/NotFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/NotFilter.java
index 21993bb..ae7b5e2 100755
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/NotFilter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/NotFilter.java
@@ -68,8 +68,8 @@ public class NotFilter implements Filter, Serializable {
   }
 
   @Override
-  public Filter clone() {
-    return new NotFilter(that.clone());
+  public Filter copy() {
+    return new NotFilter(that.copy());
   }
 
   public Filter getFilter() {
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/OrFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/OrFilter.java
index e82bbb1..8ad6d51 100755
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/OrFilter.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/OrFilter.java
@@ -44,8 +44,8 @@ public class OrFilter extends BinaryFilter implements Serializable {
   }
 
   @Override
-  public Filter clone() {
-    return new OrFilter(left.clone(), right.clone());
+  public Filter copy() {
+    return new OrFilter(left.copy(), right.copy());
   }
 
   @Override