You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ja...@apache.org on 2022/03/16 02:33:25 UTC

[iotdb] 01/02: add some interface

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

jackietien pushed a commit to branch xingtanzjr/mpp-query-basis
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit e7d8a198073e05264871d7ee2fdac9bf0829625e
Author: JackieTien97 <ja...@gmail.com>
AuthorDate: Tue Mar 15 21:33:04 2022 +0800

    add some interface
---
 pom.xml                                            |   5 +
 server/pom.xml                                     |   4 +
 .../iotdb/db/query/mpp/exec/Coordinator.java       |  18 ++++
 .../db/query/mpp/exec/ExecFragmentInstance.java    |  59 ++++++++++++
 .../iotdb/db/query/mpp/exec/QueryExecution.java    |  18 ++++
 .../iotdb/db/query/mpp/exec/QueryScheduler.java    |  18 ++++
 .../iotdb/db/query/mpp/exec/QueryStateMachine.java |  18 ++++
 .../ExchangeDataSource.java}                       |  26 +++---
 .../iotdb/db/query/mpp/operator/Operator.java      | 102 +++++++++++++++++++++
 .../OperatorContext.java}                          |  27 +++---
 .../SeriesScanOperator.java}                       |  20 +---
 .../query/mpp/plan/node/source/CsvSourceNode.java  |   1 -
 .../mpp/plan/node/source/SeriesAggregateNode.java  |  19 ++++
 .../query/mpp/plan/node/source/SeriesScanNode.java |  39 ++++++--
 14 files changed, 322 insertions(+), 52 deletions(-)

diff --git a/pom.xml b/pom.xml
index 82b90f5..68131dd 100644
--- a/pom.xml
+++ b/pom.xml
@@ -495,6 +495,11 @@
             </dependency>
             <dependency>
                 <groupId>io.airlift</groupId>
+                <artifactId>units</artifactId>
+                <version>1.6</version>
+            </dependency>
+            <dependency>
+                <groupId>io.airlift</groupId>
                 <artifactId>airline</artifactId>
                 <version>${airline.version}</version>
             </dependency>
diff --git a/server/pom.xml b/server/pom.xml
index 97ad1ae..6db8881 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -87,6 +87,10 @@
         </dependency>
         <dependency>
             <groupId>io.airlift</groupId>
+            <artifactId>units</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>io.airlift</groupId>
             <artifactId>airline</artifactId>
             <exclusions>
                 <exclusion>
diff --git a/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/Coordinator.java b/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/Coordinator.java
index 3c8c2b9..b76bc8b 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/Coordinator.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/Coordinator.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mpp.exec;
 
 import org.apache.iotdb.db.query.mpp.common.QueryId;
diff --git a/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/ExecFragmentInstance.java b/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/ExecFragmentInstance.java
new file mode 100644
index 0000000..a4333b8
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/ExecFragmentInstance.java
@@ -0,0 +1,59 @@
+/*
+ * 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.mpp.exec;
+
+import com.google.common.util.concurrent.ListenableFuture;
+
+import java.io.Closeable;
+import io.airlift.units.Duration;
+
+/**
+ * ExecutableFragmentInstance encapsulates some methods which are necessary for execution scheduler to run a fragment instance
+ */
+public interface ExecFragmentInstance extends Closeable {
+
+    /**
+     * Used to judge whether this fragment instance has any more data to process
+     *
+     * @return true if the FragmentInstance is done, otherwise false.
+     */
+    boolean isFinished();
+
+    /**
+     * run the fragment instance for {@param duration} time slice, the time of this run is likely not to be equal to {@param duration},
+     * the actual run time should be calculated by the caller
+     *
+     * @param duration how long should this fragment instance run
+     * @return the returned ListenableFuture<Void> is used to represent status of this processing
+     *         if isDone() return true, meaning that this fragment instance is not blocked and is ready for next processing
+     *         otherwise, meaning that this fragment instance is blocked and not ready for next processing.
+     */
+    ListenableFuture<Void> processFor(Duration duration);
+
+    /**
+     * @return the information about this Fragment Instance in String format
+     */
+    String getInfo();
+
+    /**
+     * clear resource used by this fragment instance
+     */
+    @Override
+    void close();
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/QueryExecution.java b/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/QueryExecution.java
index 708e186..f046f12 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/QueryExecution.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/QueryExecution.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mpp.exec;
 
 import org.apache.iotdb.db.query.mpp.common.Analysis;
diff --git a/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/QueryScheduler.java b/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/QueryScheduler.java
index 6c73244..e511abe 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/QueryScheduler.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/QueryScheduler.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mpp.exec;
 
 import org.apache.iotdb.db.query.mpp.plan.FragmentInstance;
diff --git a/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/QueryStateMachine.java b/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/QueryStateMachine.java
index c5effe2..10ba210 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/QueryStateMachine.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/mpp/exec/QueryStateMachine.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mpp.exec;
 
 /**
diff --git a/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/CsvSourceNode.java b/server/src/main/java/org/apache/iotdb/db/query/mpp/operator/ExchangeDataSource.java
similarity index 64%
copy from server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/CsvSourceNode.java
copy to server/src/main/java/org/apache/iotdb/db/query/mpp/operator/ExchangeDataSource.java
index f6dc714..91ec40f 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/CsvSourceNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/mpp/operator/ExchangeDataSource.java
@@ -7,7 +7,7 @@
  * "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
+ *      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
@@ -16,23 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.db.query.mpp.operator;
 
-package org.apache.iotdb.db.query.mpp.plan.node.source;
+import com.google.common.util.concurrent.ListenableFuture;
 
-import org.apache.iotdb.db.query.mpp.plan.node.PlanNodeId;
+import java.io.Closeable;
+import java.nio.ByteBuffer;
 
-/**
- * Not implemented in current version.
- */
-public class CsvSourceNode extends SourceNode {
+public interface ExchangeDataSource extends Closeable {
+
+    ByteBuffer pollTsBlock();
 
-  public CsvSourceNode(PlanNodeId id) {
-    super(id);
-  }
+    boolean isFinished();
 
-  @Override
-  public void close() throws Exception {}
+    ListenableFuture<Void> isBlocked();
 
-  @Override
-  public void open() throws Exception {}
+    @Override
+    void close();
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/mpp/operator/Operator.java b/server/src/main/java/org/apache/iotdb/db/query/mpp/operator/Operator.java
new file mode 100644
index 0000000..5dd28c8
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/query/mpp/operator/Operator.java
@@ -0,0 +1,102 @@
+/*
+ * 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.mpp.operator;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.iotdb.db.query.mpp.common.TsBlock;
+
+import static com.google.common.util.concurrent.Futures.immediateVoidFuture;
+
+public interface Operator extends AutoCloseable {
+    ListenableFuture<Void> NOT_BLOCKED = immediateVoidFuture();
+
+
+    OperatorContext getOperatorContext();
+
+    /**
+     * Returns a future that will be completed when the operator becomes
+     * unblocked.  If the operator is not blocked, this method should return
+     * {@code NOT_BLOCKED}.
+     */
+    default ListenableFuture<Void> isBlocked()
+    {
+        return NOT_BLOCKED;
+    }
+
+    /**
+     * Returns true if and only if this operator can accept an input page.
+     */
+    boolean needsInput();
+
+    /**
+     * Adds an input page to the operator.  This method will only be called if
+     * {@code needsInput()} returns true.
+     */
+    void addInput(TsBlock page);
+
+    /**
+     * Gets an output page from the operator.  If no output data is currently
+     * available, return null.
+     */
+    TsBlock getOutput();
+
+    /**
+     * After calling this method operator should revoke all reserved revocable memory.
+     * As soon as memory is revoked returned future should be marked as done.
+     * <p>
+     * Spawned threads cannot modify OperatorContext because it's not thread safe.
+     * For this purpose implement {@link #finishMemoryRevoke()}
+     * <p>
+     * Since memory revoking signal is delivered asynchronously to the Operator, implementation
+     * must gracefully handle the case when there no longer is any revocable memory allocated.
+     * <p>
+     * After this method is called on Operator the Driver is disallowed to call any
+     * processing methods on it (isBlocked/needsInput/addInput/getOutput) until
+     * {@link #finishMemoryRevoke()} is called.
+     */
+    default ListenableFuture<Void> startMemoryRevoke()
+    {
+        return NOT_BLOCKED;
+    }
+
+    /**
+     * Clean up and release resources after completed memory revoking. Called by driver
+     * once future returned by startMemoryRevoke is completed.
+     */
+    default void finishMemoryRevoke() {}
+
+    /**
+     * Notifies the operator that no more pages will be added and the
+     * operator should finish processing and flush results. This method
+     * will not be called if the Task is already failed or canceled.
+     */
+    void finish();
+
+    /**
+     * Is this operator completely finished processing and no more
+     * output pages will be produced.
+     */
+    boolean isFinished();
+
+    /**
+     * This method will always be called before releasing the Operator reference.
+     */
+    @Override
+    default void close() throws Exception {}
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/CsvSourceNode.java b/server/src/main/java/org/apache/iotdb/db/query/mpp/operator/OperatorContext.java
similarity index 60%
copy from server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/CsvSourceNode.java
copy to server/src/main/java/org/apache/iotdb/db/query/mpp/operator/OperatorContext.java
index f6dc714..84c2964 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/CsvSourceNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/mpp/operator/OperatorContext.java
@@ -7,7 +7,7 @@
  * "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
+ *      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
@@ -16,23 +16,24 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
-package org.apache.iotdb.db.query.mpp.plan.node.source;
+package org.apache.iotdb.db.query.mpp.operator;
 
 import org.apache.iotdb.db.query.mpp.plan.node.PlanNodeId;
 
 /**
- * Not implemented in current version.
+ * Contains information about {@link Operator} execution.
+ * <p>
+ * Not thread-safe.
  */
-public class CsvSourceNode extends SourceNode {
-
-  public CsvSourceNode(PlanNodeId id) {
-    super(id);
-  }
+public class OperatorContext {
 
-  @Override
-  public void close() throws Exception {}
+    private final int operatorId;
+    private final PlanNodeId planNodeId;
+    private final String operatorType;
 
-  @Override
-  public void open() throws Exception {}
+    public OperatorContext(int operatorId, PlanNodeId planNodeId, String operatorType) {
+        this.operatorId = operatorId;
+        this.planNodeId = planNodeId;
+        this.operatorType = operatorType;
+    }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/CsvSourceNode.java b/server/src/main/java/org/apache/iotdb/db/query/mpp/operator/SeriesScanOperator.java
similarity index 68%
copy from server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/CsvSourceNode.java
copy to server/src/main/java/org/apache/iotdb/db/query/mpp/operator/SeriesScanOperator.java
index f6dc714..eff9fa7 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/CsvSourceNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/mpp/operator/SeriesScanOperator.java
@@ -16,23 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.iotdb.db.query.mpp.operator;
 
-package org.apache.iotdb.db.query.mpp.plan.node.source;
-
-import org.apache.iotdb.db.query.mpp.plan.node.PlanNodeId;
-
-/**
- * Not implemented in current version.
- */
-public class CsvSourceNode extends SourceNode {
-
-  public CsvSourceNode(PlanNodeId id) {
-    super(id);
-  }
-
-  @Override
-  public void close() throws Exception {}
-
-  @Override
-  public void open() throws Exception {}
+public class SeriesScanOperator {
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/CsvSourceNode.java b/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/CsvSourceNode.java
index f6dc714..5d76beb 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/CsvSourceNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/CsvSourceNode.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.iotdb.db.query.mpp.plan.node.source;
 
 import org.apache.iotdb.db.query.mpp.plan.node.PlanNodeId;
diff --git a/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/SeriesAggregateNode.java b/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/SeriesAggregateNode.java
index 3b19af1..ec849f0 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/SeriesAggregateNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/SeriesAggregateNode.java
@@ -1,3 +1,22 @@
+/*
+ * 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.mpp.plan.node.source;
 
 import org.apache.iotdb.db.query.expression.unary.FunctionExpression;
diff --git a/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/SeriesScanNode.java b/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/SeriesScanNode.java
index 8ecb6de..4b6a187 100644
--- a/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/SeriesScanNode.java
+++ b/server/src/main/java/org/apache/iotdb/db/query/mpp/plan/node/source/SeriesScanNode.java
@@ -1,5 +1,25 @@
+/*
+ * 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.mpp.plan.node.source;
 
+import org.apache.iotdb.db.metadata.path.PartialPath;
 import org.apache.iotdb.db.query.mpp.common.OrderBy;
 import org.apache.iotdb.db.query.mpp.plan.node.PlanNodeId;
 import org.apache.iotdb.tsfile.read.common.Path;
@@ -15,15 +35,18 @@ import org.apache.iotdb.tsfile.read.filter.basic.Filter;
 public class SeriesScanNode extends SourceNode {
 
   // The path of the target series which will be scanned.
-  private Path seriesPath;
+  private PartialPath seriesPath;
 
   // The order to traverse the data.
   // Currently, we only support TIMESTAMP_ASC and TIMESTAMP_DESC here.
   // The default order is TIMESTAMP_ASC, which means "order by timestamp asc"
   private OrderBy scanOrder = OrderBy.TIMESTAMP_ASC;
 
-  // Filter data in current series.
-  private Filter filter;
+  // time filter for current series, could be null if doesn't exist
+  private Filter timeFilter;
+
+  // value filter for current series, could be null if doesn't exist
+  private Filter valueFilter;
 
   // Limit for result set. The default value is -1, which means no limit
   private int limit;
@@ -31,13 +54,17 @@ public class SeriesScanNode extends SourceNode {
   // offset for result set. The default value is 0
   private int offset;
 
-  public SeriesScanNode(PlanNodeId id, Path seriesPath) {
+  public SeriesScanNode(PlanNodeId id, PartialPath seriesPath) {
     super(id);
     this.seriesPath = seriesPath;
   }
 
-  public void setFilter(Filter filter) {
-    this.filter = filter;
+  public void setTimeFilter(Filter timeFilter) {
+    this.timeFilter = timeFilter;
+  }
+
+  public void setValueFilter(Filter valueFilter) {
+    this.valueFilter = valueFilter;
   }
 
   @Override