You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by qi...@apache.org on 2019/11/28 07:29:26 UTC
[incubator-iotdb] 01/02: Cluster read (#152)
This is an automated email from the ASF dual-hosted git repository.
qiaojialin pushed a commit to branch cluster
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git
commit 729935036cefd8d8e55ea2ba498c3d442ace40e7
Author: lta <li...@163.com>
AuthorDate: Tue May 7 19:28:11 2019 +0800
Cluster read (#152)
* reorganize package
* initial query module
* remove threadlocal in TSServiceClusterImpl
* add Query Manager
* update
* update
* add query manager
* add single query manager
* add ClusterRpcReaderUtils
* add query series data processor
* transfer physical plans implement Serializable
* manage package
* add reader processor
* add task id
* add cached batch data
* remove rpc batch reader
* complete code of no filter reader
* reorganize engine executor with time generator
* reorganize engine executor with time generator
* complete query with no value filter
* add construct filter query time generator
* add comments
* add comments
* add comments
* fix bug of qp executor test and add prune expression tree
* add query node process for query with value
* complete clsuer query with value filter
* remove task id
* remove task id
* fix issue of remote query finish
* update
* fix bug of unit test
* update remote unit test
* fix a serve bug
* add rpc manager ut and fix some bugs
* add ut of query utils
* add ut
* fix a bug of pertition query plans
* fix some serve bugs and change read resource lock to count
* add ut of query remote data
* fix some serve bugs and finish ut
* fix some serve bugs and finish ut
* fix some serve bugs
* add some request and response , fix bug of cluster pom
* fix a bug
* skip test
* remove disconnect function
* fix a bug
* fix some bugs and add statistic
* add read resource usage
* reimple raft node as client manager
* update jraft version
* clear property set in test
* add init router when tear down in router test
* add large data ut and add query timeout mechanism
add reset timer
* add abstract timer class
* reimpl raft node as manager
* update trvis.yml
* fix some issues according to pr comments
---
cluster/pom.xml | 12 +-
.../iotdb/cluster/concurrent/ThreadName.java | 10 +-
.../cluster/concurrent/pool/QueryTimerManager.java | 74 +++
.../cluster/concurrent/pool/ThreadPoolManager.java | 2 +-
.../apache/iotdb/cluster/config/ClusterConfig.java | 31 +-
.../iotdb/cluster/config/ClusterConstant.java | 13 +-
.../iotdb/cluster/config/ClusterDescriptor.java | 30 +-
.../org/apache/iotdb/cluster/entity/Server.java | 48 +-
.../cluster/entity/raft/DataStateMachine.java | 4 +-
.../cluster/entity/raft/MetadataStateManchine.java | 4 +-
.../apache/iotdb/cluster/qp/ClusterQPExecutor.java | 260 ----------
.../cluster/qp/executor/AbstractQPExecutor.java | 178 +++++++
.../qp/executor/ClusterQueryProcessExecutor.java | 177 +++++++
.../cluster/qp/executor/NonQueryExecutor.java | 55 ++-
.../cluster/qp/executor/QueryMetadataExecutor.java | 73 ++-
.../cluster/qp/{callback => task}/BatchQPTask.java | 17 +-
.../cluster/qp/{callback => task}/MultiQPTask.java | 8 +-
.../cluster/qp/{callback => task}/QPTask.java | 2 +-
.../apache/iotdb/cluster/qp/task/QueryTask.java | 39 +-
.../qp/{callback => task}/SingleQPTask.java | 6 +-
.../org/apache/iotdb/cluster/query/PathType.java | 19 +-
.../org/apache/iotdb/cluster/query/QueryType.java | 23 +-
.../dataset/ClusterDataSetWithTimeGenerator.java | 100 ++--
.../executor/ClusterExecutorWithTimeGenerator.java | 130 +++++
.../ClusterExecutorWithoutTimeGenerator.java | 102 ++++
.../cluster/query/executor/ClusterQueryRouter.java | 121 +++++
.../cluster/query/expression/TrueExpression.java | 31 +-
.../query/factory/ClusterSeriesReaderFactory.java | 94 ++++
.../coordinatornode/ClusterRpcQueryManager.java | 111 +++++
.../ClusterRpcSingleQueryManager.java | 415 ++++++++++++++++
.../manager/coordinatornode/FilterGroupEntity.java | 111 +++++
.../coordinatornode/IClusterRpcQueryManager.java | 69 +++
.../IClusterRpcSingleQueryManager.java | 98 ++++
.../querynode/ClusterLocalQueryManager.java | 125 +++++
.../querynode/ClusterLocalSingleQueryManager.java | 335 +++++++++++++
.../querynode/IClusterLocalQueryManager.java | 82 +++
.../querynode/IClusterLocalSingleQueryManager.java | 74 +++
.../AbstractClusterPointReader.java | 73 +++
.../coordinatornode/ClusterFilterSeriesReader.java | 126 +++++
.../coordinatornode/ClusterSelectSeriesReader.java | 167 +++++++
.../querynode/AbstractClusterBatchReader.java} | 23 +-
.../querynode/ClusterBatchReaderByTimestamp.java | 86 ++++
.../ClusterBatchReaderWithoutTimeGenerator.java | 94 ++++
.../querynode/ClusterFilterSeriesBatchReader.java | 121 +++++
.../IClusterFilterSeriesBatchReader.java} | 19 +-
.../timegenerator/ClusterLeafNode.java} | 35 +-
.../timegenerator/ClusterNodeConstructor.java | 106 ++++
.../query/timegenerator/ClusterTimeGenerator.java | 28 +-
.../cluster/query/utils/ClusterRpcReaderUtils.java | 128 +++++
.../iotdb/cluster/query/utils/ExpressionUtils.java | 131 +++++
.../query/utils/QueryPlanPartitionUtils.java | 127 +++++
.../iotdb/cluster/rpc/raft/NodeAsClient.java | 16 +-
.../rpc/raft/impl/RaftNodeAsClientManager.java | 90 ++--
.../DataGroupNonQueryAsyncProcessor.java | 7 +-
.../MetaGroupNonQueryAsyncProcessor.java | 7 +-
.../querydata/CloseSeriesReaderSyncProcessor.java | 43 ++
.../querydata/InitSeriesReaderSyncProcessor.java | 64 +++
.../QuerySeriesDataByTimestampSyncProcessor.java | 36 +-
.../querydata/QuerySeriesDataSyncProcessor.java | 38 +-
.../QueryMetadataAsyncProcessor.java | 7 +-
.../QueryMetadataInStringAsyncProcessor.java | 7 +-
.../QueryPathsAsyncProcessor.java | 7 +-
.../QuerySeriesTypeAsyncProcessor.java | 7 +-
.../QueryTimeSeriesAsyncProcessor.java | 7 +-
...BasicRequest.java => BasicNonQueryRequest.java} | 27 +-
.../rpc/raft/request/BasicQueryRequest.java | 5 +
.../cluster/rpc/raft/request/BasicRequest.java | 21 -
.../{ => nonquery}/DataGroupNonQueryRequest.java | 7 +-
.../{ => nonquery}/MetaGroupNonQueryRequest.java | 8 +-
.../CloseSeriesReaderRequest.java} | 27 +-
.../request/querydata/InitSeriesReaderRequest.java | 91 ++++
.../QuerySeriesDataByTimestampRequest.java | 91 ++++
.../request/querydata/QuerySeriesDataRequest.java | 98 ++++
.../QueryMetadataInStringRequest.java | 8 +-
.../{ => querymetadata}/QueryMetadataRequest.java | 8 +-
.../{ => querymetadata}/QueryPathsRequest.java | 7 +-
.../QuerySeriesTypeRequest.java | 7 +-
.../QueryStorageGroupRequest.java | 8 +-
.../QueryTimeSeriesRequest.java | 7 +-
...ryResponse.java => BasicQueryDataResponse.java} | 24 +-
.../{ => nonquery}/DataGroupNonQueryResponse.java | 6 +-
.../{ => nonquery}/MetaGroupNonQueryResponse.java | 6 +-
.../querydata/InitSeriesReaderResponse.java | 59 +++
.../QuerySeriesDataByTimestampResponse.java | 18 +-
.../querydata/QuerySeriesDataResponse.java} | 26 +-
.../QueryMetadataInStringResponse.java | 5 +-
.../{ => querymetadata}/QueryMetadataResponse.java | 4 +-
.../{ => querymetadata}/QueryPathsResponse.java | 4 +-
.../QuerySeriesTypeResponse.java | 4 +-
.../QueryStorageGroupResponse.java | 4 +-
.../QueryTimeSeriesResponse.java | 4 +-
.../{rpc => }/service/TSServiceClusterImpl.java | 147 ++++--
.../iotdb/cluster/utils/QPExecutorUtils.java | 156 ++++++
.../org/apache/iotdb/cluster/utils/RaftUtils.java | 85 +++-
.../iotdb/cluster/utils/hash/PhysicalNode.java | 16 +-
.../apache/iotdb/cluster/utils/hash/Router.java | 8 +
.../cluster/config/ClusterDescriptorTest.java | 9 +-
.../integration/IoTDBMetadataFetchRemoteIT.java | 13 +-
...ecutorTest.java => AbstractQPExecutorTest.java} | 40 +-
.../cluster/query/ClusterQueryLargeDataTest.java | 507 +++++++++++++++++++
.../iotdb/cluster/query/ClusterQueryTest.java | 550 +++++++++++++++++++++
.../query/manager/ClusterLocalManagerTest.java | 406 +++++++++++++++
.../query/manager/ClusterRpcManagerTest.java | 334 +++++++++++++
.../cluster/query/utils/ExpressionUtilsTest.java | 230 +++++++++
.../query/utils/QueryPlanPartitionUtilsTest.java | 332 +++++++++++++
.../apache/iotdb/cluster/utils/RaftUtilsTest.java | 6 +-
.../java/org/apache/iotdb/cluster/utils/Utils.java | 27 +
.../iotdb/cluster/utils/hash/RouterTest.java | 3 +
iotdb/iotdb/conf/iotdb-cluster.properties | 13 +-
iotdb/iotdb/conf/logback.xml | 2 -
.../iotdb/db/engine/filenode/FileNodeManager.java | 8 +-
.../db/engine/filenode/FileNodeProcessor.java | 136 ++---
.../db/exception/FileNodeManagerException.java | 4 +
.../org/apache/iotdb/db/qp/QueryProcessor.java | 11 +-
.../db/qp/executor/IQueryProcessExecutor.java | 143 ++++++
.../iotdb/db/qp/executor/OverflowQPExecutor.java | 3 +-
.../iotdb/db/qp/executor/QueryProcessExecutor.java | 98 +---
.../db/qp/logical/crud/BasicFunctionOperator.java | 3 +-
.../iotdb/db/qp/logical/crud/FilterOperator.java | 5 +-
.../apache/iotdb/db/qp/physical/PhysicalPlan.java | 4 +-
.../iotdb/db/qp/physical/crud/AggregationPlan.java | 1 +
.../iotdb/db/qp/physical/crud/DeletePlan.java | 1 +
.../iotdb/db/qp/physical/crud/FillQueryPlan.java | 1 +
.../iotdb/db/qp/physical/crud/GroupByPlan.java | 1 +
.../iotdb/db/qp/physical/crud/InsertPlan.java | 1 +
.../iotdb/db/qp/physical/crud/QueryPlan.java | 5 +-
.../iotdb/db/qp/physical/crud/UpdatePlan.java | 1 +
.../iotdb/db/qp/physical/sys/AuthorPlan.java | 1 +
.../iotdb/db/qp/physical/sys/LoadDataPlan.java | 1 +
.../iotdb/db/qp/physical/sys/MetadataPlan.java | 1 +
.../iotdb/db/qp/physical/sys/PropertyPlan.java | 1 +
.../{writelog => qp/physical}/transfer/Codec.java | 3 +-
.../physical}/transfer/CodecInstances.java | 2 +-
.../physical}/transfer/PhysicalPlanCodec.java | 14 +-
.../transfer/PhysicalPlanLogTransfer.java | 2 +-
.../physical}/transfer/SystemLogOperator.java | 2 +-
.../iotdb/db/qp/strategy/PhysicalGenerator.java | 6 +-
.../qp/strategy/optimizer/ConcatPathOptimizer.java | 5 +-
.../db/query/control/QueryResourceManager.java | 27 +-
.../dataset/EngineDataSetWithTimeGenerator.java | 35 +-
.../dataset/EngineDataSetWithoutTimeGenerator.java | 4 +
.../EngineExecutorWithoutTimeGenerator.java | 103 +---
.../iotdb/db/query/executor/EngineQueryRouter.java | 34 +-
.../executor/ExecutorWithoutTimeGenerator.java | 80 +++
.../db/query/executor/IEngineQueryRouter.java | 78 +++
...nstructor.java => AbstractNodeConstructor.java} | 57 +--
.../query/timegenerator/EngineNodeConstructor.java | 62 +--
.../query/timegenerator/EngineTimeGenerator.java | 1 -
.../org/apache/iotdb/db/service/TSServiceImpl.java | 65 +--
.../apache/iotdb/db/writelog/io/RAFLogReader.java | 2 +-
.../db/writelog/node/ExclusiveWriteLogNode.java | 2 +-
.../iotdb/db/integration/IoTDBSeriesReaderIT.java | 2 +-
.../transfer/PhysicalPlanLogTransferTest.java | 2 +-
.../org/apache/iotdb/db/qp/plan/QPUpdateTest.java | 2 +-
.../apache/iotdb/db/qp/utils/MemIntQpExecutor.java | 3 +-
.../EngineDataSetWithTimeGeneratorTest.java | 2 +-
.../org/apache/iotdb/db/tools/WalCheckerTest.java | 2 +-
.../apache/iotdb/db/writelog/PerformanceTest.java | 3 +-
.../apache/iotdb/db/writelog/WriteLogNodeTest.java | 2 +-
.../iotdb/db/writelog/io/LogWriterReaderTest.java | 3 +-
.../apache/iotdb/tsfile/read/common/BatchData.java | 4 +-
.../org/apache/iotdb/tsfile/read/common/Field.java | 22 +
.../org/apache/iotdb/tsfile/read/common/Path.java | 4 +-
.../tsfile/read/expression/ExpressionType.java | 2 +-
.../tsfile/read/expression/IBinaryExpression.java | 6 +-
.../iotdb/tsfile/read/expression/IExpression.java | 4 +-
.../read/expression/impl/BinaryExpression.java | 38 +-
.../read/expression/impl/GlobalTimeExpression.java | 10 +-
.../expression/impl/SingleSeriesExpression.java | 10 +-
.../tsfile/read/filter/basic/BinaryFilter.java | 3 +
.../iotdb/tsfile/read/filter/basic/Filter.java | 2 +
.../tsfile/read/filter/basic/UnaryFilter.java | 3 +
.../tsfile/read/filter/operator/AndFilter.java | 7 +-
.../iotdb/tsfile/read/filter/operator/Eq.java | 6 +
.../iotdb/tsfile/read/filter/operator/Gt.java | 6 +
.../iotdb/tsfile/read/filter/operator/GtEq.java | 6 +
.../iotdb/tsfile/read/filter/operator/Lt.java | 6 +
.../iotdb/tsfile/read/filter/operator/LtEq.java | 6 +
.../iotdb/tsfile/read/filter/operator/NotEq.java | 6 +
.../tsfile/read/filter/operator/NotFilter.java | 5 +
.../tsfile/read/filter/operator/OrFilter.java | 5 +
.../query/dataset/DataSetWithTimeGenerator.java | 32 --
.../tsfile/read/query/dataset/QueryDataSet.java | 36 ++
.../java/org/apache/iotdb/tsfile/utils/Pair.java | 5 +-
184 files changed, 8047 insertions(+), 1373 deletions(-)
diff --git a/cluster/pom.xml b/cluster/pom.xml
index b0bca81..25d13ea 100644
--- a/cluster/pom.xml
+++ b/cluster/pom.xml
@@ -26,10 +26,10 @@
<version>0.8.0-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
- <artifactId>IoTDB-cluster</artifactId>
+ <artifactId>iotdb-cluster</artifactId>
<name>IoTDB Cluster</name>
<properties>
- <jraft.version>1.2.4</jraft.version>
+ <jraft.version>1.2.5</jraft.version>
<antlr3.version>3.5.2</antlr3.version>
<common.lang3.version>3.8.1</common.lang3.version>
<cluster.test.skip>false</cluster.test.skip>
@@ -84,21 +84,21 @@
<configuration>
<filesets>
<fileset>
- <directory>${project.basedir}/../iotdb/lib_cluster</directory>
+ <directory>${project.basedir}/../iotdb/iotdb/lib_cluster</directory>
<includes>
<include>**/*.jar</include>
</includes>
<followSymlinks>false</followSymlinks>
</fileset>
<fileset>
- <directory>${project.basedir}/../iotdb/data</directory>
+ <directory>${project.basedir}/../iotdb/iotdb/data</directory>
<includes>
<include>**/*</include>
</includes>
<followSymlinks>false</followSymlinks>
</fileset>
<fileset>
- <directory>${project.basedir}/../iotdb/logs</directory>
+ <directory>${project.basedir}/../iotdb/iotdb/logs</directory>
<includes>
<include>**/*</include>
</includes>
@@ -142,7 +142,7 @@
<outputDirectory>${project.basedir}/../iotdb/iotdb/lib_cluster</outputDirectory>
</configuration>
</plugin>
- <!--using `mvn test` to run UT, `mvn verify` to run ITs
+ <!--using `mvn test` to triggerAction UT, `mvn verify` to triggerAction ITs
Reference: https://antoniogoncalves.org/2012/12/13/lets-turn-integration-tests-with-maven-to-a-first-class-citizen/-->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/concurrent/ThreadName.java b/cluster/src/main/java/org/apache/iotdb/cluster/concurrent/ThreadName.java
index b500c18..9212258 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/concurrent/ThreadName.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/concurrent/ThreadName.java
@@ -19,8 +19,16 @@
package org.apache.iotdb.cluster.concurrent;
public enum ThreadName {
+
+ /**
+ * QP Task thread
+ */
QP_TASK("QP-Task-Thread"),
- RAFT_TASK("Raft-Task-Thread");
+
+ /**
+ * Remote query timer
+ */
+ REMOTE_QUERY_TIMER("Remote-Query-Timer");
private String name;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/concurrent/pool/QueryTimerManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/concurrent/pool/QueryTimerManager.java
new file mode 100644
index 0000000..779488c
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/concurrent/pool/QueryTimerManager.java
@@ -0,0 +1,74 @@
+/**
+ * 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.cluster.concurrent.pool;
+
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.iotdb.cluster.concurrent.ThreadName;
+import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
+
+/**
+ * Manage all query timer in query node, if timer is timeout, close all query resource for remote
+ * coordinator node.
+ */
+public class QueryTimerManager extends ThreadPoolManager {
+
+ private static final String MANAGER_NAME = "remote-query-timer-manager";
+
+ private static final int CORE_POOL_SIZE = 1;
+
+ @Override
+ public void init() {
+ pool = IoTDBThreadPoolFactory.newScheduledThreadPool(getThreadPoolSize(), getThreadName());
+ }
+
+ public static QueryTimerManager getInstance() {
+ return QueryTimerManager.QueryTimerManagerHolder.INSTANCE;
+ }
+
+ @Override
+ public String getManagerName() {
+ return MANAGER_NAME;
+ }
+
+ @Override
+ public String getThreadName() {
+ return ThreadName.REMOTE_QUERY_TIMER.getName();
+ }
+
+ @Override
+ public int getThreadPoolSize() {
+ return CORE_POOL_SIZE;
+ }
+
+ public ScheduledFuture<?> execute(Runnable task, long delayMs) {
+ checkInit();
+ return ((ScheduledExecutorService) pool).schedule(task, delayMs, TimeUnit.MICROSECONDS);
+ }
+
+ private static class QueryTimerManagerHolder {
+
+ private static final QueryTimerManager INSTANCE = new QueryTimerManager();
+
+ private QueryTimerManagerHolder() {
+
+ }
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/concurrent/pool/ThreadPoolManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/concurrent/pool/ThreadPoolManager.java
index a6dfa42..828cc1a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/concurrent/pool/ThreadPoolManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/concurrent/pool/ThreadPoolManager.java
@@ -29,7 +29,7 @@ public abstract class ThreadPoolManager {
ExecutorService pool;
- private void checkInit() {
+ public void checkInit() {
if (pool == null) {
init();
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java
index b860835..0e6472d 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConfig.java
@@ -102,7 +102,7 @@ public class ClusterConfig {
* then it sends requests to other nodes in the cluster. This parameter represents the maximum
* timeout for these requests. The unit is milliseconds.
**/
- private int qpTaskTimeout = 1000;
+ private int qpTaskTimeout = 5000;
/**
* Number of virtual nodes
@@ -137,6 +137,19 @@ public class ClusterConfig {
*/
private int concurrentQPSubTaskThread = Runtime.getRuntime().availableProcessors() * 10;
+ /**
+ * Batch data size read from remote query node once while reading, default value is 10000.
+ * The smaller the parameter, the more communication times and the more time-consuming it is.
+ */
+ private int batchReadSize = 10000;
+
+ /**
+ * Maximum number of cached batch data list for each series in coordinator node while reading,
+ * default value is 2. The coordinator node is responsible for receiving client requests and
+ * requesting data from query nodes and collecting them.
+ */
+ private int maxCachedBatchDataListSize = 2;
+
public ClusterConfig() {
// empty constructor
}
@@ -323,4 +336,20 @@ public class ClusterConfig {
public void setConcurrentQPSubTaskThread(int concurrentQPSubTaskThread) {
this.concurrentQPSubTaskThread = concurrentQPSubTaskThread;
}
+
+ public int getBatchReadSize() {
+ return batchReadSize;
+ }
+
+ public void setBatchReadSize(int batchReadSize) {
+ this.batchReadSize = batchReadSize;
+ }
+
+ public int getMaxCachedBatchDataListSize() {
+ return maxCachedBatchDataListSize;
+ }
+
+ public void setMaxCachedBatchDataListSize(int maxCachedBatchDataListSize) {
+ this.maxCachedBatchDataListSize = maxCachedBatchDataListSize;
+ }
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConstant.java b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConstant.java
index 5aca9b0..5448847 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConstant.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterConstant.java
@@ -20,10 +20,12 @@ package org.apache.iotdb.cluster.config;
public class ClusterConstant {
- private ClusterConstant(){
-
+ private ClusterConstant() {
}
+ /**
+ * Set read metadata consistency level pattern
+ */
public static final String SET_READ_METADATA_CONSISTENCY_LEVEL_PATTERN = "set\\s+read\\s+metadata\\s+level\\s+to\\s+\\d+";
public static final String SET_READ_DATA_CONSISTENCY_LEVEL_PATTERN = "set\\s+read\\s+data\\s+level\\s+to\\s+\\d+";
public static final int MAX_CONSISTENCY_LEVEL = 2;
@@ -37,4 +39,11 @@ public class ClusterConstant {
*/
public static final int CLOSE_QP_SUB_TASK_BLOCK_TIMEOUT = 1000;
+ /**
+ * Query timeout in query node. If time interval between last communications with coordinator node
+ * and now exceed this parameter, release corresponding query resource.Each query in query node
+ * has a <code>QueryRepeaterTimer</code>, the unit is milliseconds. Default value is 30 minutes.
+ */
+ public static final int QUERY_TIMEOUT_IN_QUERY_NODE = 30 * 60 * 1000;
+
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java
index 8a49d7e..b90d781 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/config/ClusterDescriptor.java
@@ -24,7 +24,7 @@ import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.util.Properties;
-import org.apache.iotdb.cluster.rpc.service.TSServiceClusterImpl;
+import org.apache.iotdb.cluster.service.TSServiceClusterImpl;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -52,9 +52,8 @@ public class ClusterDescriptor {
}
/**
- * Load an property file and set ClusterConfig variables.
- * Change this method to public only for test.
- * In most case, you should invoke this method.
+ * Load an property file and set ClusterConfig variables. Change this method to public only for
+ * test. In most case, you should invoke this method.
*/
public void loadProps() {
ioTDBConf.setRpcImplClassName(TSServiceClusterImpl.class.getName());
@@ -104,9 +103,11 @@ public class ClusterDescriptor {
conf.setRaftLogPath(properties.getProperty("raft_log_path", conf.getRaftLogPath()));
- conf.setRaftSnapshotPath(properties.getProperty("raft_snapshot_path", conf.getRaftSnapshotPath()));
+ conf.setRaftSnapshotPath(
+ properties.getProperty("raft_snapshot_path", conf.getRaftSnapshotPath()));
- conf.setRaftMetadataPath(properties.getProperty("raft_metadata_path", conf.getRaftMetadataPath()));
+ conf.setRaftMetadataPath(
+ properties.getProperty("raft_metadata_path", conf.getRaftMetadataPath()));
conf.setElectionTimeoutMs(Integer
.parseInt(properties.getProperty("election_timeout_ms",
@@ -155,10 +156,26 @@ public class ClusterDescriptor {
conf.setConcurrentQPSubTaskThread(Integer
.parseInt(properties.getProperty("concurrent_qp_sub_task_thread",
Integer.toString(conf.getConcurrentQPSubTaskThread()))));
+
+ conf.setBatchReadSize(Integer.parseInt(properties.getProperty("batch_read_size",
+ Integer.toString(conf.getBatchReadSize()))));
+
+ conf.setMaxCachedBatchDataListSize(Integer.parseInt(properties
+ .getProperty("max_cached_batch_data_list_size",
+ Integer.toString(conf.getMaxCachedBatchDataListSize()))));
+
if (conf.getConcurrentQPSubTaskThread() <= 0) {
conf.setConcurrentQPSubTaskThread(Runtime.getRuntime().availableProcessors() * 10);
}
+ if (conf.getMaxCachedBatchDataListSize() <= 0) {
+ conf.setMaxCachedBatchDataListSize(2);
+ }
+
+ if (conf.getBatchReadSize() <= 0) {
+ conf.setBatchReadSize(10000);
+ }
+
} catch (IOException e) {
LOGGER.warn("Cannot load config file because, use default configuration", e);
} catch (Exception e) {
@@ -174,6 +191,7 @@ public class ClusterDescriptor {
}
private static class ClusterDescriptorHolder {
+
private static final ClusterDescriptor INSTANCE = new ClusterDescriptor();
}
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/entity/Server.java b/cluster/src/main/java/org/apache/iotdb/cluster/entity/Server.java
index 4b2ebef..0efb70d 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/entity/Server.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/entity/Server.java
@@ -32,13 +32,17 @@ import org.apache.iotdb.cluster.entity.metadata.MetadataHolder;
import org.apache.iotdb.cluster.entity.raft.DataPartitionRaftHolder;
import org.apache.iotdb.cluster.entity.raft.MetadataRaftHolder;
import org.apache.iotdb.cluster.rpc.raft.impl.RaftNodeAsClientManager;
-import org.apache.iotdb.cluster.rpc.raft.processor.DataGroupNonQueryAsyncProcessor;
-import org.apache.iotdb.cluster.rpc.raft.processor.MetaGroupNonQueryAsyncProcessor;
-import org.apache.iotdb.cluster.rpc.raft.processor.QueryMetadataAsyncProcessor;
-import org.apache.iotdb.cluster.rpc.raft.processor.QueryMetadataInStringAsyncProcessor;
-import org.apache.iotdb.cluster.rpc.raft.processor.QueryPathsAsyncProcessor;
-import org.apache.iotdb.cluster.rpc.raft.processor.QuerySeriesTypeAsyncProcessor;
-import org.apache.iotdb.cluster.rpc.raft.processor.QueryTimeSeriesAsyncProcessor;
+import org.apache.iotdb.cluster.rpc.raft.processor.nonquery.DataGroupNonQueryAsyncProcessor;
+import org.apache.iotdb.cluster.rpc.raft.processor.nonquery.MetaGroupNonQueryAsyncProcessor;
+import org.apache.iotdb.cluster.rpc.raft.processor.querydata.CloseSeriesReaderSyncProcessor;
+import org.apache.iotdb.cluster.rpc.raft.processor.querydata.InitSeriesReaderSyncProcessor;
+import org.apache.iotdb.cluster.rpc.raft.processor.querydata.QuerySeriesDataByTimestampSyncProcessor;
+import org.apache.iotdb.cluster.rpc.raft.processor.querydata.QuerySeriesDataSyncProcessor;
+import org.apache.iotdb.cluster.rpc.raft.processor.querymetadata.QueryMetadataAsyncProcessor;
+import org.apache.iotdb.cluster.rpc.raft.processor.querymetadata.QueryMetadataInStringAsyncProcessor;
+import org.apache.iotdb.cluster.rpc.raft.processor.querymetadata.QueryPathsAsyncProcessor;
+import org.apache.iotdb.cluster.rpc.raft.processor.querymetadata.QuerySeriesTypeAsyncProcessor;
+import org.apache.iotdb.cluster.rpc.raft.processor.querymetadata.QueryTimeSeriesAsyncProcessor;
import org.apache.iotdb.cluster.utils.RaftUtils;
import org.apache.iotdb.cluster.utils.hash.PhysicalNode;
import org.apache.iotdb.cluster.utils.hash.Router;
@@ -96,13 +100,9 @@ public class Server {
RpcServer rpcServer = new RpcServer(serverId.getPort());
RaftRpcServerFactory.addRaftRequestProcessors(rpcServer);
- rpcServer.registerUserProcessor(new DataGroupNonQueryAsyncProcessor());
- rpcServer.registerUserProcessor(new MetaGroupNonQueryAsyncProcessor());
- rpcServer.registerUserProcessor(new QueryTimeSeriesAsyncProcessor());
- rpcServer.registerUserProcessor(new QueryMetadataInStringAsyncProcessor());
- rpcServer.registerUserProcessor(new QueryMetadataAsyncProcessor());
- rpcServer.registerUserProcessor(new QuerySeriesTypeAsyncProcessor());
- rpcServer.registerUserProcessor(new QueryPathsAsyncProcessor());
+ registerNonQueryProcessor(rpcServer);
+ registerQueryMetadataProcessor(rpcServer);
+ registerQueryDataProcessor(rpcServer);
metadataHolder = new MetadataRaftHolder(peerIds, serverId, rpcServer, true);
metadataHolder.init();
@@ -128,6 +128,26 @@ public class Server {
}
+ private void registerNonQueryProcessor(RpcServer rpcServer) {
+ rpcServer.registerUserProcessor(new DataGroupNonQueryAsyncProcessor());
+ rpcServer.registerUserProcessor(new MetaGroupNonQueryAsyncProcessor());
+ }
+
+ private void registerQueryMetadataProcessor(RpcServer rpcServer) {
+ rpcServer.registerUserProcessor(new QueryTimeSeriesAsyncProcessor());
+ rpcServer.registerUserProcessor(new QueryMetadataInStringAsyncProcessor());
+ rpcServer.registerUserProcessor(new QueryMetadataAsyncProcessor());
+ rpcServer.registerUserProcessor(new QuerySeriesTypeAsyncProcessor());
+ rpcServer.registerUserProcessor(new QueryPathsAsyncProcessor());
+ }
+
+ private void registerQueryDataProcessor(RpcServer rpcServer) {
+ rpcServer.registerUserProcessor(new InitSeriesReaderSyncProcessor());
+ rpcServer.registerUserProcessor(new QuerySeriesDataSyncProcessor());
+ rpcServer.registerUserProcessor(new QuerySeriesDataByTimestampSyncProcessor());
+ rpcServer.registerUserProcessor(new CloseSeriesReaderSyncProcessor());
+ }
+
public void stop() throws ProcessorException, InterruptedException {
QPTaskManager.getInstance().close(true, ClusterConstant.CLOSE_QP_SUB_TASK_BLOCK_TIMEOUT);
iotdb.deactivate();
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/entity/raft/DataStateMachine.java b/cluster/src/main/java/org/apache/iotdb/cluster/entity/raft/DataStateMachine.java
index ebac074..b8c6f43 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/entity/raft/DataStateMachine.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/entity/raft/DataStateMachine.java
@@ -31,7 +31,7 @@ import java.nio.ByteBuffer;
import java.util.List;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.iotdb.cluster.rpc.raft.closure.ResponseClosure;
-import org.apache.iotdb.cluster.rpc.raft.request.DataGroupNonQueryRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.nonquery.DataGroupNonQueryRequest;
import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
import org.apache.iotdb.cluster.utils.RaftUtils;
import org.apache.iotdb.db.exception.PathErrorException;
@@ -41,7 +41,7 @@ import org.apache.iotdb.db.qp.executor.OverflowQPExecutor;
import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
import org.apache.iotdb.db.qp.physical.sys.MetadataPlan;
-import org.apache.iotdb.db.writelog.transfer.PhysicalPlanLogTransfer;
+import org.apache.iotdb.db.qp.physical.transfer.PhysicalPlanLogTransfer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/entity/raft/MetadataStateManchine.java b/cluster/src/main/java/org/apache/iotdb/cluster/entity/raft/MetadataStateManchine.java
index 9592718..3cc9001 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/entity/raft/MetadataStateManchine.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/entity/raft/MetadataStateManchine.java
@@ -32,7 +32,7 @@ import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.iotdb.cluster.rpc.raft.closure.ResponseClosure;
-import org.apache.iotdb.cluster.rpc.raft.request.MetaGroupNonQueryRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.nonquery.MetaGroupNonQueryRequest;
import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
import org.apache.iotdb.cluster.utils.RaftUtils;
import org.apache.iotdb.db.exception.PathErrorException;
@@ -43,7 +43,7 @@ import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
import org.apache.iotdb.db.qp.physical.sys.MetadataPlan;
-import org.apache.iotdb.db.writelog.transfer.PhysicalPlanLogTransfer;
+import org.apache.iotdb.db.qp.physical.transfer.PhysicalPlanLogTransfer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/qp/ClusterQPExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/qp/ClusterQPExecutor.java
deleted file mode 100644
index b9debe3..0000000
--- a/cluster/src/main/java/org/apache/iotdb/cluster/qp/ClusterQPExecutor.java
+++ /dev/null
@@ -1,260 +0,0 @@
-/**
- * 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.cluster.qp;
-
-import com.alipay.sofa.jraft.entity.PeerId;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.iotdb.cluster.qp.callback.QPTask;
-import org.apache.iotdb.cluster.qp.callback.QPTask.TaskState;
-import org.apache.iotdb.cluster.config.ClusterConfig;
-import org.apache.iotdb.cluster.config.ClusterConstant;
-import org.apache.iotdb.cluster.config.ClusterDescriptor;
-import org.apache.iotdb.cluster.entity.Server;
-import org.apache.iotdb.cluster.entity.raft.MetadataRaftHolder;
-import org.apache.iotdb.cluster.exception.ConsistencyLevelException;
-import org.apache.iotdb.cluster.exception.RaftConnectionException;
-import org.apache.iotdb.cluster.rpc.raft.NodeAsClient;
-import org.apache.iotdb.cluster.rpc.raft.impl.RaftNodeAsClientManager;
-import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
-import org.apache.iotdb.cluster.utils.RaftUtils;
-import org.apache.iotdb.cluster.utils.hash.PhysicalNode;
-import org.apache.iotdb.cluster.utils.hash.Router;
-import org.apache.iotdb.db.exception.PathErrorException;
-import org.apache.iotdb.db.metadata.MManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public abstract class ClusterQPExecutor {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(ClusterQPExecutor.class);
-
- private static final ClusterConfig CLUSTER_CONFIG = ClusterDescriptor.getInstance().getConfig();
-
- /**
- * Raft as client manager.
- */
- private static final RaftNodeAsClientManager CLIENT_MANAGER = RaftNodeAsClientManager
- .getInstance();
-
- protected Router router = Router.getInstance();
-
- private PhysicalNode localNode = new PhysicalNode(CLUSTER_CONFIG.getIp(),
- CLUSTER_CONFIG.getPort());
-
- protected MManager mManager = MManager.getInstance();
-
- protected final Server server = Server.getInstance();
-
- /**
- * The task in progress.
- */
- protected QPTask currentTask;
-
- /**
- * Count limit to redo a single task
- */
- private static final int TASK_MAX_RETRY = CLUSTER_CONFIG.getQpTaskRedoCount();
-
- /**
- * ReadMetadataConsistencyLevel: 1 Strong consistency, 2 Weak consistency
- */
- protected int readMetadataConsistencyLevel = CLUSTER_CONFIG.getReadMetadataConsistencyLevel();
-
- /**
- * ReadDataConsistencyLevel: 1 Strong consistency, 2 Weak consistency
- */
- private int readDataConsistencyLevel = CLUSTER_CONFIG.getReadDataConsistencyLevel();
-
- /**
- * Get Storage Group Name by device name
- */
- protected String getStroageGroupByDevice(String device) throws PathErrorException {
- String storageGroup;
- try {
- storageGroup = MManager.getInstance().getFileNameByPath(device);
- } catch (PathErrorException e) {
- throw new PathErrorException(String.format("File level of %s doesn't exist.", device));
- }
- return storageGroup;
- }
-
- /**
- * Get all Storage Group Names by path
- */
- public List<String> getAllStroageGroupsByPath(String path) throws PathErrorException {
- List<String> storageGroupList;
- try {
- storageGroupList = mManager.getAllFileNamesByPath(path);
- } catch (PathErrorException e) {
- throw new PathErrorException(String.format("File level of %s doesn't exist.", path));
- }
- return storageGroupList;
- }
-
- /**
- * Classify the input storage group list by which data group it belongs to.
- *
- * @return key is groupId, value is all SGs belong to this data group
- */
- protected Map<String, Set<String>> classifySGByGroupId(List<String> sgList) {
- Map<String, Set<String>> map = new HashMap<>();
- for (int i = 0; i < sgList.size(); i++) {
- String sg = sgList.get(i);
- String groupId = getGroupIdBySG(sg);
- if (map.containsKey(groupId)) {
- map.get(groupId).add(sg);
- } else {
- Set<String> set = new HashSet<>();
- set.add(sg);
- map.put(groupId, set);
- }
- }
- return map;
- }
-
- /**
- * Get raft group id by storage group name
- */
- protected String getGroupIdBySG(String storageGroup) {
- return router.getGroupID(router.routeGroup(storageGroup));
- }
-
- /**
- * Check if the non query command can execute in local. 1. If this node belongs to the storage
- * group 2. If this node is leader.
- */
- public boolean canHandleNonQueryByGroupId(String groupId) {
- boolean canHandle = false;
- if(groupId.equals(ClusterConfig.METADATA_GROUP_ID)){
- canHandle = ((MetadataRaftHolder) (server.getMetadataHolder())).getFsm().isLeader();
- }else {
- if (router.containPhysicalNodeByGroupId(groupId, localNode) && RaftUtils
- .getPhysicalNodeFrom(RaftUtils.getLeaderPeerID(groupId)).equals(localNode)) {
- canHandle = true;
- }
- }
- return canHandle;
- }
-
- /**
- * Check if the query command can execute in local. Check if this node belongs to the group id
- */
- protected boolean canHandleQueryByGroupId(String groupId) {
- return router.containPhysicalNodeByGroupId(groupId, localNode);
- }
-
- /**
- * Async handle QPTask by QPTask and leader id
- *
- * @param task request QPTask
- * @param leader leader of the target raft group
- * @param taskRetryNum Number of QPTask retries due to timeout and redirected.
- * @return basic response
- */
- protected BasicResponse asyncHandleNonQueryTaskGetRes(QPTask task, PeerId leader,
- int taskRetryNum)
- throws InterruptedException, RaftConnectionException {
- asyncSendNonQueryTask(task, leader, taskRetryNum);
- return asyncGetNonQueryRes(task, leader, taskRetryNum);
- }
-
- /**
- * Asynchronous send rpc task via client
- *
- * @param task rpc task
- * @param leader leader node of the group
- * @param taskRetryNum Retry time of the task
- */
- public void asyncSendNonQueryTask(QPTask task, PeerId leader, int taskRetryNum)
- throws RaftConnectionException {
- if (taskRetryNum >= TASK_MAX_RETRY) {
- throw new RaftConnectionException(String.format("QPTask retries reach the upper bound %s",
- TASK_MAX_RETRY));
- }
- NodeAsClient client = getRaftNodeAsClient();
- /** Call async method **/
- client.asyncHandleRequest(task.getRequest(), leader, task);
- }
-
- /**
- * try to get raft rpc client
- */
- private NodeAsClient getRaftNodeAsClient() throws RaftConnectionException {
- return CLIENT_MANAGER.getRaftNodeAsClient();
- }
-
- /**
- * Asynchronous get task response. If it's redirected or status is exception, the task needs to be
- * resent. Note: If status is Exception, it marks that an exception occurred during the task is
- * being sent instead of executed.
- *
- * @param task rpc task
- * @param leader leader node of the group
- * @param taskRetryNum Retry time of the task
- */
- private BasicResponse asyncGetNonQueryRes(QPTask task, PeerId leader, int taskRetryNum)
- throws InterruptedException, RaftConnectionException {
- task.await();
- if (task.getTaskState() != TaskState.FINISH) {
- if (task.getTaskState() == TaskState.REDIRECT) {
- /** redirect to the right leader **/
- leader = PeerId.parsePeer(task.getResponse().getLeaderStr());
- LOGGER.debug("Redirect leader: {}, group id = {}", leader, task.getRequest().getGroupID());
- RaftUtils.updateRaftGroupLeader(task.getRequest().getGroupID(), leader);
- }
- task.resetTask();
- return asyncHandleNonQueryTaskGetRes(task, leader, taskRetryNum + 1);
- }
- return task.getResponse();
- }
-
- public void shutdown() {
- if (currentTask != null) {
- currentTask.shutdown();
- }
- }
-
- public void setReadMetadataConsistencyLevel(int level) throws ConsistencyLevelException {
- if (level <= ClusterConstant.MAX_CONSISTENCY_LEVEL) {
- this.readMetadataConsistencyLevel = level;
- } else {
- throw new ConsistencyLevelException(String.format("Consistency level %d not support", level));
- }
- }
-
- public void setReadDataConsistencyLevel(int level) throws ConsistencyLevelException {
- if (level <= ClusterConstant.MAX_CONSISTENCY_LEVEL) {
- this.readDataConsistencyLevel = level;
- } else {
- throw new ConsistencyLevelException(String.format("Consistency level %d not support", level));
- }
- }
-
- public int getReadMetadataConsistencyLevel() {
- return readMetadataConsistencyLevel;
- }
-
- public int getReadDataConsistencyLevel() {
- return readDataConsistencyLevel;
- }
-}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/qp/executor/AbstractQPExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/qp/executor/AbstractQPExecutor.java
new file mode 100644
index 0000000..492b7ad
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/qp/executor/AbstractQPExecutor.java
@@ -0,0 +1,178 @@
+/**
+ * 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.cluster.qp.executor;
+
+import com.alipay.sofa.jraft.entity.PeerId;
+import org.apache.iotdb.cluster.config.ClusterConfig;
+import org.apache.iotdb.cluster.config.ClusterConstant;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.entity.Server;
+import org.apache.iotdb.cluster.exception.ConsistencyLevelException;
+import org.apache.iotdb.cluster.exception.RaftConnectionException;
+import org.apache.iotdb.cluster.qp.task.QPTask;
+import org.apache.iotdb.cluster.qp.task.QPTask.TaskState;
+import org.apache.iotdb.cluster.qp.task.SingleQPTask;
+import org.apache.iotdb.cluster.rpc.raft.NodeAsClient;
+import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
+import org.apache.iotdb.cluster.utils.RaftUtils;
+import org.apache.iotdb.cluster.utils.hash.Router;
+import org.apache.iotdb.db.metadata.MManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractQPExecutor {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(AbstractQPExecutor.class);
+
+ private static final ClusterConfig CLUSTER_CONFIG = ClusterDescriptor.getInstance().getConfig();
+
+ protected Router router = Router.getInstance();
+
+ protected MManager mManager = MManager.getInstance();
+
+ protected final Server server = Server.getInstance();
+
+ /**
+ * The task in progress.
+ */
+ protected ThreadLocal<QPTask> currentTask = new ThreadLocal<>();
+
+ /**
+ * Count limit to redo a single task
+ */
+ private static final int TASK_MAX_RETRY = CLUSTER_CONFIG.getQpTaskRedoCount();
+
+ /**
+ * ReadMetadataConsistencyLevel: 1 Strong consistency, 2 Weak consistency
+ */
+ private ThreadLocal<Integer> readMetadataConsistencyLevel = new ThreadLocal<>();
+
+ /**
+ * ReadDataConsistencyLevel: 1 Strong consistency, 2 Weak consistency
+ */
+ private ThreadLocal<Integer> readDataConsistencyLevel = new ThreadLocal<>();
+
+ public AbstractQPExecutor() {
+ }
+
+ /**
+ * Check init of consistency level(<code>ThreadLocal</code>)
+ */
+ private void checkInitConsistencyLevel() {
+ if (readMetadataConsistencyLevel.get() == null) {
+ readMetadataConsistencyLevel.set(CLUSTER_CONFIG.getReadMetadataConsistencyLevel());
+ }
+ if (readDataConsistencyLevel.get() == null) {
+ readDataConsistencyLevel.set(CLUSTER_CONFIG.getReadDataConsistencyLevel());
+ }
+ }
+
+ /**
+ * Async handle QPTask by QPTask and leader id
+ *
+ * @param task request QPTask
+ * @param leader leader of the target raft group
+ * @param taskRetryNum Number of QPTask retries due to timeout and redirected.
+ * @return basic response
+ */
+ protected BasicResponse asyncHandleNonQuerySingleTaskGetRes(SingleQPTask task, PeerId leader,
+ int taskRetryNum)
+ throws InterruptedException, RaftConnectionException {
+ asyncSendNonQuerySingleTask(task, leader, taskRetryNum);
+ return syncGetNonQueryRes(task, leader, taskRetryNum);
+ }
+
+ /**
+ * Asynchronous send rpc task via client
+ * @param task rpc task
+ * @param leader leader node of the group
+ * @param taskRetryNum Retry time of the task
+ */
+ protected void asyncSendNonQuerySingleTask(SingleQPTask task, PeerId leader, int taskRetryNum)
+ throws RaftConnectionException {
+ if (taskRetryNum >= TASK_MAX_RETRY) {
+ throw new RaftConnectionException(String.format("QPTask retries reach the upper bound %s",
+ TASK_MAX_RETRY));
+ }
+ NodeAsClient client = RaftUtils.getRaftNodeAsClient();
+ /** Call async method **/
+ client.asyncHandleRequest(task.getRequest(), leader, task);
+ }
+
+ /**
+ * Synchronous get task response. If it's redirected or status is exception, the task needs to be
+ * resent. Note: If status is Exception, it marks that an exception occurred during the task is
+ * being sent instead of executed.
+ * @param task rpc task
+ * @param leader leader node of the group
+ * @param taskRetryNum Retry time of the task
+ */
+ private BasicResponse syncGetNonQueryRes(SingleQPTask task, PeerId leader, int taskRetryNum)
+ throws InterruptedException, RaftConnectionException {
+ task.await();
+ if (task.getTaskState() != TaskState.FINISH) {
+ if (task.getTaskState() == TaskState.REDIRECT) {
+ /** redirect to the right leader **/
+ leader = PeerId.parsePeer(task.getResponse().getLeaderStr());
+ LOGGER.debug("Redirect leader: {}, group id = {}", leader, task.getRequest().getGroupID());
+ RaftUtils.updateRaftGroupLeader(task.getRequest().getGroupID(), leader);
+ } else {
+ String groupId = task.getRequest().getGroupID();
+ RaftUtils.removeCachedRaftGroupLeader(groupId);
+ LOGGER.debug("Remove cached raft group leader of {}", groupId);
+ leader = RaftUtils.getLeaderPeerID(groupId);
+ }
+ task.resetTask();
+ return asyncHandleNonQuerySingleTaskGetRes(task, leader, taskRetryNum + 1);
+ }
+ return task.getResponse();
+ }
+
+ public void shutdown() {
+ if (currentTask.get() != null) {
+ currentTask.get().shutdown();
+ }
+ }
+
+ public void setReadMetadataConsistencyLevel(int level) throws ConsistencyLevelException {
+ if (level <= ClusterConstant.MAX_CONSISTENCY_LEVEL) {
+ readMetadataConsistencyLevel.set(level);
+ } else {
+ throw new ConsistencyLevelException(String.format("Consistency level %d not support", level));
+ }
+ }
+
+ public void setReadDataConsistencyLevel(int level) throws ConsistencyLevelException {
+ if (level <= ClusterConstant.MAX_CONSISTENCY_LEVEL) {
+ readDataConsistencyLevel.set(level);
+ } else {
+ throw new ConsistencyLevelException(String.format("Consistency level %d not support", level));
+ }
+ }
+
+ public int getReadMetadataConsistencyLevel() {
+ checkInitConsistencyLevel();
+ return readMetadataConsistencyLevel.get();
+ }
+
+ public int getReadDataConsistencyLevel() {
+ checkInitConsistencyLevel();
+ return readDataConsistencyLevel.get();
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/qp/executor/ClusterQueryProcessExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/qp/executor/ClusterQueryProcessExecutor.java
new file mode 100644
index 0000000..c5032fc
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/qp/executor/ClusterQueryProcessExecutor.java
@@ -0,0 +1,177 @@
+/**
+ * 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.cluster.qp.executor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iotdb.cluster.query.executor.ClusterQueryRouter;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.executor.IQueryProcessExecutor;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
+import org.apache.iotdb.db.qp.physical.crud.FillQueryPlan;
+import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.fill.IFill;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.expression.IExpression;
+import org.apache.iotdb.tsfile.read.expression.QueryExpression;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public class ClusterQueryProcessExecutor extends AbstractQPExecutor implements IQueryProcessExecutor {
+
+ private ThreadLocal<Integer> fetchSize = new ThreadLocal<>();
+ private ClusterQueryRouter clusterQueryRouter = new ClusterQueryRouter();
+
+ private QueryMetadataExecutor queryMetadataExecutor = new QueryMetadataExecutor();
+
+ @Override
+ public QueryDataSet processQuery(QueryPlan queryPlan, QueryContext context)
+ throws IOException, FileNodeManagerException, PathErrorException,
+ QueryFilterOptimizationException, ProcessorException {
+
+ QueryExpression queryExpression = QueryExpression.create().setSelectSeries(queryPlan.getPaths())
+ .setExpression(queryPlan.getExpression());
+ clusterQueryRouter.setReadDataConsistencyLevel(getReadDataConsistencyLevel());
+ if (queryPlan instanceof GroupByPlan) {
+ GroupByPlan groupByPlan = (GroupByPlan) queryPlan;
+ return groupBy(groupByPlan.getPaths(), groupByPlan.getAggregations(),
+ groupByPlan.getExpression(), groupByPlan.getUnit(), groupByPlan.getOrigin(),
+ groupByPlan.getIntervals(), context);
+ }
+
+ if (queryPlan instanceof AggregationPlan) {
+ return aggregate(queryPlan.getPaths(), queryPlan.getAggregations(),
+ queryPlan.getExpression(), context);
+ }
+
+ if (queryPlan instanceof FillQueryPlan) {
+ FillQueryPlan fillQueryPlan = (FillQueryPlan) queryPlan;
+ return fill(queryPlan.getPaths(), fillQueryPlan.getQueryTime(),
+ fillQueryPlan.getFillType(), context);
+ }
+ return clusterQueryRouter.query(queryExpression, context);
+ }
+
+ @Override
+ public QueryDataSet aggregate(List<Path> paths, List<String> aggres, IExpression expression,
+ QueryContext context)
+ throws ProcessorException, IOException, PathErrorException, FileNodeManagerException, QueryFilterOptimizationException {
+ return clusterQueryRouter.aggregate(paths, aggres, expression, context);
+ }
+
+ @Override
+ public QueryDataSet groupBy(List<Path> paths, List<String> aggres, IExpression expression,
+ long unit, long origin, List<Pair<Long, Long>> intervals, QueryContext context)
+ throws ProcessorException, IOException, PathErrorException, FileNodeManagerException, QueryFilterOptimizationException {
+ return clusterQueryRouter.groupBy(paths, aggres, expression, unit, origin, intervals, context);
+ }
+
+ @Override
+ public QueryDataSet fill(List<Path> fillPaths, long queryTime, Map<TSDataType, IFill> fillTypes,
+ QueryContext context)
+ throws ProcessorException, IOException, PathErrorException, FileNodeManagerException {
+ return clusterQueryRouter.fill(fillPaths, queryTime, fillTypes, context);
+ }
+
+ @Override
+ public TSDataType getSeriesType(Path path) throws PathErrorException {
+ if (path.equals(SQLConstant.RESERVED_TIME)) {
+ return TSDataType.INT64;
+ }
+ if (path.equals(SQLConstant.RESERVED_FREQ)) {
+ return TSDataType.FLOAT;
+ }
+ try {
+ return queryMetadataExecutor.processSeriesTypeQuery(path.getFullPath());
+ } catch (InterruptedException | ProcessorException e) {
+ throw new PathErrorException(e.getMessage());
+ }
+ }
+
+ @Override
+ public List<String> getAllPaths(String originPath)
+ throws PathErrorException {
+ try {
+ return queryMetadataExecutor.processPathsQuery(originPath);
+ } catch (InterruptedException | ProcessorException e) {
+ throw new PathErrorException(e.getMessage());
+ }
+ }
+
+ @Override
+ public boolean judgePathExists(Path fullPath) {
+ try {
+ List<List<String>> results = queryMetadataExecutor.processTimeSeriesQuery(fullPath.toString());
+ return !results.isEmpty();
+ } catch (InterruptedException | PathErrorException | ProcessorException e) {
+ return false;
+ }
+ }
+
+ @Override
+ public int getFetchSize() {
+ return fetchSize.get();
+ }
+
+ @Override
+ public void setFetchSize(int fetchSize) {
+ this.fetchSize.set(fetchSize);
+ }
+
+ @Override
+ public boolean update(Path path, long startTime, long endTime, String value)
+ throws ProcessorException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean delete(List<Path> paths, long deleteTime) throws ProcessorException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean delete(Path path, long deleteTime) throws ProcessorException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public int insert(Path path, long insertTime, String value) throws ProcessorException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public int multiInsert(String deviceId, long insertTime, List<String> measurementList,
+ List<String> insertValues) throws ProcessorException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean processNonQuery(PhysicalPlan plan) throws ProcessorException {
+ throw new UnsupportedOperationException();
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/qp/executor/NonQueryExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/qp/executor/NonQueryExecutor.java
index c8c2a9b..1420370 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/qp/executor/NonQueryExecutor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/qp/executor/NonQueryExecutor.java
@@ -34,17 +34,17 @@ import org.apache.iotdb.cluster.entity.raft.DataPartitionRaftHolder;
import org.apache.iotdb.cluster.entity.raft.MetadataRaftHolder;
import org.apache.iotdb.cluster.entity.raft.RaftService;
import org.apache.iotdb.cluster.exception.RaftConnectionException;
-import org.apache.iotdb.cluster.qp.ClusterQPExecutor;
-import org.apache.iotdb.cluster.qp.callback.BatchQPTask;
-import org.apache.iotdb.cluster.qp.callback.QPTask;
-import org.apache.iotdb.cluster.qp.callback.SingleQPTask;
+import org.apache.iotdb.cluster.qp.task.BatchQPTask;
+import org.apache.iotdb.cluster.qp.task.QPTask;
+import org.apache.iotdb.cluster.qp.task.SingleQPTask;
import org.apache.iotdb.cluster.rpc.raft.request.BasicRequest;
-import org.apache.iotdb.cluster.rpc.raft.request.DataGroupNonQueryRequest;
-import org.apache.iotdb.cluster.rpc.raft.request.MetaGroupNonQueryRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.nonquery.DataGroupNonQueryRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.nonquery.MetaGroupNonQueryRequest;
import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
-import org.apache.iotdb.cluster.rpc.raft.response.DataGroupNonQueryResponse;
-import org.apache.iotdb.cluster.rpc.raft.response.MetaGroupNonQueryResponse;
-import org.apache.iotdb.cluster.rpc.service.TSServiceClusterImpl.BatchResult;
+import org.apache.iotdb.cluster.rpc.raft.response.nonquery.DataGroupNonQueryResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.nonquery.MetaGroupNonQueryResponse;
+import org.apache.iotdb.cluster.service.TSServiceClusterImpl.BatchResult;
+import org.apache.iotdb.cluster.utils.QPExecutorUtils;
import org.apache.iotdb.cluster.utils.RaftUtils;
import org.apache.iotdb.db.exception.PathErrorException;
import org.apache.iotdb.db.exception.ProcessorException;
@@ -61,7 +61,7 @@ import org.slf4j.LoggerFactory;
/**
* Handle distributed non-query logic
*/
-public class NonQueryExecutor extends ClusterQPExecutor {
+public class NonQueryExecutor extends AbstractQPExecutor {
private static final Logger LOGGER = LoggerFactory.getLogger(NonQueryExecutor.class);
@@ -100,10 +100,13 @@ public class NonQueryExecutor extends ClusterQPExecutor {
* @param batchResult batch result
*/
public void processBatch(PhysicalPlan[] physicalPlans, BatchResult batchResult)
- throws InterruptedException {
+ throws InterruptedException, ProcessorException {
Status nullReadTaskStatus = Status.OK();
RaftUtils.handleNullReadToMetaGroup(nullReadTaskStatus);
+ if(!nullReadTaskStatus.isOk()){
+ throw new ProcessorException("Null read while processing batch failed");
+ }
nullReaderEnable = false;
/** 1. Classify physical plans by group id **/
@@ -112,12 +115,12 @@ public class NonQueryExecutor extends ClusterQPExecutor {
classifyPhysicalPlanByGroupId(physicalPlans, batchResult, physicalPlansMap, planIndexMap);
/** 2. Construct Multiple Data Group Requests **/
- Map<String, QPTask> subTaskMap = new HashMap<>();
+ Map<String, SingleQPTask> subTaskMap = new HashMap<>();
constructMultipleRequests(physicalPlansMap, planIndexMap, subTaskMap, batchResult);
/** 3. Execute Multiple Sub Tasks **/
BatchQPTask task = new BatchQPTask(subTaskMap.size(), batchResult, subTaskMap, planIndexMap);
- currentTask = task;
+ currentTask.set(task);
task.execute(this);
task.await();
batchResult.setAllSuccessful(task.isAllSuccessful());
@@ -165,7 +168,7 @@ public class NonQueryExecutor extends ClusterQPExecutor {
* Construct multiple data group requests
*/
private void constructMultipleRequests(Map<String, List<PhysicalPlan>> physicalPlansMap,
- Map<String, List<Integer>> planIndexMap, Map<String, QPTask> subTaskMap,
+ Map<String, List<Integer>> planIndexMap, Map<String, SingleQPTask> subTaskMap,
BatchResult batchResult) {
int[] result = batchResult.getResult();
for (Entry<String, List<PhysicalPlan>> entry : physicalPlansMap.entrySet()) {
@@ -197,16 +200,16 @@ public class NonQueryExecutor extends ClusterQPExecutor {
switch (plan.getOperatorType()) {
case DELETE:
storageGroup = getStorageGroupFromDeletePlan((DeletePlan) plan);
- groupId = getGroupIdBySG(storageGroup);
+ groupId = router.getGroupIdBySG(storageGroup);
break;
case UPDATE:
Path path = ((UpdatePlan) plan).getPath();
- storageGroup = getStroageGroupByDevice(path.getDevice());
- groupId = getGroupIdBySG(storageGroup);
+ storageGroup = QPExecutorUtils.getStroageGroupByDevice(path.getDevice());
+ groupId = router.getGroupIdBySG(storageGroup);
break;
case INSERT:
- storageGroup = getStroageGroupByDevice(((InsertPlan) plan).getDeviceId());
- groupId = getGroupIdBySG(storageGroup);
+ storageGroup = QPExecutorUtils.getStroageGroupByDevice(((InsertPlan) plan).getDeviceId());
+ groupId = router.getGroupIdBySG(storageGroup);
break;
case CREATE_ROLE:
case DELETE_ROLE:
@@ -285,8 +288,8 @@ public class NonQueryExecutor extends ClusterQPExecutor {
case ADD_PATH:
case DELETE_PATH:
String deviceId = path.getDevice();
- String storageGroup = getStroageGroupByDevice(deviceId);
- groupId = getGroupIdBySG(storageGroup);
+ String storageGroup = QPExecutorUtils.getStroageGroupByDevice(deviceId);
+ groupId = router.getGroupIdBySG(storageGroup);
break;
case SET_FILE_LEVEL:
boolean fileLevelExist = mManager.checkStorageLevelOfMTree(path.getFullPath());
@@ -316,11 +319,11 @@ public class NonQueryExecutor extends ClusterQPExecutor {
} else {
request = new DataGroupNonQueryRequest(groupId, plans);
}
- QPTask qpTask = new SingleQPTask(false, request);
- currentTask = qpTask;
+ SingleQPTask qpTask = new SingleQPTask(false, request);
+ currentTask.set(qpTask);
/** Check if the plan can be executed locally. **/
- if (canHandleNonQueryByGroupId(groupId)) {
+ if (QPExecutorUtils.canHandleNonQueryByGroupId(groupId)) {
return handleNonQueryRequestLocally(groupId, qpTask);
} else {
PeerId leader = RaftUtils.getLeaderPeerID(groupId);
@@ -358,9 +361,9 @@ public class NonQueryExecutor extends ClusterQPExecutor {
* @param leader leader of the target raft group
* @return request result
*/
- public boolean asyncHandleNonQueryTask(QPTask task, PeerId leader)
+ public boolean asyncHandleNonQueryTask(SingleQPTask task, PeerId leader)
throws RaftConnectionException, InterruptedException {
- BasicResponse response = asyncHandleNonQueryTaskGetRes(task, leader, 0);
+ BasicResponse response = asyncHandleNonQuerySingleTaskGetRes(task, leader, 0);
return response != null && response.isSuccess();
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/qp/executor/QueryMetadataExecutor.java b/cluster/src/main/java/org/apache/iotdb/cluster/qp/executor/QueryMetadataExecutor.java
index 1dfbc7e..82325e1 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/qp/executor/QueryMetadataExecutor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/qp/executor/QueryMetadataExecutor.java
@@ -26,27 +26,26 @@ import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
-import org.apache.iotdb.cluster.qp.callback.SingleQPTask;
+import org.apache.iotdb.cluster.qp.task.SingleQPTask;
import org.apache.iotdb.cluster.config.ClusterConfig;
import org.apache.iotdb.cluster.config.ClusterConstant;
-import org.apache.iotdb.cluster.entity.raft.DataPartitionRaftHolder;
import org.apache.iotdb.cluster.entity.raft.MetadataRaftHolder;
import org.apache.iotdb.cluster.entity.raft.RaftService;
import org.apache.iotdb.cluster.exception.RaftConnectionException;
-import org.apache.iotdb.cluster.qp.ClusterQPExecutor;
-import org.apache.iotdb.cluster.rpc.raft.request.QueryMetadataInStringRequest;
-import org.apache.iotdb.cluster.rpc.raft.request.QueryMetadataRequest;
-import org.apache.iotdb.cluster.rpc.raft.request.QueryPathsRequest;
-import org.apache.iotdb.cluster.rpc.raft.request.QuerySeriesTypeRequest;
-import org.apache.iotdb.cluster.rpc.raft.request.QueryStorageGroupRequest;
-import org.apache.iotdb.cluster.rpc.raft.request.QueryTimeSeriesRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querymetadata.QueryMetadataInStringRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querymetadata.QueryMetadataRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querymetadata.QueryPathsRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querymetadata.QuerySeriesTypeRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querymetadata.QueryStorageGroupRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querymetadata.QueryTimeSeriesRequest;
import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
-import org.apache.iotdb.cluster.rpc.raft.response.QueryMetadataInStringResponse;
-import org.apache.iotdb.cluster.rpc.raft.response.QueryMetadataResponse;
-import org.apache.iotdb.cluster.rpc.raft.response.QueryPathsResponse;
-import org.apache.iotdb.cluster.rpc.raft.response.QuerySeriesTypeResponse;
-import org.apache.iotdb.cluster.rpc.raft.response.QueryStorageGroupResponse;
-import org.apache.iotdb.cluster.rpc.raft.response.QueryTimeSeriesResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querymetadata.QueryMetadataInStringResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querymetadata.QueryMetadataResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querymetadata.QueryPathsResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querymetadata.QuerySeriesTypeResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querymetadata.QueryStorageGroupResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querymetadata.QueryTimeSeriesResponse;
+import org.apache.iotdb.cluster.utils.QPExecutorUtils;
import org.apache.iotdb.cluster.utils.RaftUtils;
import org.apache.iotdb.db.exception.PathErrorException;
import org.apache.iotdb.db.exception.ProcessorException;
@@ -59,7 +58,7 @@ import org.slf4j.LoggerFactory;
/**
* Handle < show timeseries <path> > logic
*/
-public class QueryMetadataExecutor extends ClusterQPExecutor {
+public class QueryMetadataExecutor extends AbstractQPExecutor {
private static final Logger LOGGER = LoggerFactory.getLogger(QueryMetadataExecutor.class);
private static final String DOUB_SEPARATOR = "\\.";
@@ -83,7 +82,7 @@ public class QueryMetadataExecutor extends ClusterQPExecutor {
if (storageGroupList.isEmpty()) {
return new ArrayList<>();
} else {
- Map<String, Set<String>> groupIdSGMap = classifySGByGroupId(storageGroupList);
+ Map<String, Set<String>> groupIdSGMap = QPExecutorUtils.classifySGByGroupId(storageGroupList);
for (Entry<String, Set<String>> entry : groupIdSGMap.entrySet()) {
List<String> paths = getSubQueryPaths(entry.getValue(), path);
String groupId = entry.getKey();
@@ -126,13 +125,13 @@ public class QueryMetadataExecutor extends ClusterQPExecutor {
private void handleTimseriesQuery(String groupId, List<String> pathList, List<List<String>> res)
throws ProcessorException, InterruptedException {
QueryTimeSeriesRequest request = new QueryTimeSeriesRequest(groupId,
- readMetadataConsistencyLevel, pathList);
+ getReadMetadataConsistencyLevel(), pathList);
SingleQPTask task = new SingleQPTask(false, request);
LOGGER.debug("Execute show timeseries {} statement for group {}.", pathList, groupId);
PeerId holder;
/** Check if the plan can be executed locally. **/
- if (canHandleQueryByGroupId(groupId)) {
+ if (QPExecutorUtils.canHandleQueryByGroupId(groupId)) {
LOGGER.debug("Execute show timeseries {} statement locally for group {} by sending request to local node.", pathList, groupId);
holder = this.server.getServerId();
} else {
@@ -153,21 +152,21 @@ public class QueryMetadataExecutor extends ClusterQPExecutor {
List<SingleQPTask> taskList = new ArrayList<>();
for (String groupId : groupIdSet) {
QueryMetadataInStringRequest request = new QueryMetadataInStringRequest(groupId,
- readMetadataConsistencyLevel);
+ getReadMetadataConsistencyLevel());
SingleQPTask task = new SingleQPTask(false, request);
taskList.add(task);
LOGGER.debug("Execute show metadata in string statement for group {}.", groupId);
PeerId holder;
/** Check if the plan can be executed locally. **/
- if (canHandleQueryByGroupId(groupId)) {
+ if (QPExecutorUtils.canHandleQueryByGroupId(groupId)) {
LOGGER.debug("Execute show metadata in string statement locally for group {} by sending request to local node.", groupId);
holder = this.server.getServerId();
} else {
holder = RaftUtils.getRandomPeerID(groupId);
}
try {
- asyncSendNonQueryTask(task, holder, 0);
+ asyncSendNonQuerySingleTask(task, holder, 0);
} catch (RaftConnectionException e) {
throw new ProcessorException("Raft connection occurs error.", e);
}
@@ -192,21 +191,21 @@ public class QueryMetadataExecutor extends ClusterQPExecutor {
List<SingleQPTask> taskList = new ArrayList<>();
for (String groupId : groupIdSet) {
QueryMetadataRequest request = new QueryMetadataRequest(groupId,
- readMetadataConsistencyLevel);
+ getReadMetadataConsistencyLevel());
SingleQPTask task = new SingleQPTask(false, request);
taskList.add(task);
LOGGER.debug("Execute query metadata statement for group {}.", groupId);
PeerId holder;
/** Check if the plan can be executed locally. **/
- if (canHandleQueryByGroupId(groupId)) {
+ if (QPExecutorUtils.canHandleQueryByGroupId(groupId)) {
LOGGER.debug("Execute query metadata statement locally for group {} by sending request to local node.", groupId);
holder = this.server.getServerId();
} else {
holder = RaftUtils.getRandomPeerID(groupId);
}
try {
- asyncSendNonQueryTask(task, holder, 0);
+ asyncSendNonQuerySingleTask(task, holder, 0);
} catch (RaftConnectionException e) {
throw new ProcessorException("Raft connection occurs error.", e);
}
@@ -229,20 +228,20 @@ public class QueryMetadataExecutor extends ClusterQPExecutor {
public TSDataType processSeriesTypeQuery(String path)
throws InterruptedException, ProcessorException, PathErrorException {
- TSDataType dataType = null;
+ TSDataType dataType;
List<String> storageGroupList = mManager.getAllFileNamesByPath(path);
if (storageGroupList.size() != 1) {
throw new PathErrorException("path " + path + " is not valid.");
} else {
- String groupId = getGroupIdBySG(storageGroupList.get(0));
+ String groupId = router.getGroupIdBySG(storageGroupList.get(0));
QuerySeriesTypeRequest request = new QuerySeriesTypeRequest(groupId,
- readMetadataConsistencyLevel, path);
+ getReadMetadataConsistencyLevel(), path);
SingleQPTask task = new SingleQPTask(false, request);
LOGGER.debug("Execute get series type for {} statement for group {}.", path, groupId);
PeerId holder;
/** Check if the plan can be executed locally. **/
- if (canHandleQueryByGroupId(groupId)) {
+ if (QPExecutorUtils.canHandleQueryByGroupId(groupId)) {
LOGGER.debug("Execute get series type for {} statement locally for group {} by sending request to local node.", path, groupId);
holder = this.server.getServerId();
} else {
@@ -267,7 +266,7 @@ public class QueryMetadataExecutor extends ClusterQPExecutor {
if (storageGroupList.isEmpty()) {
return new ArrayList<>();
} else {
- Map<String, Set<String>> groupIdSGMap = classifySGByGroupId(storageGroupList);
+ Map<String, Set<String>> groupIdSGMap = QPExecutorUtils.classifySGByGroupId(storageGroupList);
for (Entry<String, Set<String>> entry : groupIdSGMap.entrySet()) {
List<String> paths = getSubQueryPaths(entry.getValue(), path);
String groupId = entry.getKey();
@@ -285,13 +284,13 @@ public class QueryMetadataExecutor extends ClusterQPExecutor {
private void handlePathsQuery(String groupId, List<String> pathList, List<String> res)
throws ProcessorException, InterruptedException {
QueryPathsRequest request = new QueryPathsRequest(groupId,
- readMetadataConsistencyLevel, pathList);
+ getReadMetadataConsistencyLevel(), pathList);
SingleQPTask task = new SingleQPTask(false, request);
LOGGER.debug("Execute get paths for {} statement for group {}.", pathList, groupId);
PeerId holder;
/** Check if the plan can be executed locally. **/
- if (canHandleQueryByGroupId(groupId)) {
+ if (QPExecutorUtils.canHandleQueryByGroupId(groupId)) {
LOGGER.debug("Execute get paths for {} statement locally for group {} by sending request to local node.", pathList, groupId);
holder = this.server.getServerId();
} else {
@@ -306,14 +305,14 @@ public class QueryMetadataExecutor extends ClusterQPExecutor {
private List<List<String>> queryTimeSeries(SingleQPTask task, PeerId leader)
throws InterruptedException, RaftConnectionException {
- BasicResponse response = asyncHandleNonQueryTaskGetRes(task, leader, 0);
+ BasicResponse response = asyncHandleNonQuerySingleTaskGetRes(task, leader, 0);
return response == null ? new ArrayList<>()
: ((QueryTimeSeriesResponse) response).getTimeSeries();
}
private TSDataType querySeriesType(SingleQPTask task, PeerId leader)
throws InterruptedException, RaftConnectionException {
- BasicResponse response = asyncHandleNonQueryTaskGetRes(task, leader, 0);
+ BasicResponse response = asyncHandleNonQuerySingleTaskGetRes(task, leader, 0);
return response == null ? null
: ((QuerySeriesTypeResponse) response).getDataType();
}
@@ -326,10 +325,10 @@ public class QueryMetadataExecutor extends ClusterQPExecutor {
private Set<String> queryStorageGroupLocally() throws InterruptedException {
final byte[] reqContext = RaftUtils.createRaftRequestContext();
QueryStorageGroupRequest request = new QueryStorageGroupRequest(
- ClusterConfig.METADATA_GROUP_ID, readMetadataConsistencyLevel);
+ ClusterConfig.METADATA_GROUP_ID, getReadMetadataConsistencyLevel());
SingleQPTask task = new SingleQPTask(false, request);
MetadataRaftHolder metadataHolder = (MetadataRaftHolder) server.getMetadataHolder();
- if (readMetadataConsistencyLevel == ClusterConstant.WEAK_CONSISTENCY_LEVEL) {
+ if (getReadMetadataConsistencyLevel() == ClusterConstant.WEAK_CONSISTENCY_LEVEL) {
QueryStorageGroupResponse response;
try {
response = QueryStorageGroupResponse
@@ -365,7 +364,7 @@ public class QueryMetadataExecutor extends ClusterQPExecutor {
private List<String> queryPaths(SingleQPTask task, PeerId leader)
throws InterruptedException, RaftConnectionException {
- BasicResponse response = asyncHandleNonQueryTaskGetRes(task, leader, 0);
+ BasicResponse response = asyncHandleNonQuerySingleTaskGetRes(task, leader, 0);
return response == null ? new ArrayList<>()
: ((QueryPathsResponse) response).getPaths();
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/qp/callback/BatchQPTask.java b/cluster/src/main/java/org/apache/iotdb/cluster/qp/task/BatchQPTask.java
similarity index 90%
rename from cluster/src/main/java/org/apache/iotdb/cluster/qp/callback/BatchQPTask.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/qp/task/BatchQPTask.java
index 2706388..43edd67 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/qp/callback/BatchQPTask.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/qp/task/BatchQPTask.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.qp.callback;
+package org.apache.iotdb.cluster.qp.task;
import com.alipay.sofa.jraft.entity.PeerId;
import java.sql.Statement;
@@ -30,8 +30,9 @@ import org.apache.iotdb.cluster.concurrent.pool.QPTaskManager;
import org.apache.iotdb.cluster.exception.RaftConnectionException;
import org.apache.iotdb.cluster.qp.executor.NonQueryExecutor;
import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
-import org.apache.iotdb.cluster.rpc.raft.response.DataGroupNonQueryResponse;
-import org.apache.iotdb.cluster.rpc.service.TSServiceClusterImpl.BatchResult;
+import org.apache.iotdb.cluster.rpc.raft.response.nonquery.DataGroupNonQueryResponse;
+import org.apache.iotdb.cluster.service.TSServiceClusterImpl.BatchResult;
+import org.apache.iotdb.cluster.utils.QPExecutorUtils;
import org.apache.iotdb.cluster.utils.RaftUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -73,7 +74,7 @@ public class BatchQPTask extends MultiQPTask {
private NonQueryExecutor executor;
- public BatchQPTask(int taskNum, BatchResult batchResult, Map<String, QPTask> taskMap,
+ public BatchQPTask(int taskNum, BatchResult batchResult, Map<String, SingleQPTask> taskMap,
Map<String, List<Integer>> planIndexMap) {
super(false, taskNum, TaskType.BATCH);
this.batchResult = batchResult.getResult();
@@ -117,11 +118,11 @@ public class BatchQPTask extends MultiQPTask {
public void execute(NonQueryExecutor executor) {
this.executor = executor;
- for (Entry<String, QPTask> entry : taskMap.entrySet()) {
+ for (Entry<String, SingleQPTask> entry : taskMap.entrySet()) {
String groupId = entry.getKey();
- QPTask subTask = entry.getValue();
+ SingleQPTask subTask = entry.getValue();
Future<?> taskThread;
- if (executor.canHandleNonQueryByGroupId(groupId)) {
+ if (QPExecutorUtils.canHandleNonQueryByGroupId(groupId)) {
taskThread = QPTaskManager.getInstance()
.submit(() -> executeLocalSubTask(subTask, groupId));
} else {
@@ -149,7 +150,7 @@ public class BatchQPTask extends MultiQPTask {
/**
* Execute RPC sub task
*/
- private void executeRpcSubTask(QPTask subTask, PeerId leader, String groupId) {
+ private void executeRpcSubTask(SingleQPTask subTask, PeerId leader, String groupId) {
try {
executor.asyncHandleNonQueryTask(subTask, leader);
this.run(subTask.getResponse());
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/qp/callback/MultiQPTask.java b/cluster/src/main/java/org/apache/iotdb/cluster/qp/task/MultiQPTask.java
similarity index 89%
rename from cluster/src/main/java/org/apache/iotdb/cluster/qp/callback/MultiQPTask.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/qp/task/MultiQPTask.java
index f400eaf..e451f3e 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/qp/callback/MultiQPTask.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/qp/task/MultiQPTask.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.qp.callback;
+package org.apache.iotdb.cluster.qp.task;
import java.util.Map;
import java.util.concurrent.Future;
@@ -26,7 +26,7 @@ public abstract class MultiQPTask extends QPTask {
/**
* Each request is corresponding to a group id. String: group id
*/
- Map<String, QPTask> taskMap;
+ Map<String, SingleQPTask> taskMap;
/**
* Each future task handle a request in taskMap, which is corresponding to a group id. String:
@@ -45,6 +45,8 @@ public abstract class MultiQPTask extends QPTask {
task.cancel(true);
}
}
- this.taskCountDownLatch.countDown();
+ while (taskCountDownLatch.getCount() != 0) {
+ this.taskCountDownLatch.countDown();
+ }
}
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/qp/callback/QPTask.java b/cluster/src/main/java/org/apache/iotdb/cluster/qp/task/QPTask.java
similarity index 98%
rename from cluster/src/main/java/org/apache/iotdb/cluster/qp/callback/QPTask.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/qp/task/QPTask.java
index fd21f3f..96a517a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/qp/callback/QPTask.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/qp/task/QPTask.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.qp.callback;
+package org.apache.iotdb.cluster.qp.task;
import java.util.concurrent.CountDownLatch;
import org.apache.iotdb.cluster.entity.Server;
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/basic/BinaryFilter.java b/cluster/src/main/java/org/apache/iotdb/cluster/qp/task/QueryTask.java
old mode 100755
new mode 100644
similarity index 55%
copy from tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/basic/BinaryFilter.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/qp/task/QueryTask.java
index aeba875..f4cb4b5
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/basic/BinaryFilter.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/qp/task/QueryTask.java
@@ -16,35 +16,34 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.tsfile.read.filter.basic;
+package org.apache.iotdb.cluster.qp.task;
-import java.io.Serializable;
+import org.apache.iotdb.cluster.qp.task.QPTask.TaskState;
+import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
-/**
- * Definition for binary filter operations.
- */
-public abstract class BinaryFilter implements Filter, Serializable {
+public class QueryTask {
+ private BasicResponse basicResponse;
+ private TaskState state;
- private static final long serialVersionUID = 1039585564327602465L;
-
- protected final Filter left;
- protected final Filter right;
+ public QueryTask(BasicResponse basicResponse,
+ TaskState state) {
+ this.basicResponse = basicResponse;
+ this.state = state;
+ }
- protected BinaryFilter(Filter left, Filter right) {
- this.left = left;
- this.right = right;
+ public BasicResponse getBasicResponse() {
+ return basicResponse;
}
- public Filter getLeft() {
- return left;
+ public void setBasicResponse(BasicResponse basicResponse) {
+ this.basicResponse = basicResponse;
}
- public Filter getRight() {
- return right;
+ public TaskState getState() {
+ return state;
}
- @Override
- public String toString() {
- return "( " + left + "," + right + " )";
+ public void setState(TaskState state) {
+ this.state = state;
}
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/qp/callback/SingleQPTask.java b/cluster/src/main/java/org/apache/iotdb/cluster/qp/task/SingleQPTask.java
similarity index 93%
rename from cluster/src/main/java/org/apache/iotdb/cluster/qp/callback/SingleQPTask.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/qp/task/SingleQPTask.java
index 7fc7ba2..805834e 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/qp/callback/SingleQPTask.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/qp/task/SingleQPTask.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.qp.callback;
+package org.apache.iotdb.cluster.qp.task;
import org.apache.iotdb.cluster.rpc.raft.request.BasicRequest;
import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
@@ -57,6 +57,8 @@ public class SingleQPTask extends QPTask {
@Override
public void shutdown() {
- this.taskCountDownLatch.countDown();
+ if (taskCountDownLatch.getCount() != 0) {
+ this.taskCountDownLatch.countDown();
+ }
}
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/IBinaryExpression.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/PathType.java
similarity index 81%
copy from tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/IBinaryExpression.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/query/PathType.java
index b97310a..78c54f7 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/IBinaryExpression.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/PathType.java
@@ -16,15 +16,18 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.tsfile.read.expression;
+package org.apache.iotdb.cluster.query;
/**
- * @author Jinrui Zhang
+ * Type of path
*/
-public interface IBinaryExpression extends IExpression {
-
- IExpression getLeft();
-
- IExpression getRight();
-
+public enum PathType {
+ /**
+ * Select paths in a query
+ */
+ SELECT_PATH,
+ /**
+ * Filter paths in a query
+ */
+ FILTER_PATH
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/IBinaryExpression.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/QueryType.java
similarity index 77%
copy from tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/IBinaryExpression.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/query/QueryType.java
index b97310a..5bf8c53 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/IBinaryExpression.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/QueryType.java
@@ -16,15 +16,22 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.tsfile.read.expression;
+package org.apache.iotdb.cluster.query;
/**
- * @author Jinrui Zhang
+ * Type of query
*/
-public interface IBinaryExpression extends IExpression {
-
- IExpression getLeft();
-
- IExpression getRight();
-
+public enum QueryType {
+ /**
+ * Query with no filter
+ */
+ NO_FILTER,
+ /**
+ * Query with global time
+ */
+ GLOBAL_TIME,
+ /**
+ * Query with value filter
+ */
+ FILTER
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/dataset/ClusterDataSetWithTimeGenerator.java
similarity index 55%
copy from iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/query/dataset/ClusterDataSetWithTimeGenerator.java
index 6e76e66..f3e4eaf 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/dataset/ClusterDataSetWithTimeGenerator.java
@@ -16,25 +16,44 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.db.query.dataset;
+package org.apache.iotdb.cluster.query.dataset;
import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
import java.util.List;
+import org.apache.iotdb.cluster.config.ClusterConfig;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.exception.RaftConnectionException;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcSingleQueryManager;
+import org.apache.iotdb.cluster.query.timegenerator.ClusterTimeGenerator;
import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
-import org.apache.iotdb.db.query.timegenerator.EngineTimeGenerator;
-import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.common.Field;
import org.apache.iotdb.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.read.common.RowRecord;
import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
-import org.apache.iotdb.tsfile.utils.Binary;
-public class EngineDataSetWithTimeGenerator extends QueryDataSet {
+/**
+ * Dataset with time generator for cluster
+ */
+public class ClusterDataSetWithTimeGenerator extends QueryDataSet {
+
+ private ClusterRpcSingleQueryManager queryManager;
+
+ private ClusterTimeGenerator timeGenerator;
+
+ private EngineReaderByTimeStamp[] readers;
+
+ private static final ClusterConfig CLUSTER_CONF = ClusterDescriptor.getInstance().getConfig();
+
+ /**
+ * Cached batch timestamp
+ */
+ private Iterator<Long> cachedBatchTimestamp;
- private EngineTimeGenerator timeGenerator;
- private List<EngineReaderByTimeStamp> readers;
private boolean hasCachedRowRecord;
+
private RowRecord cachedRowRecord;
/**
@@ -45,11 +64,13 @@ public class EngineDataSetWithTimeGenerator extends QueryDataSet {
* @param timeGenerator EngineTimeGenerator object
* @param readers readers in List(EngineReaderByTimeStamp) structure
*/
- public EngineDataSetWithTimeGenerator(List<Path> paths, List<TSDataType> dataTypes,
- EngineTimeGenerator timeGenerator, List<EngineReaderByTimeStamp> readers) {
+ public ClusterDataSetWithTimeGenerator(List<Path> paths, List<TSDataType> dataTypes,
+ ClusterTimeGenerator timeGenerator, EngineReaderByTimeStamp[] readers,
+ ClusterRpcSingleQueryManager queryManager) {
super(paths, dataTypes);
this.timeGenerator = timeGenerator;
this.readers = readers;
+ this.queryManager = queryManager;
}
@Override
@@ -75,12 +96,12 @@ public class EngineDataSetWithTimeGenerator extends QueryDataSet {
* @return if there has next row record.
*/
private boolean cacheRowRecord() throws IOException {
- while (timeGenerator.hasNext()) {
+ while (hasNextTimestamp()) {
boolean hasField = false;
- long timestamp = timeGenerator.next();
+ long timestamp = cachedBatchTimestamp.next();
RowRecord rowRecord = new RowRecord(timestamp);
- for (int i = 0; i < readers.size(); i++) {
- EngineReaderByTimeStamp reader = readers.get(i);
+ for (int i = 0; i < readers.length; i++) {
+ EngineReaderByTimeStamp reader = readers[i];
Object value = reader.getValueInTimestamp(timestamp);
if (value == null) {
rowRecord.addField(new Field(null));
@@ -98,36 +119,31 @@ public class EngineDataSetWithTimeGenerator extends QueryDataSet {
return hasCachedRowRecord;
}
- private Field getField(Object value, TSDataType dataType) {
- Field field = new Field(dataType);
-
- if (value == null) {
- field.setNull();
- return field;
+ /**
+ * Check if it has next valid timestamp
+ */
+ private boolean hasNextTimestamp() throws IOException {
+ if (cachedBatchTimestamp == null || !cachedBatchTimestamp.hasNext()) {
+ List<Long> batchTimestamp = new ArrayList<>();
+ for (int i = 0; i < CLUSTER_CONF.getBatchReadSize(); i++) {
+ if (timeGenerator.hasNext()) {
+ batchTimestamp.add(timeGenerator.next());
+ } else {
+ break;
+ }
+ }
+ if (!batchTimestamp.isEmpty()) {
+ cachedBatchTimestamp = batchTimestamp.iterator();
+ try {
+ queryManager.fetchBatchDataByTimestampForAllSelectPaths(batchTimestamp);
+ } catch (RaftConnectionException e) {
+ throw new IOException(e);
+ }
+ }
}
-
- switch (dataType) {
- case DOUBLE:
- field.setDoubleV((double) value);
- break;
- case FLOAT:
- field.setFloatV((float) value);
- break;
- case INT64:
- field.setLongV((long) value);
- break;
- case INT32:
- field.setIntV((int) value);
- break;
- case BOOLEAN:
- field.setBoolV((boolean) value);
- break;
- case TEXT:
- field.setBinaryV((Binary) value);
- break;
- default:
- throw new UnSupportedDataTypeException("UnSupported: " + dataType);
+ if (cachedBatchTimestamp != null && cachedBatchTimestamp.hasNext()) {
+ return true;
}
- return field;
+ return false;
}
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/executor/ClusterExecutorWithTimeGenerator.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/executor/ClusterExecutorWithTimeGenerator.java
new file mode 100644
index 0000000..fed8c0d
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/executor/ClusterExecutorWithTimeGenerator.java
@@ -0,0 +1,130 @@
+/**
+ * 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.cluster.query.executor;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iotdb.cluster.query.dataset.ClusterDataSetWithTimeGenerator;
+import org.apache.iotdb.cluster.query.factory.ClusterSeriesReaderFactory;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcSingleQueryManager;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.FilterGroupEntity;
+import org.apache.iotdb.cluster.query.reader.coordinatornode.ClusterSelectSeriesReader;
+import org.apache.iotdb.cluster.query.timegenerator.ClusterTimeGenerator;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.expression.QueryExpression;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+
+public class ClusterExecutorWithTimeGenerator {
+
+ /**
+ * query expression
+ */
+ private QueryExpression queryExpression;
+
+ /**
+ * Manger for all remote query series reader resource in the query
+ */
+ private ClusterRpcSingleQueryManager queryManager;
+
+ /**
+ * Constructor of ClusterExecutorWithTimeGenerator
+ */
+ public ClusterExecutorWithTimeGenerator(QueryExpression queryExpression,
+ ClusterRpcSingleQueryManager queryManager) {
+ this.queryExpression = queryExpression;
+ this.queryManager = queryManager;
+ }
+
+ /**
+ * Execute query with value filter.
+ *
+ * @return QueryDataSet object
+ */
+ public QueryDataSet execute(QueryContext context) throws FileNodeManagerException {
+
+ /** add query token for query series which can handle locally **/
+ List<Path> localQuerySeries = new ArrayList<>(queryExpression.getSelectedSeries());
+ Set<Path> remoteQuerySeries = queryManager.getSelectSeriesReaders().keySet();
+ localQuerySeries.removeAll(remoteQuerySeries);
+ QueryResourceManager.getInstance()
+ .beginQueryOfGivenQueryPaths(context.getJobId(), localQuerySeries);
+
+ /** add query token for filter series which can handle locally **/
+ Set<String> deviceIdSet = new HashSet<>();
+ for (FilterGroupEntity filterGroupEntity : queryManager.getFilterGroupEntityMap().values()) {
+ List<Path> remoteFilterSeries = filterGroupEntity.getFilterPaths();
+ remoteFilterSeries.forEach(seriesPath -> deviceIdSet.add(seriesPath.getDevice()));
+ }
+ QueryResourceManager.getInstance()
+ .beginQueryOfGivenExpression(context.getJobId(), queryExpression.getExpression(),
+ deviceIdSet);
+
+ ClusterTimeGenerator timestampGenerator;
+ List<EngineReaderByTimeStamp> readersOfSelectedSeries;
+ try {
+ timestampGenerator = new ClusterTimeGenerator(queryExpression.getExpression(), context,
+ queryManager);
+ readersOfSelectedSeries = ClusterSeriesReaderFactory
+ .createReadersByTimestampOfSelectedPaths(queryExpression.getSelectedSeries(), context,
+ queryManager);
+ } catch (IOException ex) {
+ throw new FileNodeManagerException(ex);
+ }
+
+ /** Get data type of select paths **/
+ List<TSDataType> dataTypes = new ArrayList<>();
+ Map<Path, ClusterSelectSeriesReader> selectSeriesReaders = queryManager
+ .getSelectSeriesReaders();
+ for (Path path : queryExpression.getSelectedSeries()) {
+ try {
+ if (selectSeriesReaders.containsKey(path)) {
+ dataTypes.add(selectSeriesReaders.get(path).getDataType());
+ } else {
+ dataTypes.add(MManager.getInstance().getSeriesType(path.getFullPath()));
+ }
+ } catch (PathErrorException e) {
+ throw new FileNodeManagerException(e);
+ }
+
+ }
+
+ EngineReaderByTimeStamp[] readersOfSelectedSeriesArray = new EngineReaderByTimeStamp[readersOfSelectedSeries
+ .size()];
+ int index = 0;
+ for (EngineReaderByTimeStamp reader : readersOfSelectedSeries) {
+ readersOfSelectedSeriesArray[index] = reader;
+ index++;
+ }
+
+ return new ClusterDataSetWithTimeGenerator(queryExpression.getSelectedSeries(), dataTypes,
+ timestampGenerator,
+ readersOfSelectedSeriesArray, queryManager);
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/executor/ClusterExecutorWithoutTimeGenerator.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/executor/ClusterExecutorWithoutTimeGenerator.java
new file mode 100644
index 0000000..65bd87b
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/executor/ClusterExecutorWithoutTimeGenerator.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.cluster.query.executor;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcSingleQueryManager;
+import org.apache.iotdb.cluster.query.reader.coordinatornode.ClusterSelectSeriesReader;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.dataset.EngineDataSetWithoutTimeGenerator;
+import org.apache.iotdb.db.query.executor.ExecutorWithoutTimeGenerator;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.expression.QueryExpression;
+import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+
+public class ClusterExecutorWithoutTimeGenerator extends ExecutorWithoutTimeGenerator {
+
+ /**
+ * Query expression
+ */
+ private QueryExpression queryExpression;
+
+ /**
+ * Manger for all remote query series reader resource in the query
+ */
+ private ClusterRpcSingleQueryManager queryManager;
+
+ /**
+ * Constructor of ClusterExecutorWithoutTimeGenerator
+ */
+ public ClusterExecutorWithoutTimeGenerator(QueryExpression queryExpression,
+ ClusterRpcSingleQueryManager queryManager) {
+ this.queryExpression = queryExpression;
+ this.queryManager = queryManager;
+ }
+
+ /**
+ * Execute query without filter or with only global time filter.
+ */
+ public QueryDataSet execute(QueryContext context)
+ throws FileNodeManagerException {
+
+ Filter timeFilter = null;
+ if (queryExpression.getExpression() != null) {
+ timeFilter = ((GlobalTimeExpression) queryExpression.getExpression()).getFilter();
+ }
+
+ List<IPointReader> readersOfSelectedSeries = new ArrayList<>();
+ List<TSDataType> dataTypes = new ArrayList<>();
+
+ Map<Path, ClusterSelectSeriesReader> selectPathReaders = queryManager.getSelectSeriesReaders();
+ List<Path> paths = new ArrayList<>();
+ for (Path path : queryExpression.getSelectedSeries()) {
+
+ if (selectPathReaders.containsKey(path)) {
+ ClusterSelectSeriesReader reader = selectPathReaders.get(path);
+ readersOfSelectedSeries.add(reader);
+ dataTypes.add(reader.getDataType());
+
+ } else {
+ IPointReader reader = createSeriesReader(context, path, dataTypes, timeFilter);
+ readersOfSelectedSeries.add(reader);
+ paths.add(path);
+ }
+ }
+
+ QueryResourceManager.getInstance()
+ .beginQueryOfGivenQueryPaths(context.getJobId(), paths);
+
+ try {
+ return new EngineDataSetWithoutTimeGenerator(queryExpression.getSelectedSeries(), dataTypes,
+ readersOfSelectedSeries);
+ } catch (IOException e) {
+ throw new FileNodeManagerException(e);
+ }
+ }
+
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/executor/ClusterQueryRouter.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/executor/ClusterQueryRouter.java
new file mode 100644
index 0000000..4211528
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/executor/ClusterQueryRouter.java
@@ -0,0 +1,121 @@
+/**
+ * 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.cluster.query.executor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iotdb.cluster.exception.RaftConnectionException;
+import org.apache.iotdb.cluster.query.QueryType;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcQueryManager;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcSingleQueryManager;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.executor.IEngineQueryRouter;
+import org.apache.iotdb.db.query.fill.IFill;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.expression.ExpressionType;
+import org.apache.iotdb.tsfile.read.expression.IExpression;
+import org.apache.iotdb.tsfile.read.expression.QueryExpression;
+import org.apache.iotdb.tsfile.read.expression.util.ExpressionOptimizer;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+/**
+ * Query entrance class of cluster query process. All query clause will be transformed to physical
+ * plan, physical plan will be executed by ClusterQueryRouter.
+ */
+public class ClusterQueryRouter implements IEngineQueryRouter {
+
+ /**
+ * Consistency level of reading data
+ */
+ private ThreadLocal<Integer> readDataConsistencyLevel = new ThreadLocal<>();
+
+ @Override
+ public QueryDataSet query(QueryExpression queryExpression, QueryContext context)
+ throws FileNodeManagerException, PathErrorException {
+
+ ClusterRpcSingleQueryManager queryManager = ClusterRpcQueryManager.getInstance()
+ .getSingleQuery(context.getJobId());
+ try {
+ if (queryExpression.hasQueryFilter()) {
+
+ IExpression optimizedExpression = ExpressionOptimizer.getInstance()
+ .optimize(queryExpression.getExpression(), queryExpression.getSelectedSeries());
+ queryExpression.setExpression(optimizedExpression);
+ // update query expression of origin query plan, it's necessary.
+ queryManager.getOriginQueryPlan().setExpression(optimizedExpression);
+
+ if (optimizedExpression.getType() == ExpressionType.GLOBAL_TIME) {
+ queryManager.initQueryResource(QueryType.GLOBAL_TIME, getReadDataConsistencyLevel());
+ ClusterExecutorWithoutTimeGenerator engineExecutor =
+ new ClusterExecutorWithoutTimeGenerator(queryExpression, queryManager);
+ return engineExecutor.execute(context);
+ } else {
+ queryManager.initQueryResource(QueryType.FILTER, getReadDataConsistencyLevel());
+ ClusterExecutorWithTimeGenerator engineExecutor = new ClusterExecutorWithTimeGenerator(
+ queryExpression, queryManager);
+ return engineExecutor.execute(context);
+ }
+
+ } else {
+ queryManager.initQueryResource(QueryType.NO_FILTER, getReadDataConsistencyLevel());
+ ClusterExecutorWithoutTimeGenerator engineExecutor =
+ new ClusterExecutorWithoutTimeGenerator(queryExpression, queryManager);
+ return engineExecutor.execute(context);
+ }
+ } catch (QueryFilterOptimizationException | IOException | RaftConnectionException e) {
+ throw new FileNodeManagerException(e);
+ }
+ }
+
+ @Override
+ public QueryDataSet aggregate(List<Path> selectedSeries, List<String> aggres,
+ IExpression expression, QueryContext context)
+ throws QueryFilterOptimizationException, FileNodeManagerException, IOException, PathErrorException, ProcessorException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public QueryDataSet groupBy(List<Path> selectedSeries, List<String> aggres,
+ IExpression expression, long unit, long origin, List<Pair<Long, Long>> intervals,
+ QueryContext context)
+ throws ProcessorException, QueryFilterOptimizationException, FileNodeManagerException, PathErrorException, IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public QueryDataSet fill(List<Path> fillPaths, long queryTime, Map<TSDataType, IFill> fillType,
+ QueryContext context) throws FileNodeManagerException, PathErrorException, IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ public int getReadDataConsistencyLevel() {
+ return readDataConsistencyLevel.get();
+ }
+
+ public void setReadDataConsistencyLevel(int readDataConsistencyLevel) {
+ this.readDataConsistencyLevel.set(readDataConsistencyLevel);
+ }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/GlobalTimeExpression.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/expression/TrueExpression.java
similarity index 63%
copy from tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/GlobalTimeExpression.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/query/expression/TrueExpression.java
index d69a65a..d62c885 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/GlobalTimeExpression.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/expression/TrueExpression.java
@@ -16,37 +16,28 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.tsfile.read.expression.impl;
+package org.apache.iotdb.cluster.query.expression;
import org.apache.iotdb.tsfile.read.expression.ExpressionType;
-import org.apache.iotdb.tsfile.read.expression.IUnaryExpression;
-import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.read.expression.IExpression;
-public class GlobalTimeExpression implements IUnaryExpression {
-
- private Filter filter;
-
- public GlobalTimeExpression(Filter filter) {
- this.filter = filter;
- }
-
- @Override
- public Filter getFilter() {
- return filter;
- }
+/**
+ * This type of Expression is used in pruning filter tree while handling query with value filter
+ */
+public class TrueExpression implements IExpression {
@Override
- public void setFilter(Filter filter) {
- this.filter = filter;
+ public ExpressionType getType() {
+ return ExpressionType.TRUE;
}
@Override
- public ExpressionType getType() {
- return ExpressionType.GLOBAL_TIME;
+ public IExpression clone() {
+ return new TrueExpression();
}
@Override
public String toString() {
- return "[" + this.filter.toString() + "]";
+ return "TrueExpression{}";
}
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/factory/ClusterSeriesReaderFactory.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/factory/ClusterSeriesReaderFactory.java
new file mode 100644
index 0000000..ddfa5eb
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/factory/ClusterSeriesReaderFactory.java
@@ -0,0 +1,94 @@
+/**
+ * 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.cluster.query.factory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcSingleQueryManager;
+import org.apache.iotdb.cluster.query.reader.coordinatornode.ClusterSelectSeriesReader;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.factory.SeriesReaderFactory;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.query.reader.merge.PriorityMergeReaderByTimestamp;
+import org.apache.iotdb.db.query.reader.sequence.SequenceDataReaderByTimestamp;
+import org.apache.iotdb.tsfile.read.common.Path;
+
+/**
+ * Reader factory for cluster
+ */
+public class ClusterSeriesReaderFactory {
+
+ /**
+ * Construct ReaderByTimestamp , include sequential data and unsequential data.
+ *
+ * @param paths selected series path
+ * @param context query context
+ * @return the list of EngineReaderByTimeStamp
+ */
+ public static List<EngineReaderByTimeStamp> createReadersByTimestampOfSelectedPaths(
+ List<Path> paths, QueryContext context, ClusterRpcSingleQueryManager queryManager)
+ throws IOException, FileNodeManagerException {
+
+ Map<Path, ClusterSelectSeriesReader> selectSeriesReaders = queryManager.getSelectSeriesReaders();
+ List<EngineReaderByTimeStamp> readersOfSelectedSeries = new ArrayList<>();
+
+ for (Path path : paths) {
+
+ if (selectSeriesReaders.containsKey(path)) {
+ readersOfSelectedSeries.add(selectSeriesReaders.get(path));
+ } else {
+ /** can handle series query locally **/
+ EngineReaderByTimeStamp readerByTimeStamp = createReaderByTimeStamp(path, context);
+ readersOfSelectedSeries.add(readerByTimeStamp);
+ }
+ }
+ return readersOfSelectedSeries;
+ }
+
+ /**
+ * Create single ReaderByTimestamp
+ *
+ * @param path series path
+ * @param context query context
+ */
+ public static EngineReaderByTimeStamp createReaderByTimeStamp(Path path, QueryContext context)
+ throws IOException, FileNodeManagerException {
+ QueryDataSource queryDataSource = QueryResourceManager.getInstance()
+ .getQueryDataSource(path,
+ context);
+
+ PriorityMergeReaderByTimestamp mergeReaderByTimestamp = new PriorityMergeReaderByTimestamp();
+
+ // reader for sequence data
+ SequenceDataReaderByTimestamp tsFilesReader = new SequenceDataReaderByTimestamp(
+ queryDataSource.getSeqDataSource(), context);
+ mergeReaderByTimestamp.addReaderWithPriority(tsFilesReader, 1);
+
+ // reader for unSequence data
+ PriorityMergeReaderByTimestamp unSeqMergeReader = SeriesReaderFactory.getInstance()
+ .createUnSeqMergeReaderByTimestamp(queryDataSource.getOverflowSeriesDataSource());
+ mergeReaderByTimestamp.addReaderWithPriority(unSeqMergeReader, 2);
+ return mergeReaderByTimestamp;
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/coordinatornode/ClusterRpcQueryManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/coordinatornode/ClusterRpcQueryManager.java
new file mode 100644
index 0000000..faece2b
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/coordinatornode/ClusterRpcQueryManager.java
@@ -0,0 +1,111 @@
+/**
+ * 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.cluster.query.manager.coordinatornode;
+
+import com.alipay.sofa.jraft.util.OnlyForTest;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.iotdb.cluster.config.ClusterConfig;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.exception.RaftConnectionException;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+
+public class ClusterRpcQueryManager implements IClusterRpcQueryManager {
+
+ /**
+ * Key is job id, value is task id.
+ */
+ private static final ConcurrentHashMap<Long, String> JOB_ID_MAP_TASK_ID = new ConcurrentHashMap<>();
+
+ /**
+ * Key is task id, value is manager of a client query.
+ */
+ private static final ConcurrentHashMap<String, ClusterRpcSingleQueryManager> SINGLE_QUERY_MANAGER_MAP = new ConcurrentHashMap<>();
+
+ private static final ClusterConfig CLUSTER_CONFIG = ClusterDescriptor.getInstance().getConfig();
+
+ /**
+ * Local address
+ */
+ private static final String LOCAL_ADDR = String
+ .format("%s:%d", CLUSTER_CONFIG.getIp(), CLUSTER_CONFIG.getPort());
+
+ @Override
+ public void addSingleQuery(long jobId, QueryPlan physicalPlan) {
+ String taskId = createTaskId(jobId);
+ JOB_ID_MAP_TASK_ID.put(jobId, taskId);
+ SINGLE_QUERY_MANAGER_MAP.put(taskId, new ClusterRpcSingleQueryManager(taskId, physicalPlan));
+ }
+
+ @Override
+ public String createTaskId(long jobId) {
+ return String.format("%s:%d", LOCAL_ADDR, jobId);
+ }
+
+ @Override
+ public ClusterRpcSingleQueryManager getSingleQuery(long jobId) {
+ return SINGLE_QUERY_MANAGER_MAP.get(JOB_ID_MAP_TASK_ID.get(jobId));
+ }
+
+ @Override
+ public ClusterRpcSingleQueryManager getSingleQuery(String taskId) {
+ return SINGLE_QUERY_MANAGER_MAP.get(taskId);
+ }
+
+ @Override
+ public void releaseQueryResource(long jobId) throws RaftConnectionException {
+ if (JOB_ID_MAP_TASK_ID.containsKey(jobId)) {
+ SINGLE_QUERY_MANAGER_MAP.remove(JOB_ID_MAP_TASK_ID.remove(jobId)).releaseQueryResource();
+ }
+ }
+
+ @Override
+ public Map<String, Integer> getAllReadUsage() {
+ Map<String, Integer> readerUsageMap = new HashMap<>();
+ SINGLE_QUERY_MANAGER_MAP.values().forEach(singleQueryManager -> {
+ for(String groupId:singleQueryManager.getDataGroupUsage()) {
+ readerUsageMap.put(groupId, readerUsageMap.getOrDefault(groupId, 0) + 1);
+ }
+ });
+ return readerUsageMap;
+ }
+
+ @OnlyForTest
+ public static ConcurrentHashMap<Long, String> getJobIdMapTaskId() {
+ return JOB_ID_MAP_TASK_ID;
+ }
+
+ private ClusterRpcQueryManager() {
+ }
+
+ public static final ClusterRpcQueryManager getInstance() {
+ return ClusterRpcQueryManagerHolder.INSTANCE;
+ }
+
+ private static class ClusterRpcQueryManagerHolder {
+
+ private static final ClusterRpcQueryManager INSTANCE = new ClusterRpcQueryManager();
+
+ private ClusterRpcQueryManagerHolder() {
+
+ }
+ }
+
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/coordinatornode/ClusterRpcSingleQueryManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/coordinatornode/ClusterRpcSingleQueryManager.java
new file mode 100644
index 0000000..d9a5859
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/coordinatornode/ClusterRpcSingleQueryManager.java
@@ -0,0 +1,415 @@
+/**
+ * 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.cluster.query.manager.coordinatornode;
+
+import com.alipay.sofa.jraft.entity.PeerId;
+import java.io.IOException;
+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.Map.Entry;
+import java.util.Set;
+import org.apache.iotdb.cluster.config.ClusterConfig;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.exception.RaftConnectionException;
+import org.apache.iotdb.cluster.query.PathType;
+import org.apache.iotdb.cluster.query.QueryType;
+import org.apache.iotdb.cluster.query.reader.coordinatornode.ClusterFilterSeriesReader;
+import org.apache.iotdb.cluster.query.reader.coordinatornode.ClusterSelectSeriesReader;
+import org.apache.iotdb.cluster.query.utils.ClusterRpcReaderUtils;
+import org.apache.iotdb.cluster.query.utils.QueryPlanPartitionUtils;
+import org.apache.iotdb.cluster.rpc.raft.response.BasicQueryDataResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.InitSeriesReaderResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.QuerySeriesDataByTimestampResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.QuerySeriesDataResponse;
+import org.apache.iotdb.cluster.utils.QPExecutorUtils;
+import org.apache.iotdb.cluster.utils.RaftUtils;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+
+/**
+ * Manage all remote series reader resource in a query resource in coordinator node.
+ */
+public class ClusterRpcSingleQueryManager implements IClusterRpcSingleQueryManager {
+
+ /**
+ * Statistic all usage of local data group.
+ */
+ private Set<String> dataGroupUsage = new HashSet<>();
+
+ /**
+ * Query job id assigned by ClusterRpcQueryManager
+ */
+ private String taskId;
+
+ /**
+ * Represents the number of query rounds
+ */
+ private long queryRounds = 0;
+
+ /**
+ * Origin query plan parsed by QueryProcessor
+ */
+ private QueryPlan originQueryPlan;
+
+ /**
+ * Represent selected reader nodes, key is group id and value is selected peer id
+ */
+ private Map<String, PeerId> queryNodes = new HashMap<>();
+
+ // select path resource
+ /**
+ * Query plans of select paths which are divided from queryPlan group by group id, it contains all
+ * group id ,including local data group if it involves.
+ */
+ private Map<String, QueryPlan> selectPathPlans = new HashMap<>();
+
+ /**
+ * Key is group id (only contains remote group id), value is all select series in group id.
+ */
+ private Map<String, List<Path>> selectSeriesByGroupId = new HashMap<>();
+
+ /**
+ * Series reader of select paths (only contains remote series), key is series path , value is
+ * reader
+ */
+ private Map<Path, ClusterSelectSeriesReader> selectSeriesReaders = new HashMap<>();
+
+ // filter path resource
+ /**
+ * Filter group entity group by data group, key is group id(only contain remote group id)
+ */
+ private Map<String, FilterGroupEntity> filterGroupEntityMap = new HashMap<>();
+
+ private static final ClusterConfig CLUSTER_CONF = ClusterDescriptor.getInstance().getConfig();
+
+ public ClusterRpcSingleQueryManager(String taskId,
+ QueryPlan queryPlan) {
+ this.taskId = taskId;
+ this.originQueryPlan = queryPlan;
+ }
+
+ @Override
+ public void initQueryResource(QueryType queryType, int readDataConsistencyLevel)
+ throws PathErrorException, IOException, RaftConnectionException {
+ switch (queryType) {
+ case NO_FILTER:
+ case GLOBAL_TIME:
+ QueryPlanPartitionUtils.splitQueryPlanWithoutValueFilter(this);
+ break;
+ case FILTER:
+ QueryPlanPartitionUtils.splitQueryPlanWithValueFilter(this);
+ break;
+ default:
+ throw new UnsupportedOperationException();
+ }
+ initSeriesReader(readDataConsistencyLevel);
+ }
+
+ /**
+ * Init series reader, complete all initialization with all remote query node of a specific data
+ * group
+ */
+ private void initSeriesReader(int readDataConsistencyLevel)
+ throws IOException, RaftConnectionException {
+ // Init all series with data group of select series,if filter series has the same data group, init them together.
+ for (Entry<String, QueryPlan> entry : selectPathPlans.entrySet()) {
+ String groupId = entry.getKey();
+ QueryPlan queryPlan = entry.getValue();
+ if (!QPExecutorUtils.canHandleQueryByGroupId(groupId)) {
+ PeerId randomPeer = RaftUtils.getRandomPeerID(groupId);
+ queryNodes.put(groupId, randomPeer);
+ Map<PathType, QueryPlan> allQueryPlan = new EnumMap<>(PathType.class);
+ allQueryPlan.put(PathType.SELECT_PATH, queryPlan);
+ List<Filter> filterList = null;
+ if (filterGroupEntityMap.containsKey(groupId)) {
+ FilterGroupEntity filterGroupEntity = filterGroupEntityMap.get(groupId);
+ allQueryPlan.put(PathType.FILTER_PATH, filterGroupEntity.getQueryPlan());
+ filterList = filterGroupEntity.getFilters();
+ }
+ InitSeriesReaderResponse response = (InitSeriesReaderResponse) ClusterRpcReaderUtils
+ .createClusterSeriesReader(groupId, randomPeer, readDataConsistencyLevel,
+ allQueryPlan, taskId, filterList);
+ handleInitReaderResponse(groupId, allQueryPlan, response);
+ } else {
+ dataGroupUsage.add(groupId);
+ selectSeriesByGroupId.remove(groupId);
+ if (filterGroupEntityMap.containsKey(groupId)) {
+ filterGroupEntityMap.remove(groupId);
+ }
+ }
+ }
+
+ //Init series reader with data groups of filter series, which don't exist in data groups list of select series.
+ for (Entry<String, FilterGroupEntity> entry : filterGroupEntityMap.entrySet()) {
+ String groupId = entry.getKey();
+ if (!selectPathPlans.containsKey(groupId)) {
+ PeerId randomPeer = RaftUtils.getRandomPeerID(groupId);
+ Map<PathType, QueryPlan> allQueryPlan = new EnumMap<>(PathType.class);
+ FilterGroupEntity filterGroupEntity = filterGroupEntityMap.get(groupId);
+ allQueryPlan.put(PathType.FILTER_PATH, filterGroupEntity.getQueryPlan());
+ List<Filter> filterList = filterGroupEntity.getFilters();
+ InitSeriesReaderResponse response = (InitSeriesReaderResponse) ClusterRpcReaderUtils
+ .createClusterSeriesReader(groupId, randomPeer, readDataConsistencyLevel,
+ allQueryPlan, taskId, filterList);
+ handleInitReaderResponse(groupId, allQueryPlan, response);
+ }
+ }
+ }
+
+ /**
+ * Handle response of initialization with remote query node
+ */
+ private void handleInitReaderResponse(String groupId, Map<PathType, QueryPlan> allQueryPlan,
+ InitSeriesReaderResponse response) {
+ /** create cluster series reader **/
+ if (allQueryPlan.containsKey(PathType.SELECT_PATH)) {
+ QueryPlan plan = allQueryPlan.get(PathType.SELECT_PATH);
+ List<Path> paths = plan.getPaths();
+ List<TSDataType> seriesType = response.getSeriesDataTypes().get(PathType.SELECT_PATH);
+ for (int i = 0; i < paths.size(); i++) {
+ Path seriesPath = paths.get(i);
+ TSDataType dataType = seriesType.get(i);
+ ClusterSelectSeriesReader seriesReader = new ClusterSelectSeriesReader(groupId, seriesPath,
+ dataType, this);
+ selectSeriesReaders.put(seriesPath, seriesReader);
+ }
+ }
+ if (allQueryPlan.containsKey(PathType.FILTER_PATH)) {
+ QueryPlan plan = allQueryPlan.get(PathType.FILTER_PATH);
+ List<Path> paths = plan.getPaths();
+ List<TSDataType> seriesType = response.getSeriesDataTypes().get(PathType.FILTER_PATH);
+ for (int i = 0; i < paths.size(); i++) {
+ Path seriesPath = paths.get(i);
+ TSDataType dataType = seriesType.get(i);
+ ClusterFilterSeriesReader seriesReader = new ClusterFilterSeriesReader(groupId, seriesPath,
+ dataType, this);
+ if (!filterGroupEntityMap.containsKey(groupId)) {
+ filterGroupEntityMap.put(groupId, new FilterGroupEntity(groupId));
+ }
+ filterGroupEntityMap.get(groupId).addFilterSeriesReader(seriesReader);
+ }
+ }
+ }
+
+ @Override
+ public void fetchBatchDataForSelectPaths(String groupId) throws RaftConnectionException {
+ List<String> fetchDataSeries = new ArrayList<>();
+ Map<String, List<Path>> seriesByGroupId;
+ Map<Path, ClusterSelectSeriesReader> seriesReaders;
+ seriesByGroupId = selectSeriesByGroupId;
+ seriesReaders = selectSeriesReaders;
+ if (seriesByGroupId.containsKey(groupId)) {
+ List<Path> allFilterSeries = seriesByGroupId.get(groupId);
+ for (Path series : allFilterSeries) {
+ if (seriesReaders.get(series).enableFetchData()) {
+ fetchDataSeries.add(series.getFullPath());
+ }
+ }
+ }
+ QuerySeriesDataResponse response = ClusterRpcReaderUtils
+ .fetchBatchData(groupId, queryNodes.get(groupId), taskId, PathType.SELECT_PATH,
+ fetchDataSeries,
+ queryRounds++);
+ handleFetchDataResponseForSelectPaths(fetchDataSeries, response);
+ }
+
+ @Override
+ public void fetchBatchDataForFilterPaths(String groupId) throws RaftConnectionException {
+ QuerySeriesDataResponse response = ClusterRpcReaderUtils
+ .fetchBatchData(groupId, queryNodes.get(groupId), taskId, PathType.FILTER_PATH, null,
+ queryRounds++);
+ handleFetchDataResponseForFilterPaths(groupId, response);
+ }
+
+
+ @Override
+ public void fetchBatchDataByTimestampForAllSelectPaths(List<Long> batchTimestamp)
+ throws RaftConnectionException {
+ for (Entry<String, List<Path>> entry : selectSeriesByGroupId.entrySet()) {
+ String groupId = entry.getKey();
+ List<String> fetchDataFilterSeries = new ArrayList<>();
+ entry.getValue().forEach(path -> fetchDataFilterSeries.add(path.getFullPath()));
+ QuerySeriesDataByTimestampResponse response = ClusterRpcReaderUtils
+ .fetchBatchDataByTimestamp(groupId, queryNodes.get(groupId), taskId, queryRounds++,
+ batchTimestamp, fetchDataFilterSeries);
+ handleFetchDataByTimestampResponseForSelectPaths(fetchDataFilterSeries, response);
+ }
+ }
+
+ /**
+ * Handle response of fetching data, and add batch data to corresponding reader.
+ */
+ private void handleFetchDataByTimestampResponseForSelectPaths(List<String> fetchDataSeries,
+ BasicQueryDataResponse response) {
+ List<BatchData> batchDataList = response.getSeriesBatchData();
+ for (int i = 0; i < fetchDataSeries.size(); i++) {
+ String series = fetchDataSeries.get(i);
+ BatchData batchData = batchDataList.get(i);
+ selectSeriesReaders.get(new Path(series))
+ .addBatchData(batchData, true);
+ }
+ }
+
+ /**
+ * Handle response of fetching data, and add batch data to corresponding reader.
+ */
+ private void handleFetchDataResponseForSelectPaths(List<String> fetchDataSeries,
+ BasicQueryDataResponse response) {
+ List<BatchData> batchDataList = response.getSeriesBatchData();
+ for (int i = 0; i < fetchDataSeries.size(); i++) {
+ String series = fetchDataSeries.get(i);
+ BatchData batchData = batchDataList.get(i);
+ selectSeriesReaders.get(new Path(series))
+ .addBatchData(batchData, batchData.length() < CLUSTER_CONF.getBatchReadSize());
+ }
+ }
+
+ /**
+ * Handle response of fetching data, and add batch data to corresponding reader.
+ */
+ private void handleFetchDataResponseForFilterPaths(String groupId,
+ QuerySeriesDataResponse response) {
+ FilterGroupEntity filterGroupEntity = filterGroupEntityMap.get(groupId);
+ List<Path> fetchDataSeries = filterGroupEntity.getFilterPaths();
+ List<BatchData> batchDataList = response.getSeriesBatchData();
+ List<ClusterFilterSeriesReader> filterReaders = filterGroupEntity.getFilterSeriesReaders();
+ boolean remoteDataFinish = true;
+ for (int i = 0; i < batchDataList.size(); i++) {
+ if (batchDataList.get(i).length() != 0) {
+ remoteDataFinish = false;
+ break;
+ }
+ }
+ for (int i = 0; i < fetchDataSeries.size(); i++) {
+ BatchData batchData = batchDataList.get(i);
+ if (batchData.length() != 0) {
+ filterReaders.get(i).addBatchData(batchData, remoteDataFinish);
+ }
+ }
+ }
+
+ @Override
+ public QueryPlan getSelectPathQueryPlan(String fullPath) {
+ return selectPathPlans.get(fullPath);
+ }
+
+ @Override
+ public void setDataGroupReaderNode(String groupId, PeerId readerNode) {
+ queryNodes.put(groupId, readerNode);
+ }
+
+ @Override
+ public PeerId getDataGroupReaderNode(String groupId) {
+ return queryNodes.get(groupId);
+ }
+
+ @Override
+ public void releaseQueryResource() throws RaftConnectionException {
+ for (Entry<String, PeerId> entry : queryNodes.entrySet()) {
+ String groupId = entry.getKey();
+ PeerId queryNode = entry.getValue();
+ ClusterRpcReaderUtils.releaseRemoteQueryResource(groupId, queryNode, taskId);
+ }
+ }
+
+ public Set<String> getDataGroupUsage() {
+ return dataGroupUsage;
+ }
+
+ public void addDataGroupUsage(String groupId) {
+ this.dataGroupUsage.add(groupId);
+ }
+
+ public String getTaskId() {
+ return taskId;
+ }
+
+ public void setTaskId(String taskId) {
+ this.taskId = taskId;
+ }
+
+ public long getQueryRounds() {
+ return queryRounds;
+ }
+
+ public void setQueryRounds(long queryRounds) {
+ this.queryRounds = queryRounds;
+ }
+
+ public QueryPlan getOriginQueryPlan() {
+ return originQueryPlan;
+ }
+
+ public void setOriginQueryPlan(QueryPlan queryPlan) {
+ this.originQueryPlan = queryPlan;
+ }
+
+ public Map<String, PeerId> getQueryNodes() {
+ return queryNodes;
+ }
+
+ public void setQueryNodes(
+ Map<String, PeerId> queryNodes) {
+ this.queryNodes = queryNodes;
+ }
+
+ public Map<String, QueryPlan> getSelectPathPlans() {
+ return selectPathPlans;
+ }
+
+ public void setSelectPathPlans(
+ Map<String, QueryPlan> selectPathPlans) {
+ this.selectPathPlans = selectPathPlans;
+ }
+
+ public Map<String, List<Path>> getSelectSeriesByGroupId() {
+ return selectSeriesByGroupId;
+ }
+
+ public void setSelectSeriesByGroupId(
+ Map<String, List<Path>> selectSeriesByGroupId) {
+ this.selectSeriesByGroupId = selectSeriesByGroupId;
+ }
+
+ public Map<Path, ClusterSelectSeriesReader> getSelectSeriesReaders() {
+ return selectSeriesReaders;
+ }
+
+ public void setSelectSeriesReaders(
+ Map<Path, ClusterSelectSeriesReader> selectSeriesReaders) {
+ this.selectSeriesReaders = selectSeriesReaders;
+ }
+
+ public Map<String, FilterGroupEntity> getFilterGroupEntityMap() {
+ return filterGroupEntityMap;
+ }
+
+ public void setFilterGroupEntityMap(
+ Map<String, FilterGroupEntity> filterGroupEntityMap) {
+ this.filterGroupEntityMap = filterGroupEntityMap;
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/coordinatornode/FilterGroupEntity.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/coordinatornode/FilterGroupEntity.java
new file mode 100644
index 0000000..326af11
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/coordinatornode/FilterGroupEntity.java
@@ -0,0 +1,111 @@
+/**
+ * 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.cluster.query.manager.coordinatornode;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.cluster.query.reader.coordinatornode.ClusterFilterSeriesReader;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+
+/**
+ * Filter entities of a data group, concluding QueryPlan, filters, all filter paths and filter readers
+ */
+public class FilterGroupEntity {
+
+ /**
+ * Group id
+ */
+ private String groupId;
+
+ /**
+ * Query plans of filter paths which are divided from queryPlan
+ */
+ private QueryPlan queryPlan;
+
+ /**
+ * Filters of filter path.
+ */
+ private List<Filter> filters;
+
+ /**
+ *
+ * all filter series
+ * <p>
+ * Note: It may contain multiple series in a complicated tree
+ * for example: select * from root.vehicle where d0.s0 > 10 and d0.s0 < 101 or time = 12,
+ * filter tree: <code>[[[[root.vehicle.d0.s0:time == 12] || [root.vehicle.d0.s1:time == 12]] || [root.vehicle.d1.s2:time == 12]] || [root.vehicle.d1.s3:time == 12]]</code>
+ * </p>
+ */
+ private List<Path> filterPaths;
+
+
+ /**
+ * Series reader of filter paths (only contains remote series)
+ */
+ private List<ClusterFilterSeriesReader> filterSeriesReaders;
+
+ public FilterGroupEntity(String groupId) {
+ this.groupId = groupId;
+ this.filterPaths = new ArrayList<>();
+ this.filters = new ArrayList<>();
+ this.filterSeriesReaders = new ArrayList<>();
+ }
+
+ public String getGroupId() {
+ return groupId;
+ }
+
+ public void setGroupId(String groupId) {
+ this.groupId = groupId;
+ }
+
+ public QueryPlan getQueryPlan() {
+ return queryPlan;
+ }
+
+ public void setQueryPlan(QueryPlan queryPlan) {
+ this.queryPlan = queryPlan;
+ }
+
+ public List<Filter> getFilters() {
+ return filters;
+ }
+
+ public void addFilter(Filter filter) {
+ this.filters.add(filter);
+ }
+
+ public List<Path> getFilterPaths() {
+ return filterPaths;
+ }
+
+ public void addFilterPaths(Path filterPath) {
+ this.filterPaths.add(filterPath);
+ }
+
+ public List<ClusterFilterSeriesReader> getFilterSeriesReaders() {
+ return filterSeriesReaders;
+ }
+
+ public void addFilterSeriesReader(ClusterFilterSeriesReader filterSeriesReader) {
+ this.filterSeriesReaders.add(filterSeriesReader);
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/coordinatornode/IClusterRpcQueryManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/coordinatornode/IClusterRpcQueryManager.java
new file mode 100644
index 0000000..b8e4f5d
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/coordinatornode/IClusterRpcQueryManager.java
@@ -0,0 +1,69 @@
+/**
+ * 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.cluster.query.manager.coordinatornode;
+
+import java.util.Map;
+import org.apache.iotdb.cluster.exception.RaftConnectionException;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+
+/**
+ * Manage all query series reader resources which fetch data from remote query nodes in coordinator
+ * node
+ */
+public interface IClusterRpcQueryManager {
+
+ /**
+ * Add a query
+ *
+ * @param jobId job id assigned by QueryResourceManager
+ * @param physicalPlan physical plan
+ */
+ void addSingleQuery(long jobId, QueryPlan physicalPlan);
+
+ /**
+ * Get full task id (local address + job id)
+ */
+ String createTaskId(long jobId);
+
+ /**
+ * Get query manager by jobId
+ *
+ * @param jobId job id assigned by QueryResourceManager
+ */
+ ClusterRpcSingleQueryManager getSingleQuery(long jobId);
+
+ /**
+ * Get query manager by taskId
+ *
+ * @param taskId task id assigned by getAndIncreaTaskId() method
+ */
+ ClusterRpcSingleQueryManager getSingleQuery(String taskId);
+
+ /**
+ * Release query resource
+ *
+ * @param jobId job id
+ */
+ void releaseQueryResource(long jobId) throws RaftConnectionException;
+
+ /**
+ * Get all read usage count group by data group id, key is group id, value is usage count
+ */
+ Map<String, Integer> getAllReadUsage();
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/coordinatornode/IClusterRpcSingleQueryManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/coordinatornode/IClusterRpcSingleQueryManager.java
new file mode 100644
index 0000000..c4aec9c
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/coordinatornode/IClusterRpcSingleQueryManager.java
@@ -0,0 +1,98 @@
+/**
+ * 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.cluster.query.manager.coordinatornode;
+
+import com.alipay.sofa.jraft.entity.PeerId;
+import java.io.IOException;
+import java.util.List;
+import org.apache.iotdb.cluster.exception.RaftConnectionException;
+import org.apache.iotdb.cluster.query.QueryType;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+
+/**
+ * Manage a single query.
+ */
+public interface IClusterRpcSingleQueryManager {
+
+ /**
+ * Divide physical plan into several sub physical plans according to timeseries full path and
+ * create sereis reader.
+ *
+ * @param queryType query type
+ * @param readDataConsistencyLevel consistency level of reading data
+ */
+ void initQueryResource(QueryType queryType, int readDataConsistencyLevel)
+ throws PathErrorException, IOException, RaftConnectionException;
+
+ /**
+ * <p>
+ * Fetch data for select paths. In order to reduce the number of RPC communications, fetching data
+ * from remote query node will fetch for all series in the same data group. If the cached data for
+ * specific series exceed limit, ignore this fetching data process of the series.
+ * </p>
+ *
+ * @param groupId data group id
+ */
+ void fetchBatchDataForSelectPaths(String groupId) throws RaftConnectionException;
+
+ /**
+ * Fetch data for filter path.
+ *
+ * @param groupId data group id
+ */
+ void fetchBatchDataForFilterPaths(String groupId) throws RaftConnectionException;
+
+ /**
+ * Fetch batch data for all select paths by batch timestamp. If target data can be fetched, skip
+ * corresponding group id.
+ *
+ * @param batchTimestamp valid batch timestamp
+ */
+ void fetchBatchDataByTimestampForAllSelectPaths(List<Long> batchTimestamp)
+ throws RaftConnectionException;
+
+ /**
+ * Get query plan of select path
+ *
+ * @param fullPath Timeseries full path in select paths
+ */
+ QueryPlan getSelectPathQueryPlan(String fullPath);
+
+ /**
+ * Set reader node of a data group
+ *
+ * @param groupId data group id
+ * @param readerNode reader peer id
+ */
+ void setDataGroupReaderNode(String groupId, PeerId readerNode);
+
+ /**
+ * Get reader node of a data group by group id
+ *
+ * @param groupId data group id
+ * @return peer id of reader node
+ */
+ PeerId getDataGroupReaderNode(String groupId);
+
+ /**
+ * Release query resource in remote query node
+ */
+ void releaseQueryResource() throws RaftConnectionException;
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/querynode/ClusterLocalQueryManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/querynode/ClusterLocalQueryManager.java
new file mode 100644
index 0000000..fe3ac52
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/querynode/ClusterLocalQueryManager.java
@@ -0,0 +1,125 @@
+/**
+ * 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.cluster.query.manager.querynode;
+
+import com.alipay.sofa.jraft.util.OnlyForTest;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.InitSeriesReaderRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.QuerySeriesDataByTimestampRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.QuerySeriesDataRequest;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.InitSeriesReaderResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.QuerySeriesDataByTimestampResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.QuerySeriesDataResponse;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+
+public class ClusterLocalQueryManager implements IClusterLocalQueryManager {
+
+ /**
+ * Key is task id which is assigned by coordinator node, value is job id which is assigned by
+ * query node(local).
+ */
+ private static final ConcurrentHashMap<String, Long> TASK_ID_MAP_JOB_ID = new ConcurrentHashMap<>();
+
+ /**
+ * Key is job id, value is manager of a client query.
+ */
+ private static final ConcurrentHashMap<Long, ClusterLocalSingleQueryManager> SINGLE_QUERY_MANAGER_MAP = new ConcurrentHashMap<>();
+
+ private ClusterLocalQueryManager() {
+ }
+
+ @Override
+ public InitSeriesReaderResponse createQueryDataSet(InitSeriesReaderRequest request)
+ throws IOException, FileNodeManagerException, PathErrorException, ProcessorException, QueryFilterOptimizationException {
+ long jobId = QueryResourceManager.getInstance().assignJobId();
+ String taskId = request.getTaskId();
+ TASK_ID_MAP_JOB_ID.put(taskId, jobId);
+ ClusterLocalSingleQueryManager localQueryManager = new ClusterLocalSingleQueryManager(jobId);
+ SINGLE_QUERY_MANAGER_MAP.put(jobId, localQueryManager);
+ return localQueryManager.createSeriesReader(request);
+ }
+
+ @Override
+ public QuerySeriesDataResponse readBatchData(QuerySeriesDataRequest request)
+ throws IOException {
+ long jobId = TASK_ID_MAP_JOB_ID.get(request.getTaskId());
+ return SINGLE_QUERY_MANAGER_MAP.get(jobId).readBatchData(request);
+ }
+
+ @Override
+ public QuerySeriesDataByTimestampResponse readBatchDataByTimestamp(
+ QuerySeriesDataByTimestampRequest request)
+ throws IOException {
+ long jobId = TASK_ID_MAP_JOB_ID.get(request.getTaskId());
+ return SINGLE_QUERY_MANAGER_MAP.get(jobId).readBatchDataByTimestamp(request);
+ }
+
+ @Override
+ public void close(String taskId) throws FileNodeManagerException {
+ if (TASK_ID_MAP_JOB_ID.containsKey(taskId)) {
+ SINGLE_QUERY_MANAGER_MAP.remove(TASK_ID_MAP_JOB_ID.remove(taskId)).close();
+ }
+ }
+
+ @Override
+ public ClusterLocalSingleQueryManager getSingleQuery(String taskId) {
+ long jobId = TASK_ID_MAP_JOB_ID.get(taskId);
+ return SINGLE_QUERY_MANAGER_MAP.get(jobId);
+ }
+
+ public static final ClusterLocalQueryManager getInstance() {
+ return ClusterLocalQueryManager.ClusterLocalQueryManagerHolder.INSTANCE;
+ }
+
+ private static class ClusterLocalQueryManagerHolder {
+
+ private static final ClusterLocalQueryManager INSTANCE = new ClusterLocalQueryManager();
+
+ private ClusterLocalQueryManagerHolder() {
+
+ }
+ }
+
+ @Override
+ public Map<String, Integer> getAllReadUsage() {
+ Map<String, Integer> readerUsageMap = new HashMap<>();
+ SINGLE_QUERY_MANAGER_MAP.values().forEach(singleQueryManager -> {
+ String groupId = singleQueryManager.getGroupId();
+ readerUsageMap.put(groupId, readerUsageMap.getOrDefault(groupId, 0) + 1);
+ });
+ return readerUsageMap;
+ }
+
+ @OnlyForTest
+ public static ConcurrentHashMap<String, Long> getTaskIdMapJobId() {
+ return TASK_ID_MAP_JOB_ID;
+ }
+
+ @OnlyForTest
+ public static ConcurrentHashMap<Long, ClusterLocalSingleQueryManager> getSingleQueryManagerMap() {
+ return SINGLE_QUERY_MANAGER_MAP;
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/querynode/ClusterLocalSingleQueryManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/querynode/ClusterLocalSingleQueryManager.java
new file mode 100644
index 0000000..559575a
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/querynode/ClusterLocalSingleQueryManager.java
@@ -0,0 +1,335 @@
+/**
+ * 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.cluster.query.manager.querynode;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledFuture;
+import org.apache.iotdb.cluster.concurrent.pool.QueryTimerManager;
+import org.apache.iotdb.cluster.config.ClusterConstant;
+import org.apache.iotdb.cluster.query.PathType;
+import org.apache.iotdb.cluster.query.factory.ClusterSeriesReaderFactory;
+import org.apache.iotdb.cluster.query.reader.querynode.AbstractClusterBatchReader;
+import org.apache.iotdb.cluster.query.reader.querynode.ClusterBatchReaderByTimestamp;
+import org.apache.iotdb.cluster.query.reader.querynode.ClusterBatchReaderWithoutTimeGenerator;
+import org.apache.iotdb.cluster.query.reader.querynode.ClusterFilterSeriesBatchReader;
+import org.apache.iotdb.cluster.query.reader.querynode.IClusterFilterSeriesBatchReader;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.InitSeriesReaderRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.QuerySeriesDataByTimestampRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.QuerySeriesDataRequest;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.InitSeriesReaderResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.QuerySeriesDataByTimestampResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.QuerySeriesDataResponse;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.qp.executor.OverflowQPExecutor;
+import org.apache.iotdb.db.qp.executor.QueryProcessExecutor;
+import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
+import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.executor.ExecutorWithoutTimeGenerator;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.expression.ExpressionType;
+import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class ClusterLocalSingleQueryManager implements IClusterLocalSingleQueryManager {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(ClusterLocalSingleQueryManager.class);
+
+ private String groupId;
+
+ /**
+ * Timer of Query, if the time is up, close query resource.
+ */
+ private ScheduledFuture<?> queryTimer;
+
+ /**
+ * Job id assigned by local QueryResourceManager
+ */
+ private long jobId;
+
+ /**
+ * Represents the number of query rounds, initial value is -1.
+ */
+ private long queryRound = -1;
+
+ /**
+ * Key is series full path, value is reader of select series
+ */
+ private Map<String, AbstractClusterBatchReader> selectSeriesReaders = new HashMap<>();
+
+ /**
+ * Filter reader
+ */
+ private IClusterFilterSeriesBatchReader filterReader;
+
+ /**
+ * Key is series full path, value is data type of series
+ */
+ private Map<String, TSDataType> dataTypeMap = new HashMap<>();
+
+ /**
+ * Cached batch data result
+ */
+ private List<BatchData> cachedBatchDataResult = new ArrayList<>();
+
+ private QueryProcessExecutor queryProcessExecutor = new OverflowQPExecutor();
+
+ /**
+ * Constructor of ClusterLocalSingleQueryManager
+ */
+ public ClusterLocalSingleQueryManager(long jobId) {
+ this.jobId = jobId;
+ queryTimer = QueryTimerManager.getInstance()
+ .execute(new QueryTimerRunnable(), ClusterConstant.QUERY_TIMEOUT_IN_QUERY_NODE);
+ }
+
+ @Override
+ public InitSeriesReaderResponse createSeriesReader(InitSeriesReaderRequest request)
+ throws IOException, PathErrorException, FileNodeManagerException, ProcessorException, QueryFilterOptimizationException {
+ this.groupId = request.getGroupID();
+ InitSeriesReaderResponse response = new InitSeriesReaderResponse(groupId);
+ QueryContext context = new QueryContext(jobId);
+ Map<PathType, QueryPlan> queryPlanMap = request.getAllQueryPlan();
+ if (queryPlanMap.containsKey(PathType.SELECT_PATH)) {
+ QueryPlan plan = queryPlanMap.get(PathType.SELECT_PATH);
+ if (plan instanceof GroupByPlan) {
+ throw new UnsupportedOperationException();
+ } else if (plan instanceof AggregationPlan) {
+ throw new UnsupportedOperationException();
+ } else {
+ if (plan.getExpression() == null
+ || plan.getExpression().getType() == ExpressionType.GLOBAL_TIME) {
+ handleSelectReaderWithoutTimeGenerator(plan, context, response);
+ } else {
+ handleSelectReaderWithTimeGenerator(plan, context, response);
+ }
+ }
+ }
+ if (queryPlanMap.containsKey(PathType.FILTER_PATH)) {
+ QueryPlan queryPlan = queryPlanMap.get(PathType.FILTER_PATH);
+ handleFilterSeriesReader(queryPlan, context, request, response, PathType.FILTER_PATH);
+ }
+ return response;
+ }
+
+ /**
+ * Handle filter series reader
+ *
+ * @param plan filter series query plan
+ */
+ private void handleFilterSeriesReader(QueryPlan plan, QueryContext context,
+ InitSeriesReaderRequest request, InitSeriesReaderResponse response, PathType pathType)
+ throws PathErrorException, QueryFilterOptimizationException, FileNodeManagerException, ProcessorException, IOException {
+ QueryDataSet queryDataSet = queryProcessExecutor
+ .processQuery(plan, context);
+ List<Path> paths = plan.getPaths();
+ List<TSDataType> dataTypes = queryDataSet.getDataTypes();
+ for (int i = 0; i < paths.size(); i++) {
+ dataTypeMap.put(paths.get(i).getFullPath(), dataTypes.get(i));
+ }
+ response.getSeriesDataTypes().put(pathType, dataTypes);
+ filterReader = new ClusterFilterSeriesBatchReader(queryDataSet, paths, request.getFilterList());
+ }
+
+ /**
+ * Handle select series query with no filter or only global time filter
+ *
+ * @param plan plan query plan
+ * @param context query context
+ * @param response response for coordinator node
+ */
+ private void handleSelectReaderWithoutTimeGenerator(QueryPlan plan, QueryContext context,
+ InitSeriesReaderResponse response)
+ throws FileNodeManagerException {
+ List<Path> paths = plan.getPaths();
+ Filter timeFilter = null;
+ if (plan.getExpression() != null) {
+ timeFilter = ((GlobalTimeExpression) plan.getExpression()).getFilter();
+ }
+ List<TSDataType> dataTypes = new ArrayList<>();
+ QueryResourceManager.getInstance()
+ .beginQueryOfGivenQueryPaths(context.getJobId(), plan.getPaths());
+ for (int i = 0; i < paths.size(); i++) {
+ String fullPath = paths.get(i).getFullPath();
+ IPointReader reader = ExecutorWithoutTimeGenerator
+ .createSeriesReader(context, paths.get(i), dataTypes, timeFilter);
+ selectSeriesReaders
+ .put(fullPath, new ClusterBatchReaderWithoutTimeGenerator(dataTypes.get(i), reader));
+ dataTypeMap.put(fullPath, dataTypes.get(i));
+ }
+ response.getSeriesDataTypes().put(PathType.SELECT_PATH, dataTypes);
+ }
+
+ /**
+ * Handle select series query with value filter
+ *
+ * @param plan plan query plan
+ * @param context query context
+ * @param response response for coordinator node
+ */
+ private void handleSelectReaderWithTimeGenerator(QueryPlan plan, QueryContext context,
+ InitSeriesReaderResponse response)
+ throws PathErrorException, FileNodeManagerException, IOException {
+ List<Path> paths = plan.getPaths();
+ List<TSDataType> dataTypeList = new ArrayList<>();
+ for (int i = 0; i < paths.size(); i++) {
+ Path path = paths.get(i);
+ EngineReaderByTimeStamp readerByTimeStamp = ClusterSeriesReaderFactory
+ .createReaderByTimeStamp(path, context);
+ TSDataType dataType = MManager.getInstance().getSeriesType(path.getFullPath());
+ selectSeriesReaders
+ .put(path.getFullPath(), new ClusterBatchReaderByTimestamp(readerByTimeStamp, dataType));
+ dataTypeMap.put(path.getFullPath(), dataType);
+ dataTypeList.add(dataType);
+ }
+ response.getSeriesDataTypes().put(PathType.SELECT_PATH, dataTypeList);
+ }
+
+ @Override
+ public QuerySeriesDataResponse readBatchData(QuerySeriesDataRequest request)
+ throws IOException {
+ resetQueryTimer();
+ QuerySeriesDataResponse response = new QuerySeriesDataResponse(request.getGroupID());
+ long targetQueryRounds = request.getQueryRounds();
+ if (targetQueryRounds != this.queryRound) {
+ this.queryRound = targetQueryRounds;
+ PathType pathType = request.getPathType();
+ List<String> paths = request.getSeriesPaths();
+ List<BatchData> batchDataList;
+ if (pathType == PathType.SELECT_PATH) {
+ batchDataList = readSelectSeriesBatchData(paths);
+ } else {
+ batchDataList = readFilterSeriesBatchData();
+ }
+ cachedBatchDataResult = batchDataList;
+ }
+ response.setSeriesBatchData(cachedBatchDataResult);
+ return response;
+ }
+
+ @Override
+ public QuerySeriesDataByTimestampResponse readBatchDataByTimestamp(
+ QuerySeriesDataByTimestampRequest request)
+ throws IOException {
+ resetQueryTimer();
+ QuerySeriesDataByTimestampResponse response = new QuerySeriesDataByTimestampResponse(groupId);
+ List<String> fetchDataSeries = request.getFetchDataSeries();
+ long targetQueryRounds = request.getQueryRounds();
+ if (targetQueryRounds != this.queryRound) {
+ this.queryRound = targetQueryRounds;
+ List<BatchData> batchDataList = new ArrayList<>();
+ for (String series : fetchDataSeries) {
+ AbstractClusterBatchReader reader = selectSeriesReaders.get(series);
+ batchDataList.add(reader.nextBatch(request.getBatchTimestamp()));
+ }
+ cachedBatchDataResult = batchDataList;
+ }
+ response.setSeriesBatchData(cachedBatchDataResult);
+ return response;
+ }
+
+ @Override
+ public void resetQueryTimer() {
+ queryTimer.cancel(false);
+ queryTimer = QueryTimerManager.getInstance()
+ .execute(new QueryTimerRunnable(), ClusterConstant.QUERY_TIMEOUT_IN_QUERY_NODE);
+ }
+
+ /**
+ * Read batch data of select series
+ *
+ * @param paths all series to query
+ */
+ private List<BatchData> readSelectSeriesBatchData(List<String> paths) throws IOException {
+ List<BatchData> batchDataList = new ArrayList<>();
+ for (String fullPath : paths) {
+ batchDataList.add(selectSeriesReaders.get(fullPath).nextBatch());
+ }
+ return batchDataList;
+ }
+
+ /**
+ * Read batch data of filter series
+ *
+ * @return batch data of all filter series
+ */
+ private List<BatchData> readFilterSeriesBatchData() throws IOException {
+ return filterReader.nextBatchList();
+ }
+
+ public String getGroupId() {
+ return groupId;
+ }
+
+ @Override
+ public void close() throws FileNodeManagerException {
+ queryTimer.cancel(false);
+ QueryResourceManager.getInstance().endQueryForGivenJob(jobId);
+ }
+
+ public long getJobId() {
+ return jobId;
+ }
+
+ public long getQueryRound() {
+ return queryRound;
+ }
+
+ public Map<String, AbstractClusterBatchReader> getSelectSeriesReaders() {
+ return selectSeriesReaders;
+ }
+
+ public IClusterFilterSeriesBatchReader getFilterReader() {
+ return filterReader;
+ }
+
+ public Map<String, TSDataType> getDataTypeMap() {
+ return dataTypeMap;
+ }
+
+ public class QueryTimerRunnable implements Runnable {
+
+ @Override
+ public void run() {
+ try {
+ close();
+ } catch (FileNodeManagerException e) {
+ LOGGER.error(e.getMessage());
+ }
+ }
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/querynode/IClusterLocalQueryManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/querynode/IClusterLocalQueryManager.java
new file mode 100644
index 0000000..cc0f103
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/querynode/IClusterLocalQueryManager.java
@@ -0,0 +1,82 @@
+/**
+ * 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.cluster.query.manager.querynode;
+
+import java.io.IOException;
+import java.util.Map;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.InitSeriesReaderRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.QuerySeriesDataByTimestampRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.QuerySeriesDataRequest;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.InitSeriesReaderResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.QuerySeriesDataByTimestampResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.QuerySeriesDataResponse;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+
+/**
+ * Manage all local query resources which provide data for coordinator node in cluster query node.
+ */
+public interface IClusterLocalQueryManager {
+
+ /**
+ * Initially create query data set for coordinator node.
+ *
+ * @param request request for query data from coordinator node
+ */
+ InitSeriesReaderResponse createQueryDataSet(InitSeriesReaderRequest request)
+ throws IOException, FileNodeManagerException, PathErrorException, ProcessorException, QueryFilterOptimizationException;
+
+ /**
+ * Read batch data of all querying series in request and set response.
+ *
+ * @param request request of querying series
+ */
+ QuerySeriesDataResponse readBatchData(QuerySeriesDataRequest request)
+ throws IOException;
+
+ /**
+ * Read batch data of select series by batch timestamp which is used in query with value filter
+ * @param request request of querying select paths
+ *
+ */
+ QuerySeriesDataByTimestampResponse readBatchDataByTimestamp(
+ QuerySeriesDataByTimestampRequest request) throws IOException;
+
+ /**
+ * Close query resource of a task
+ *
+ * @param taskId task id of local single query manager
+ */
+ void close(String taskId) throws FileNodeManagerException;
+
+
+ /**
+ * Get query manager by taskId
+ *
+ * @param taskId task id assigned by ClusterRpcQueryManager
+ */
+ ClusterLocalSingleQueryManager getSingleQuery(String taskId);
+
+ /**
+ * Get all read usage count group by data group id, key is group id, value is usage count
+ */
+ Map<String, Integer> getAllReadUsage();
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/querynode/IClusterLocalSingleQueryManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/querynode/IClusterLocalSingleQueryManager.java
new file mode 100644
index 0000000..318772f
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/manager/querynode/IClusterLocalSingleQueryManager.java
@@ -0,0 +1,74 @@
+/**
+ * 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.cluster.query.manager.querynode;
+
+import java.io.IOException;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.InitSeriesReaderRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.QuerySeriesDataByTimestampRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.QuerySeriesDataRequest;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.InitSeriesReaderResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.QuerySeriesDataByTimestampResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.QuerySeriesDataResponse;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
+
+/**
+ * <p>
+ * Manage all series reader in a query as a query node, cooperate with coordinator node for a client
+ * query
+ * </p>
+ */
+public interface IClusterLocalSingleQueryManager {
+
+ /**
+ * Initially create corresponding series readers.
+ * @param request request of querying series data
+ */
+ InitSeriesReaderResponse createSeriesReader(InitSeriesReaderRequest request)
+ throws IOException, PathErrorException, FileNodeManagerException, ProcessorException, QueryFilterOptimizationException;
+
+ /**
+ * <p>
+ * Read batch data If query round in cache is equal to target query round, it means that batch
+ * data in query node transfer to coordinator fail and return cached batch data.
+ * </p>
+ * @param request request of querying series data
+ *
+ */
+ QuerySeriesDataResponse readBatchData(QuerySeriesDataRequest request)
+ throws IOException;
+
+ /**
+ * Read batch data of select paths by timestamp
+ */
+ QuerySeriesDataByTimestampResponse readBatchDataByTimestamp(
+ QuerySeriesDataByTimestampRequest request) throws IOException;
+
+ /**
+ * Reset query timer and restart timer
+ */
+ void resetQueryTimer();
+
+ /**
+ * Release query resource
+ */
+ void close() throws FileNodeManagerException;
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/coordinatornode/AbstractClusterPointReader.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/coordinatornode/AbstractClusterPointReader.java
new file mode 100644
index 0000000..72c7c70
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/coordinatornode/AbstractClusterPointReader.java
@@ -0,0 +1,73 @@
+/**
+ * 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.cluster.query.reader.coordinatornode;
+
+import java.io.IOException;
+import org.apache.iotdb.cluster.exception.RaftConnectionException;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.db.utils.TimeValuePairUtils;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+/**
+ * Cluster point reader
+ */
+public abstract class AbstractClusterPointReader implements IPointReader {
+
+ /**
+ * Current time value pair
+ */
+ protected TimeValuePair currentTimeValuePair;
+
+ /**
+ * Current batch data
+ */
+ protected BatchData currentBatchData;
+
+ @Override
+ public boolean hasNext() throws IOException {
+ if (currentBatchData == null || !currentBatchData.hasNext()) {
+ try {
+ updateCurrentBatchData();
+ } catch (RaftConnectionException e) {
+ throw new IOException(e);
+ }
+ if (currentBatchData == null || !currentBatchData.hasNext()) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ /**
+ * Update current batch data. If necessary ,fetch batch data from remote query node
+ */
+ protected abstract void updateCurrentBatchData() throws RaftConnectionException;
+
+ @Override
+ public TimeValuePair next() throws IOException {
+ if (hasNext()) {
+ TimeValuePair timeValuePair = TimeValuePairUtils.getCurrentTimeValuePair(currentBatchData);
+ currentTimeValuePair = timeValuePair;
+ currentBatchData.next();
+ return timeValuePair;
+ }
+ return null;
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/coordinatornode/ClusterFilterSeriesReader.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/coordinatornode/ClusterFilterSeriesReader.java
new file mode 100644
index 0000000..805d3af
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/coordinatornode/ClusterFilterSeriesReader.java
@@ -0,0 +1,126 @@
+/**
+ * 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.cluster.query.reader.coordinatornode;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import org.apache.iotdb.cluster.exception.RaftConnectionException;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcSingleQueryManager;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+
+/**
+ * Filter series reader which is used in coordinator node.
+ */
+public class ClusterFilterSeriesReader extends AbstractClusterPointReader {
+
+ /**
+ * Data group id
+ */
+ private String groupId;
+
+ /**
+ * Manager of the whole query
+ */
+ private ClusterRpcSingleQueryManager queryManager;
+
+ /**
+ * Series name
+ */
+ private Path seriesPath;
+
+ /**
+ * Data type
+ */
+ private TSDataType dataType;
+
+ /**
+ * Batch data
+ */
+ private LinkedList<BatchData> batchDataList;
+
+ /**
+ * Mark whether remote has data
+ */
+ private boolean remoteDataFinish;
+
+ public ClusterFilterSeriesReader(String groupId, Path seriesPath, TSDataType dataType,
+ ClusterRpcSingleQueryManager queryManager) {
+ this.groupId = groupId;
+ this.seriesPath = seriesPath;
+ this.dataType = dataType;
+ this.queryManager = queryManager;
+ this.batchDataList = new LinkedList<>();
+ remoteDataFinish = false;
+ }
+
+ @Override
+ public TimeValuePair current() throws IOException {
+ return currentTimeValuePair;
+ }
+
+ /**
+ * Update current batch data. If necessary ,fetch batch data from remote query node
+ */
+ @Override
+ protected void updateCurrentBatchData() throws RaftConnectionException {
+ if (batchDataList.isEmpty() && !remoteDataFinish) {
+ queryManager.fetchBatchDataForFilterPaths(groupId);
+ }
+ if (!batchDataList.isEmpty()) {
+ currentBatchData = batchDataList.removeFirst();
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ //Do nothing
+ }
+
+ public Path getSeriesPath() {
+ return seriesPath;
+ }
+
+ public void setSeriesPath(Path seriesPath) {
+ this.seriesPath = seriesPath;
+ }
+
+ public TSDataType getDataType() {
+ return dataType;
+ }
+
+ public void setDataType(TSDataType dataType) {
+ this.dataType = dataType;
+ }
+
+ public BatchData getCurrentBatchData() {
+ return currentBatchData;
+ }
+
+ public void setCurrentBatchData(BatchData currentBatchData) {
+ this.currentBatchData = currentBatchData;
+ }
+
+ public void addBatchData(BatchData batchData, boolean remoteDataFinish) {
+ batchDataList.addLast(batchData);
+ this.remoteDataFinish = remoteDataFinish;
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/coordinatornode/ClusterSelectSeriesReader.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/coordinatornode/ClusterSelectSeriesReader.java
new file mode 100644
index 0000000..0a507d5
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/coordinatornode/ClusterSelectSeriesReader.java
@@ -0,0 +1,167 @@
+/**
+ * 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.cluster.query.reader.coordinatornode;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.exception.RaftConnectionException;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcSingleQueryManager;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+
+/**
+ * Select series reader which is used in coordinator node.
+ */
+public class ClusterSelectSeriesReader extends AbstractClusterPointReader implements
+ EngineReaderByTimeStamp {
+
+ /**
+ * Data group id
+ */
+ private String groupId;
+
+ /**
+ * Manager of the whole query
+ */
+ private ClusterRpcSingleQueryManager queryManager;
+
+ /**
+ * Series name
+ */
+ private Path seriesPath;
+
+ /**
+ * Data type
+ */
+ private TSDataType dataType;
+
+ /**
+ * Batch data
+ */
+ private LinkedList<BatchData> batchDataList;
+
+ /**
+ * Mark whether remote has data
+ */
+ private boolean remoteDataFinish;
+
+ public ClusterSelectSeriesReader(String groupId, Path seriesPath, TSDataType dataType,
+ ClusterRpcSingleQueryManager queryManager) {
+ this.groupId = groupId;
+ this.seriesPath = seriesPath;
+ this.dataType = dataType;
+ this.queryManager = queryManager;
+ this.batchDataList = new LinkedList<>();
+ this.remoteDataFinish = false;
+ }
+
+ @Override
+ public TimeValuePair current() throws IOException {
+ return currentTimeValuePair;
+ }
+
+ @Override
+ public Object getValueInTimestamp(long timestamp) throws IOException {
+ if (currentTimeValuePair != null && currentTimeValuePair.getTimestamp() == timestamp) {
+ return currentTimeValuePair.getValue().getValue();
+ } else if (currentTimeValuePair != null && currentTimeValuePair.getTimestamp() > timestamp) {
+ return null;
+ }
+ while (true) {
+ if (hasNext()) {
+ next();
+ if (currentTimeValuePair.getTimestamp() == timestamp) {
+ return currentTimeValuePair.getValue().getValue();
+ } else if (currentTimeValuePair.getTimestamp() > timestamp) {
+ return null;
+ }
+ } else {
+ return null;
+ }
+ }
+ }
+
+ /**
+ * Update current batch data. If necessary ,fetch batch data from remote query node
+ */
+ @Override
+ protected void updateCurrentBatchData() throws RaftConnectionException {
+ if (batchDataList.isEmpty() && !remoteDataFinish) {
+ queryManager.fetchBatchDataForSelectPaths(groupId);
+ }
+ if (!batchDataList.isEmpty()) {
+ currentBatchData = batchDataList.removeFirst();
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ batchDataList = null;
+ }
+
+ public Path getSeriesPath() {
+ return seriesPath;
+ }
+
+ public void setSeriesPath(Path seriesPath) {
+ this.seriesPath = seriesPath;
+ }
+
+ public TSDataType getDataType() {
+ return dataType;
+ }
+
+ public void setDataType(TSDataType dataType) {
+ this.dataType = dataType;
+ }
+
+ public BatchData getCurrentBatchData() {
+ return currentBatchData;
+ }
+
+ public void setCurrentBatchData(BatchData currentBatchData) {
+ this.currentBatchData = currentBatchData;
+ }
+
+ public void addBatchData(BatchData batchData, boolean remoteDataFinish) {
+ batchDataList.addLast(batchData);
+ this.remoteDataFinish = remoteDataFinish;
+ }
+
+ public boolean isRemoteDataFinish() {
+ return remoteDataFinish;
+ }
+
+ public void setRemoteDataFinish(boolean remoteDataFinish) {
+ this.remoteDataFinish = remoteDataFinish;
+ }
+
+ /**
+ * Check if this series need to fetch data from remote query node
+ */
+ public boolean enableFetchData() {
+ return !remoteDataFinish
+ && batchDataList.size() <= ClusterDescriptor.getInstance().getConfig()
+ .getMaxCachedBatchDataListSize();
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/DataGroupNonQueryRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/querynode/AbstractClusterBatchReader.java
similarity index 61%
copy from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/DataGroupNonQueryRequest.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/query/reader/querynode/AbstractClusterBatchReader.java
index c1bcf5f..b0a86bd 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/DataGroupNonQueryRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/querynode/AbstractClusterBatchReader.java
@@ -16,23 +16,24 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.request;
+package org.apache.iotdb.cluster.query.reader.querynode;
import java.io.IOException;
-import java.io.Serializable;
import java.util.List;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.query.reader.IBatchReader;
+import org.apache.iotdb.tsfile.read.common.BatchData;
/**
- * Handle request to data group
+ * Cluster batch reader, which provides another method to get batch data by batch timestamp.
*/
-public class DataGroupNonQueryRequest extends BasicRequest implements Serializable {
+public abstract class AbstractClusterBatchReader implements IBatchReader {
-
- public DataGroupNonQueryRequest(String groupID, List<PhysicalPlan> physicalPlanBytes)
- throws IOException {
- super(groupID);
- init(physicalPlanBytes);
- }
+ /**
+ * Get batch data by batch time
+ *
+ * @param batchTime valid batch timestamp
+ * @return corresponding batch data
+ */
+ public abstract BatchData nextBatch(List<Long> batchTime) throws IOException;
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/querynode/ClusterBatchReaderByTimestamp.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/querynode/ClusterBatchReaderByTimestamp.java
new file mode 100644
index 0000000..b8c36eb
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/querynode/ClusterBatchReaderByTimestamp.java
@@ -0,0 +1,86 @@
+/**
+ * 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.cluster.query.reader.querynode;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+/**
+ * BatchReader by timestamp for cluster which is used in query node.
+ */
+public class ClusterBatchReaderByTimestamp extends AbstractClusterBatchReader {
+
+ /**
+ * Reader
+ */
+ private EngineReaderByTimeStamp readerByTimeStamp;
+
+ /**
+ * Data type
+ */
+ private TSDataType dataType;
+
+ public ClusterBatchReaderByTimestamp(
+ EngineReaderByTimeStamp readerByTimeStamp,
+ TSDataType dataType) {
+ this.readerByTimeStamp = readerByTimeStamp;
+ this.dataType = dataType;
+ }
+
+ @Override
+ public boolean hasNext() throws IOException {
+ return readerByTimeStamp.hasNext();
+ }
+
+ @Override
+ public BatchData nextBatch() throws IOException {
+ throw new UnsupportedOperationException(
+ "nextBatch() in ClusterBatchReaderByTimestamp is an empty method.");
+ }
+
+
+ @Override
+ public void close() throws IOException {
+ // do nothing
+ }
+
+ @Override
+ public BatchData nextBatch(List<Long> batchTime) throws IOException {
+ BatchData batchData = new BatchData(dataType, true);
+ for(long time: batchTime){
+ Object value = readerByTimeStamp.getValueInTimestamp(time);
+ if(value != null){
+ batchData.putTime(time);
+ batchData.putAnObject(value);
+ }
+ }
+ return batchData;
+ }
+
+ public EngineReaderByTimeStamp getReaderByTimeStamp() {
+ return readerByTimeStamp;
+ }
+
+ public TSDataType getDataType() {
+ return dataType;
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/querynode/ClusterBatchReaderWithoutTimeGenerator.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/querynode/ClusterBatchReaderWithoutTimeGenerator.java
new file mode 100644
index 0000000..f3d443f
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/querynode/ClusterBatchReaderWithoutTimeGenerator.java
@@ -0,0 +1,94 @@
+/**
+ * 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.cluster.query.reader.querynode;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.iotdb.cluster.config.ClusterConfig;
+import org.apache.iotdb.cluster.config.ClusterConstant;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+/**
+ * BatchReader without time generator for cluster which is used in query node.
+ */
+public class ClusterBatchReaderWithoutTimeGenerator extends AbstractClusterBatchReader {
+
+ /**
+ * Data type
+ */
+ private TSDataType dataType;
+
+ /**
+ * Point reader
+ */
+ private IPointReader reader;
+
+ private static final ClusterConfig CLUSTER_CONF = ClusterDescriptor.getInstance().getConfig();
+
+ public ClusterBatchReaderWithoutTimeGenerator(
+ TSDataType dataType, IPointReader reader) {
+ this.dataType = dataType;
+ this.reader = reader;
+ }
+
+ @Override
+ public boolean hasNext() throws IOException {
+ return reader.hasNext();
+ }
+
+ @Override
+ public BatchData nextBatch() throws IOException {
+ BatchData batchData = new BatchData(dataType, true);
+ for (int i = 0; i < CLUSTER_CONF.getBatchReadSize(); i++) {
+ if (hasNext()) {
+ TimeValuePair pair = reader.next();
+ batchData.putTime(pair.getTimestamp());
+ batchData.putAnObject(pair.getValue().getValue());
+ } else {
+ break;
+ }
+ }
+ return batchData;
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (reader != null) {
+ reader.close();
+ }
+ }
+
+ @Override
+ public BatchData nextBatch(List<Long> batchTime) throws IOException {
+ throw new IOException(
+ "nextBatch(List<Long> batchTime) in ClusterBatchReaderWithoutTimeGenerator is an empty method.");
+ }
+
+ public TSDataType getDataType() {
+ return dataType;
+ }
+
+ public IPointReader getReader() {
+ return reader;
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/querynode/ClusterFilterSeriesBatchReader.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/querynode/ClusterFilterSeriesBatchReader.java
new file mode 100644
index 0000000..6690999
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/querynode/ClusterFilterSeriesBatchReader.java
@@ -0,0 +1,121 @@
+/**
+ * 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.cluster.query.reader.querynode;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.cluster.config.ClusterConfig;
+import org.apache.iotdb.cluster.config.ClusterConstant;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Field;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+
+/**
+ * Batch reader for all filter paths.
+ */
+public class ClusterFilterSeriesBatchReader implements IClusterFilterSeriesBatchReader {
+
+ private List<Path> allFilterPath;
+
+ private List<Filter> filters;
+
+ private QueryDataSet queryDataSet;
+
+ private static final ClusterConfig CLUSTER_CONF = ClusterDescriptor.getInstance().getConfig();
+
+ public ClusterFilterSeriesBatchReader(QueryDataSet queryDataSet, List<Path> allFilterPath,
+ List<Filter> filters) {
+ this.queryDataSet = queryDataSet;
+ this.allFilterPath = allFilterPath;
+ this.filters = filters;
+ }
+
+ @Override
+ public boolean hasNext() throws IOException {
+ return queryDataSet.hasNext();
+ }
+
+ /**
+ * Get batch data of all filter series by next batch time which is determined by
+ * <code>queryDataSet</code>
+ */
+ @Override
+ public List<BatchData> nextBatchList() throws IOException {
+ List<BatchData> batchDataList = new ArrayList<>(allFilterPath.size());
+ List<TSDataType> dataTypeList = queryDataSet.getDataTypes();
+ for (int i = 0; i < allFilterPath.size(); i++) {
+ batchDataList.add(new BatchData(dataTypeList.get(i), true));
+ }
+ int dataPointCount = 0;
+ while(true){
+ if(!hasNext() || dataPointCount == CLUSTER_CONF.getBatchReadSize()){
+ break;
+ }
+ if(hasNext() && addTimeValuePair(batchDataList, dataTypeList)){
+ dataPointCount++;
+ }
+ }
+ return batchDataList;
+ }
+
+ /**
+ * Add a time-value pair to batch data
+ */
+ private boolean addTimeValuePair(List<BatchData> batchDataList, List<TSDataType> dataTypeList)
+ throws IOException {
+ boolean hasField = false;
+ RowRecord rowRecord = queryDataSet.next();
+ long time = rowRecord.getTimestamp();
+ List<Field> fieldList = rowRecord.getFields();
+ for (int j = 0; j < allFilterPath.size(); j++) {
+ if (!fieldList.get(j).isNull()) {
+ BatchData batchData = batchDataList.get(j);
+ Object value = fieldList.get(j).getObjectValue(dataTypeList.get(j));
+ if (filters.get(j).satisfy(time, value)) {
+ hasField = true;
+ batchData.putTime(time);
+ batchData.putAnObject(value);
+ }
+ }
+ }
+ return hasField;
+ }
+
+ public List<Path> getAllFilterPath() {
+ return allFilterPath;
+ }
+
+ public void setAllFilterPath(List<Path> allFilterPath) {
+ this.allFilterPath = allFilterPath;
+ }
+
+ public QueryDataSet getQueryDataSet() {
+ return queryDataSet;
+ }
+
+ public void setQueryDataSet(QueryDataSet queryDataSet) {
+ this.queryDataSet = queryDataSet;
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/MetaGroupNonQueryRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/querynode/IClusterFilterSeriesBatchReader.java
similarity index 68%
copy from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/MetaGroupNonQueryRequest.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/query/reader/querynode/IClusterFilterSeriesBatchReader.java
index 69625ff..218d68b 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/MetaGroupNonQueryRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/reader/querynode/IClusterFilterSeriesBatchReader.java
@@ -16,22 +16,21 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.request;
+package org.apache.iotdb.cluster.query.reader.querynode;
import java.io.IOException;
-import java.io.Serializable;
import java.util.List;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.tsfile.read.common.BatchData;
/**
- * Handle request to metadata group leader
+ * Batch reader for filter series which is used in query node.
*/
-public class MetaGroupNonQueryRequest extends BasicRequest implements Serializable {
+public interface IClusterFilterSeriesBatchReader {
- public MetaGroupNonQueryRequest(String groupID, List<PhysicalPlan> plans)
- throws IOException {
- super(groupID);
- this.init(plans);
- }
+ boolean hasNext() throws IOException;
+ /**
+ * Get next batch data of all filter series.
+ */
+ List<BatchData> nextBatchList() throws IOException;
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/DataGroupNonQueryRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/timegenerator/ClusterLeafNode.java
similarity index 56%
copy from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/DataGroupNonQueryRequest.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/query/timegenerator/ClusterLeafNode.java
index c1bcf5f..39d4be4 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/DataGroupNonQueryRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/timegenerator/ClusterLeafNode.java
@@ -16,23 +16,34 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.request;
+package org.apache.iotdb.cluster.query.timegenerator;
import java.io.IOException;
-import java.io.Serializable;
-import java.util.List;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.query.reader.IReader;
+import org.apache.iotdb.tsfile.read.query.timegenerator.node.Node;
+import org.apache.iotdb.tsfile.read.query.timegenerator.node.NodeType;
-/**
- * Handle request to data group
- */
-public class DataGroupNonQueryRequest extends BasicRequest implements Serializable {
+public class ClusterLeafNode implements Node {
+
+ private IReader reader;
+
+ public ClusterLeafNode(IReader reader) {
+ this.reader = reader;
+ }
+
+ @Override
+ public boolean hasNext() throws IOException {
+ return reader.hasNext();
+ }
+ @Override
+ public long next() throws IOException {
+ return reader.next().getTimestamp();
+ }
- public DataGroupNonQueryRequest(String groupID, List<PhysicalPlan> physicalPlanBytes)
- throws IOException {
- super(groupID);
- init(physicalPlanBytes);
+ @Override
+ public NodeType getType() {
+ return NodeType.LEAF;
}
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/timegenerator/ClusterNodeConstructor.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/timegenerator/ClusterNodeConstructor.java
new file mode 100644
index 0000000..639dce8
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/timegenerator/ClusterNodeConstructor.java
@@ -0,0 +1,106 @@
+/**
+ * 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.cluster.query.timegenerator;
+
+import static org.apache.iotdb.tsfile.read.expression.ExpressionType.SERIES;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcSingleQueryManager;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.FilterGroupEntity;
+import org.apache.iotdb.cluster.query.reader.coordinatornode.ClusterFilterSeriesReader;
+import org.apache.iotdb.cluster.utils.QPExecutorUtils;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.timegenerator.AbstractNodeConstructor;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.expression.IExpression;
+import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
+import org.apache.iotdb.tsfile.read.query.timegenerator.node.Node;
+
+public class ClusterNodeConstructor extends AbstractNodeConstructor {
+
+ /**
+ * Single query manager
+ */
+ private ClusterRpcSingleQueryManager queryManager;
+
+ /**
+ * Filter series reader group by group id
+ */
+ private Map<String, List<ClusterFilterSeriesReader>> filterSeriesReadersByGroupId;
+
+ /**
+ * Mark filter series reader index group by group id
+ */
+ private Map<String, Integer> filterSeriesReaderIndex;
+
+ public ClusterNodeConstructor(ClusterRpcSingleQueryManager queryManager) {
+ this.queryManager = queryManager;
+ this.filterSeriesReadersByGroupId = new HashMap<>();
+ this.filterSeriesReaderIndex = new HashMap<>();
+ this.init(queryManager);
+ }
+
+ /**
+ * Init filter series reader
+ */
+ private void init(ClusterRpcSingleQueryManager queryManager) {
+ Map<String, FilterGroupEntity> filterGroupEntityMap = queryManager.getFilterGroupEntityMap();
+ filterGroupEntityMap.forEach(
+ (key, value) -> filterSeriesReadersByGroupId.put(key, value.getFilterSeriesReaders()));
+ filterSeriesReadersByGroupId.forEach((key, value) -> filterSeriesReaderIndex.put(key, 0));
+ }
+
+ /**
+ * Construct expression node.
+ *
+ * @param expression expression
+ * @return Node object
+ * @throws IOException IOException
+ * @throws FileNodeManagerException FileNodeManagerException
+ */
+ @Override
+ public Node construct(IExpression expression, QueryContext context)
+ throws FileNodeManagerException {
+ if (expression.getType() == SERIES) {
+ try {
+ Path seriesPath = ((SingleSeriesExpression) expression).getSeriesPath();
+ String groupId = QPExecutorUtils.getGroupIdByDevice(seriesPath.getDevice());
+ if (filterSeriesReadersByGroupId.containsKey(groupId)) {
+ List<ClusterFilterSeriesReader> seriesReaders = filterSeriesReadersByGroupId.get(groupId);
+ int readerIndex = filterSeriesReaderIndex.get(groupId);
+ filterSeriesReaderIndex.put(groupId, readerIndex + 1);
+ return new ClusterLeafNode(seriesReaders.get(readerIndex));
+ } else {
+ queryManager.addDataGroupUsage(groupId);
+ return new ClusterLeafNode(generateSeriesReader((SingleSeriesExpression) expression,
+ context));
+ }
+ } catch (IOException | PathErrorException e) {
+ throw new FileNodeManagerException(e);
+ }
+ } else {
+ return constructNotSeriesNode(expression, context);
+ }
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineTimeGenerator.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/timegenerator/ClusterTimeGenerator.java
similarity index 67%
copy from iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineTimeGenerator.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/query/timegenerator/ClusterTimeGenerator.java
index 350ea6f..f2b72d1 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineTimeGenerator.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/timegenerator/ClusterTimeGenerator.java
@@ -16,9 +16,10 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.db.query.timegenerator;
+package org.apache.iotdb.cluster.query.timegenerator;
import java.io.IOException;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcSingleQueryManager;
import org.apache.iotdb.db.exception.FileNodeManagerException;
import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.tsfile.read.common.Path;
@@ -26,27 +27,24 @@ import org.apache.iotdb.tsfile.read.expression.IExpression;
import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGenerator;
import org.apache.iotdb.tsfile.read.query.timegenerator.node.Node;
-/**
- * A timestamp generator for query with filter. e.g. For query clause "select s1, s2 form root where
- * s3 < 0 and time > 100", this class can iterate back to every timestamp of the query.
- */
-public class EngineTimeGenerator implements TimeGenerator {
-
+public class ClusterTimeGenerator implements TimeGenerator {
private IExpression expression;
private Node operatorNode;
/**
- * Constructor of EngineTimeGenerator.
+ * Constructor of Cluster TimeGenerator.
*/
- public EngineTimeGenerator(IExpression expression, QueryContext context)
+ public ClusterTimeGenerator(IExpression expression, QueryContext context,
+ ClusterRpcSingleQueryManager queryManager)
throws FileNodeManagerException {
this.expression = expression;
- initNode(context);
+ initNode(context, queryManager);
}
- private void initNode(QueryContext context) throws FileNodeManagerException {
- EngineNodeConstructor engineNodeConstructor = new EngineNodeConstructor();
- this.operatorNode = engineNodeConstructor.construct(expression, context);
+ private void initNode(QueryContext context, ClusterRpcSingleQueryManager queryManager)
+ throws FileNodeManagerException {
+ ClusterNodeConstructor nodeConstructor = new ClusterNodeConstructor(queryManager);
+ this.operatorNode = nodeConstructor.construct(expression, context);
}
@Override
@@ -60,9 +58,7 @@ public class EngineTimeGenerator implements TimeGenerator {
}
@Override
- public Object getValue(Path path, long time) {
- // TODO implement the optimization
+ public Object getValue(Path path, long time) throws IOException {
return null;
}
-
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/utils/ClusterRpcReaderUtils.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/utils/ClusterRpcReaderUtils.java
new file mode 100644
index 0000000..c3df421
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/utils/ClusterRpcReaderUtils.java
@@ -0,0 +1,128 @@
+/**
+ * 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.cluster.query.utils;
+
+import com.alipay.sofa.jraft.entity.PeerId;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.exception.RaftConnectionException;
+import org.apache.iotdb.cluster.qp.task.QPTask.TaskState;
+import org.apache.iotdb.cluster.qp.task.QueryTask;
+import org.apache.iotdb.cluster.query.PathType;
+import org.apache.iotdb.cluster.rpc.raft.NodeAsClient;
+import org.apache.iotdb.cluster.rpc.raft.request.BasicRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.CloseSeriesReaderRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.InitSeriesReaderRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.QuerySeriesDataByTimestampRequest;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.QuerySeriesDataRequest;
+import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.QuerySeriesDataByTimestampResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.QuerySeriesDataResponse;
+import org.apache.iotdb.cluster.utils.RaftUtils;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+
+/**
+ * Utils for cluster reader which needs to acquire data from remote query node.
+ */
+public class ClusterRpcReaderUtils {
+
+ /**
+ * Count limit to redo a task
+ */
+ private static final int TASK_MAX_RETRY = ClusterDescriptor.getInstance().getConfig()
+ .getQpTaskRedoCount();
+
+ /**
+ * Create cluster series reader
+ *
+ * @param peerId query node to fetch data
+ * @param readDataConsistencyLevel consistency level of read data
+ * @param taskId task id assigned by coordinator node
+ */
+ public static BasicResponse createClusterSeriesReader(String groupId, PeerId peerId,
+ int readDataConsistencyLevel, Map<PathType, QueryPlan> allQueryPlan, String taskId,
+ List<Filter> filterList)
+ throws IOException, RaftConnectionException {
+
+ /** handle request **/
+ BasicRequest request = InitSeriesReaderRequest
+ .createInitialQueryRequest(groupId, taskId, readDataConsistencyLevel,
+ allQueryPlan, filterList);
+ return handleQueryRequest(request, peerId, 0);
+ }
+
+ public static QuerySeriesDataResponse fetchBatchData(String groupID, PeerId peerId, String taskId,
+ PathType pathType, List<String> fetchDataSeries, long queryRounds)
+ throws RaftConnectionException {
+ BasicRequest request = QuerySeriesDataRequest
+ .createFetchDataRequest(groupID, taskId, pathType, fetchDataSeries, queryRounds);
+ return (QuerySeriesDataResponse) handleQueryRequest(request, peerId, 0);
+ }
+
+ public static QuerySeriesDataByTimestampResponse fetchBatchDataByTimestamp(String groupId,
+ PeerId peerId, String taskId, long queryRounds, List<Long> batchTimestamp,
+ List<String> fetchDataSeries)
+ throws RaftConnectionException {
+ BasicRequest request = QuerySeriesDataByTimestampRequest
+ .createRequest(groupId, queryRounds, taskId, batchTimestamp, fetchDataSeries);
+ return (QuerySeriesDataByTimestampResponse) handleQueryRequest(request, peerId, 0);
+ }
+
+ /**
+ * Release remote query resources
+ *
+ * @param groupId data group id
+ * @param peerId target query node
+ * @param taskId unique task id
+ */
+ public static void releaseRemoteQueryResource(String groupId, PeerId peerId, String taskId)
+ throws RaftConnectionException {
+
+ BasicRequest request = CloseSeriesReaderRequest.createReleaseResourceRequest(groupId, taskId);
+ handleQueryRequest(request, peerId, 0);
+ }
+
+ /**
+ * Send query request to remote node and return response
+ *
+ * @param request query request
+ * @param peerId target remote query node
+ * @param taskRetryNum retry num of the request
+ * @return Response from remote query node
+ */
+ private static BasicResponse handleQueryRequest(BasicRequest request, PeerId peerId,
+ int taskRetryNum)
+ throws RaftConnectionException {
+ if (taskRetryNum > TASK_MAX_RETRY) {
+ throw new RaftConnectionException(
+ String.format("Query request retries reach the upper bound %s",
+ TASK_MAX_RETRY));
+ }
+ NodeAsClient nodeAsClient = RaftUtils.getRaftNodeAsClient();
+ QueryTask queryTask = nodeAsClient.syncHandleRequest(request, peerId);
+ if (queryTask.getState() == TaskState.FINISH) {
+ return queryTask.getBasicResponse();
+ } else {
+ return handleQueryRequest(request, peerId, taskRetryNum + 1);
+ }
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/utils/ExpressionUtils.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/utils/ExpressionUtils.java
new file mode 100644
index 0000000..0024138
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/utils/ExpressionUtils.java
@@ -0,0 +1,131 @@
+/**
+ * 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.cluster.query.utils;
+
+import static org.apache.iotdb.tsfile.read.expression.ExpressionType.AND;
+import static org.apache.iotdb.tsfile.read.expression.ExpressionType.OR;
+import static org.apache.iotdb.tsfile.read.expression.ExpressionType.SERIES;
+import static org.apache.iotdb.tsfile.read.expression.ExpressionType.TRUE;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.iotdb.cluster.query.expression.TrueExpression;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.FilterGroupEntity;
+import org.apache.iotdb.cluster.utils.QPExecutorUtils;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.expression.ExpressionType;
+import org.apache.iotdb.tsfile.read.expression.IBinaryExpression;
+import org.apache.iotdb.tsfile.read.expression.IExpression;
+import org.apache.iotdb.tsfile.read.expression.impl.BinaryExpression;
+import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
+
+public class ExpressionUtils {
+
+ private ExpressionUtils() {
+ }
+
+ /**
+ * Get all series path of expression group by group id
+ */
+ public static void getAllExpressionSeries(IExpression expression,
+ Map<String, FilterGroupEntity> filterGroupEntityMap)
+ throws PathErrorException {
+ if (expression.getType() == ExpressionType.SERIES) {
+ Path path = ((SingleSeriesExpression) expression).getSeriesPath();
+ String groupId = QPExecutorUtils.getGroupIdByDevice(path.getDevice());
+ if (!filterGroupEntityMap.containsKey(groupId)) {
+ filterGroupEntityMap.put(groupId, new FilterGroupEntity(groupId));
+ }
+ FilterGroupEntity filterGroupEntity = filterGroupEntityMap.get(groupId);
+ filterGroupEntity.addFilterPaths(path);
+ filterGroupEntity.addFilter(((SingleSeriesExpression) expression).getFilter());
+ } else if (expression.getType() == OR || expression.getType() == AND) {
+ getAllExpressionSeries(((IBinaryExpression) expression).getLeft(), filterGroupEntityMap);
+ getAllExpressionSeries(((IBinaryExpression) expression).getRight(), filterGroupEntityMap);
+ } else {
+ throw new UnSupportedDataTypeException(
+ "Unsupported QueryFilterType when construct OperatorNode: " + expression.getType());
+ }
+ }
+
+ /**
+ * Prune filter true by group id
+ *
+ * @param pathList all paths of a data group
+ */
+ public static IExpression pruneFilterTree(IExpression expression, List<Path> pathList) {
+ if (expression.getType() == SERIES) {
+ if (pathList.contains(((SingleSeriesExpression) expression).getSeriesPath())) {
+ return expression;
+ } else{
+ return new TrueExpression();
+ }
+ } else if(expression.getType() == OR){
+ return pruneOrFilterTree(expression, pathList);
+ } else if(expression.getType() == AND){
+ return pruneAndFilterTree(expression, pathList);
+ } else {
+ throw new UnSupportedDataTypeException(
+ "Unsupported ExpressionType when prune filter tree: " + expression.getType());
+ }
+ }
+
+ /**
+ * Prune or filter tree
+ *
+ * @param expression origin expression
+ * @param pathList all series path of the same data group
+ */
+ private static IExpression pruneOrFilterTree(IExpression expression, List<Path> pathList) {
+ IExpression left = pruneFilterTree(((BinaryExpression) expression).getLeft(), pathList);
+ IExpression right = pruneFilterTree(((BinaryExpression) expression).getRight(), pathList);
+ if (left.getType() == TRUE || right.getType() == TRUE) {
+ return new TrueExpression();
+ } else {
+ ((BinaryExpression) expression).setLeft(left);
+ ((BinaryExpression) expression).setRight(right);
+ return expression;
+ }
+ }
+
+ /**
+ * Prune and filter tree
+ *
+ * @param expression origin expression
+ * @param pathList all series path of the same data group
+ */
+ private static IExpression pruneAndFilterTree(IExpression expression, List<Path> pathList) {
+ IExpression left = pruneFilterTree(((BinaryExpression) expression).getLeft(), pathList);
+ IExpression right = pruneFilterTree(((BinaryExpression) expression).getRight(), pathList);
+ if (left.getType() == TRUE && right.getType() == TRUE) {
+ return new TrueExpression();
+ } else if (left.getType() == TRUE) {
+ return right;
+ } else if (right.getType() == TRUE) {
+ return left;
+ } else {
+ ((BinaryExpression) expression).setLeft(left);
+ ((BinaryExpression) expression).setRight(right);
+ return expression;
+ }
+ }
+
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/query/utils/QueryPlanPartitionUtils.java b/cluster/src/main/java/org/apache/iotdb/cluster/query/utils/QueryPlanPartitionUtils.java
new file mode 100644
index 0000000..4f7a5fe
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/query/utils/QueryPlanPartitionUtils.java
@@ -0,0 +1,127 @@
+/**
+ * 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.cluster.query.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcSingleQueryManager;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.FilterGroupEntity;
+import org.apache.iotdb.cluster.utils.QPExecutorUtils;
+import org.apache.iotdb.cluster.utils.hash.Router;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
+import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.expression.ExpressionType;
+import org.apache.iotdb.tsfile.read.expression.IExpression;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+
+/**
+ * Utils for splitting query plan to several sub query plans by group id.
+ */
+public class QueryPlanPartitionUtils {
+
+ private QueryPlanPartitionUtils() {
+
+ }
+
+ /**
+ * Split query plan with no filter or with only global time filter by group id
+ */
+ public static void splitQueryPlanWithoutValueFilter(ClusterRpcSingleQueryManager singleQueryManager)
+ throws PathErrorException {
+ splitQueryPlanBySelectPath(singleQueryManager);
+ }
+
+ /**
+ * Split query plan by select paths
+ */
+ private static void splitQueryPlanBySelectPath(ClusterRpcSingleQueryManager singleQueryManager)
+ throws PathErrorException {
+ QueryPlan queryPlan = singleQueryManager.getOriginQueryPlan();
+ Map<String, List<Path>> selectSeriesByGroupId = singleQueryManager.getSelectSeriesByGroupId();
+ Map<String, QueryPlan> selectPathPlans = singleQueryManager.getSelectPathPlans();
+ List<Path> selectPaths = queryPlan.getPaths();
+ for (Path path : selectPaths) {
+ String groupId = QPExecutorUtils.getGroupIdByDevice(path.getDevice());
+ if (!selectSeriesByGroupId.containsKey(groupId)) {
+ selectSeriesByGroupId.put(groupId, new ArrayList<>());
+ }
+ selectSeriesByGroupId.get(groupId).add(path);
+ }
+ for (Entry<String, List<Path>> entry : selectSeriesByGroupId.entrySet()) {
+ String groupId = entry.getKey();
+ List<Path> paths = entry.getValue();
+ QueryPlan subQueryPlan = new QueryPlan();
+ subQueryPlan.setProposer(queryPlan.getProposer());
+ subQueryPlan.setPaths(paths);
+ subQueryPlan.setExpression(queryPlan.getExpression());
+ selectPathPlans.put(groupId, subQueryPlan);
+ }
+ }
+
+ /**
+ * Split query plan with filter.
+ */
+ public static void splitQueryPlanWithValueFilter(
+ ClusterRpcSingleQueryManager singleQueryManager) throws PathErrorException {
+ QueryPlan queryPlan = singleQueryManager.getOriginQueryPlan();
+ if (queryPlan instanceof GroupByPlan) {
+ splitGroupByPlan((GroupByPlan) queryPlan, singleQueryManager);
+ } else if (queryPlan instanceof AggregationPlan) {
+ splitAggregationPlan((AggregationPlan) queryPlan, singleQueryManager);
+ } else {
+ splitQueryPlan(queryPlan, singleQueryManager);
+ }
+ }
+
+ private static void splitGroupByPlan(GroupByPlan queryPlan,
+ ClusterRpcSingleQueryManager singleQueryManager) {
+ throw new UnsupportedOperationException();
+ }
+
+ private static void splitAggregationPlan(AggregationPlan aggregationPlan,
+ ClusterRpcSingleQueryManager singleQueryManager) {
+ throw new UnsupportedOperationException();
+ }
+
+ private static void splitQueryPlan(QueryPlan queryPlan,
+ ClusterRpcSingleQueryManager singleQueryManager) throws PathErrorException {
+ splitQueryPlanBySelectPath(singleQueryManager);
+ // split query plan by filter path
+ Map<String, FilterGroupEntity> filterGroupEntityMap = singleQueryManager.getFilterGroupEntityMap();
+ IExpression expression = queryPlan.getExpression();
+ ExpressionUtils.getAllExpressionSeries(expression, filterGroupEntityMap);
+ for(FilterGroupEntity filterGroupEntity: filterGroupEntityMap.values()){
+ List<Path> filterSeriesList = filterGroupEntity.getFilterPaths();
+ // create filter sub query plan
+ QueryPlan subQueryPlan = new QueryPlan();
+ subQueryPlan.setPaths(filterSeriesList);
+ IExpression subExpression = ExpressionUtils
+ .pruneFilterTree(expression.clone(), filterSeriesList);
+ if (subExpression.getType() != ExpressionType.TRUE) {
+ subQueryPlan.setExpression(subExpression);
+ }
+ filterGroupEntity.setQueryPlan(subQueryPlan);
+ }
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/NodeAsClient.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/NodeAsClient.java
index ca5d238..bab1536 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/NodeAsClient.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/NodeAsClient.java
@@ -19,9 +19,10 @@
package org.apache.iotdb.cluster.rpc.raft;
import com.alipay.sofa.jraft.entity.PeerId;
-import org.apache.iotdb.cluster.qp.callback.QPTask;
import org.apache.iotdb.cluster.exception.RaftConnectionException;
+import org.apache.iotdb.cluster.qp.task.SingleQPTask;
import org.apache.iotdb.cluster.rpc.raft.request.BasicRequest;
+import org.apache.iotdb.cluster.qp.task.QueryTask;
/**
* Handle the request and process the result as a client with the current node
@@ -30,21 +31,18 @@ public interface NodeAsClient {
/**
* Asynchronous processing requests
- *
- * @param leader leader node of the target group
- * @param qpTask the QPTask to be executed
+ * @param leader leader node of the target group
+ * @param qpTask single QPTask to be executed
*/
void asyncHandleRequest(BasicRequest request, PeerId leader,
- QPTask qpTask) throws RaftConnectionException;
+ SingleQPTask qpTask) throws RaftConnectionException;
/**
* Synchronous processing requests
+ * @param peerId leader node of the target group
*
- * @param clientService client rpc service handle
- * @param leader leader node of the target group
- * @param qpTask the QPTask to be executed
*/
- void syncHandleRequest(BasicRequest request, PeerId leader, QPTask qpTask)
+ QueryTask syncHandleRequest(BasicRequest request, PeerId peerId)
throws RaftConnectionException;
/**
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/impl/RaftNodeAsClientManager.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/impl/RaftNodeAsClientManager.java
index fc5df44..19f1343 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/impl/RaftNodeAsClientManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/impl/RaftNodeAsClientManager.java
@@ -25,14 +25,16 @@ import com.alipay.sofa.jraft.option.CliOptions;
import com.alipay.sofa.jraft.rpc.impl.cli.BoltCliClientService;
import java.util.LinkedList;
import java.util.concurrent.Executor;
-import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.iotdb.cluster.config.ClusterConfig;
import org.apache.iotdb.cluster.config.ClusterDescriptor;
import org.apache.iotdb.cluster.exception.RaftConnectionException;
-import org.apache.iotdb.cluster.qp.callback.QPTask;
-import org.apache.iotdb.cluster.qp.callback.QPTask.TaskState;
+import org.apache.iotdb.cluster.qp.task.QPTask.TaskState;
+import org.apache.iotdb.cluster.qp.task.QueryTask;
+import org.apache.iotdb.cluster.qp.task.SingleQPTask;
import org.apache.iotdb.cluster.rpc.raft.NodeAsClient;
import org.apache.iotdb.cluster.rpc.raft.request.BasicRequest;
import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
@@ -53,8 +55,8 @@ public class RaftNodeAsClientManager {
private static final int TASK_TIMEOUT_MS = CLUSTER_CONFIG.getQpTaskTimeout();
/**
- * Max valid number of @NodeAsClient usage, represent the number can run simultaneously at the
- * same time
+ * Max valid number of @NodeAsClient usage, represent the number can run simultaneously
+ * at the same time
*/
private static final int MAX_VALID_CLIENT_NUM = CLUSTER_CONFIG.getMaxNumOfInnerRpcClient();
@@ -81,17 +83,17 @@ public class RaftNodeAsClientManager {
/**
* Lock to update clientNumInUse
*/
- private ReentrantLock resourceLock = new ReentrantLock();
+ private Lock resourceLock = new ReentrantLock();
/**
- * Mark whether system is shutting down
+ * Condition to get client
*/
- private volatile boolean isShuttingDown;
+ private Condition resourceCondition = resourceLock.newCondition();
/**
- * Interval of thread sleep, unit is millisecond.
+ * Mark whether system is shutting down
*/
- private static final int THREAD_SLEEP_INTERVAL = 10;
+ private volatile boolean isShuttingDown;
private RaftNodeAsClientManager() {
@@ -105,59 +107,38 @@ public class RaftNodeAsClientManager {
* Try to get clientList, return null if num of queue clientList exceeds threshold.
*/
public RaftNodeAsClient getRaftNodeAsClient() throws RaftConnectionException {
+ resourceLock.lock();
try {
- resourceLock.lock();
if (queueClientNum >= MAX_QUEUE_CLIENT_NUM) {
throw new RaftConnectionException(String
.format("Raft inner rpc clients have reached the max numbers %s",
CLUSTER_CONFIG.getMaxNumOfInnerRpcClient() + CLUSTER_CONFIG
.getMaxQueueNumOfInnerRpcClient()));
}
- checkShuttingDown();
- if (clientNumInUse.get() < MAX_VALID_CLIENT_NUM) {
- clientNumInUse.incrementAndGet();
- return getClient();
- }
queueClientNum++;
- } finally {
- resourceLock.unlock();
- }
- return tryToGetClient();
- }
-
- private void checkShuttingDown() throws RaftConnectionException {
- if (isShuttingDown) {
- throw new RaftConnectionException(
- "Reject to provide RaftNodeAsClient client because cluster system is shutting down");
- }
- }
-
- /**
- * Check whether it can get the clientList
- */
- private RaftNodeAsClient tryToGetClient() throws RaftConnectionException {
- for (; ; ) {
- if (clientNumInUse.get() < MAX_VALID_CLIENT_NUM) {
- resourceLock.lock();
- try {
+ try {
+ while (true) {
checkShuttingDown();
if (clientNumInUse.get() < MAX_VALID_CLIENT_NUM) {
clientNumInUse.incrementAndGet();
- queueClientNum--;
return getClient();
}
- } catch (RaftConnectionException e) {
- queueClientNum--;
- throw new RaftConnectionException(e);
- } finally {
- resourceLock.unlock();
+ resourceCondition.await();
}
- }
- try {
- Thread.sleep(THREAD_SLEEP_INTERVAL);
} catch (InterruptedException e) {
throw new RaftConnectionException("An error occurred when trying to get NodeAsClient", e);
+ } finally {
+ queueClientNum--;
}
+ } finally {
+ resourceLock.unlock();
+ }
+ }
+
+ private void checkShuttingDown() throws RaftConnectionException {
+ if (isShuttingDown) {
+ throw new RaftConnectionException(
+ "Reject to provide RaftNodeAsClient client because cluster system is shutting down");
}
}
@@ -179,6 +160,7 @@ public class RaftNodeAsClientManager {
resourceLock.lock();
try {
clientNumInUse.decrementAndGet();
+ resourceCondition.signalAll();
clientList.addLast(client);
} finally {
resourceLock.unlock();
@@ -189,7 +171,7 @@ public class RaftNodeAsClientManager {
isShuttingDown = true;
while (clientNumInUse.get() != 0 && queueClientNum != 0) {
// wait until releasing all usage of clients.
- Thread.sleep(THREAD_SLEEP_INTERVAL);
+ resourceCondition.await();
}
while (!clientList.isEmpty()) {
clientList.removeFirst().shutdown();
@@ -246,7 +228,7 @@ public class RaftNodeAsClientManager {
@Override
public void asyncHandleRequest(BasicRequest request, PeerId leader,
- QPTask qpTask)
+ SingleQPTask qpTask)
throws RaftConnectionException {
LOGGER.debug("Node as client to send request to leader: {}", leader);
try {
@@ -284,17 +266,13 @@ public class RaftNodeAsClientManager {
}
@Override
- public void syncHandleRequest(BasicRequest request, PeerId leader,
- QPTask qpTask)
- throws RaftConnectionException {
+ public QueryTask syncHandleRequest(BasicRequest request, PeerId peerId) {
try {
BasicResponse response = (BasicResponse) boltClientService.getRpcClient()
- .invokeSync(leader.getEndpoint().toString(), request, TASK_TIMEOUT_MS);
- qpTask.run(response);
+ .invokeSync(peerId.getEndpoint().toString(), request, TASK_TIMEOUT_MS);
+ return new QueryTask(response, TaskState.FINISH);
} catch (RemotingException | InterruptedException e) {
- qpTask.setTaskState(TaskState.EXCEPTION);
- qpTask.run(null);
- throw new RaftConnectionException(e);
+ return new QueryTask(null, TaskState.EXCEPTION);
} finally {
releaseClient(RaftNodeAsClient.this);
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/DataGroupNonQueryAsyncProcessor.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/nonquery/DataGroupNonQueryAsyncProcessor.java
similarity index 90%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/DataGroupNonQueryAsyncProcessor.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/nonquery/DataGroupNonQueryAsyncProcessor.java
index fb00c0d..de2d2ab 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/DataGroupNonQueryAsyncProcessor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/nonquery/DataGroupNonQueryAsyncProcessor.java
@@ -16,16 +16,17 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.processor;
+package org.apache.iotdb.cluster.rpc.raft.processor.nonquery;
import com.alipay.remoting.AsyncContext;
import com.alipay.remoting.BizContext;
import com.alipay.sofa.jraft.entity.PeerId;
import org.apache.iotdb.cluster.entity.raft.DataPartitionRaftHolder;
import org.apache.iotdb.cluster.entity.raft.RaftService;
-import org.apache.iotdb.cluster.rpc.raft.request.DataGroupNonQueryRequest;
+import org.apache.iotdb.cluster.rpc.raft.processor.BasicAsyncUserProcessor;
+import org.apache.iotdb.cluster.rpc.raft.request.nonquery.DataGroupNonQueryRequest;
import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
-import org.apache.iotdb.cluster.rpc.raft.response.DataGroupNonQueryResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.nonquery.DataGroupNonQueryResponse;
import org.apache.iotdb.cluster.utils.RaftUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/MetaGroupNonQueryAsyncProcessor.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/nonquery/MetaGroupNonQueryAsyncProcessor.java
similarity index 89%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/MetaGroupNonQueryAsyncProcessor.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/nonquery/MetaGroupNonQueryAsyncProcessor.java
index d6f6270..9f09bbb 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/MetaGroupNonQueryAsyncProcessor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/nonquery/MetaGroupNonQueryAsyncProcessor.java
@@ -16,16 +16,17 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.processor;
+package org.apache.iotdb.cluster.rpc.raft.processor.nonquery;
import com.alipay.remoting.AsyncContext;
import com.alipay.remoting.BizContext;
import com.alipay.sofa.jraft.entity.PeerId;
import org.apache.iotdb.cluster.entity.raft.MetadataRaftHolder;
import org.apache.iotdb.cluster.entity.raft.RaftService;
-import org.apache.iotdb.cluster.rpc.raft.request.MetaGroupNonQueryRequest;
+import org.apache.iotdb.cluster.rpc.raft.processor.BasicAsyncUserProcessor;
+import org.apache.iotdb.cluster.rpc.raft.request.nonquery.MetaGroupNonQueryRequest;
import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
-import org.apache.iotdb.cluster.rpc.raft.response.MetaGroupNonQueryResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.nonquery.MetaGroupNonQueryResponse;
import org.apache.iotdb.cluster.utils.RaftUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querydata/CloseSeriesReaderSyncProcessor.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querydata/CloseSeriesReaderSyncProcessor.java
new file mode 100644
index 0000000..f6ec67a
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querydata/CloseSeriesReaderSyncProcessor.java
@@ -0,0 +1,43 @@
+/**
+ * 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.cluster.rpc.raft.processor.querydata;
+
+import com.alipay.remoting.BizContext;
+import org.apache.iotdb.cluster.query.manager.querynode.ClusterLocalQueryManager;
+import org.apache.iotdb.cluster.rpc.raft.processor.BasicSyncUserProcessor;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.CloseSeriesReaderRequest;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.QuerySeriesDataResponse;
+
+public class CloseSeriesReaderSyncProcessor extends
+ BasicSyncUserProcessor<CloseSeriesReaderRequest> {
+
+ @Override
+ public Object handleRequest(BizContext bizContext, CloseSeriesReaderRequest request)
+ throws Exception {
+ String groupId = request.getGroupID();
+ QuerySeriesDataResponse response = new QuerySeriesDataResponse(groupId);
+ ClusterLocalQueryManager.getInstance().close(request.getTaskId());
+ return response;
+ }
+
+ @Override
+ public String interest() {
+ return CloseSeriesReaderRequest.class.getName();
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querydata/InitSeriesReaderSyncProcessor.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querydata/InitSeriesReaderSyncProcessor.java
new file mode 100644
index 0000000..894d9eb
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querydata/InitSeriesReaderSyncProcessor.java
@@ -0,0 +1,64 @@
+/**
+ * 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.cluster.rpc.raft.processor.querydata;
+
+import com.alipay.remoting.BizContext;
+import com.alipay.sofa.jraft.Status;
+import org.apache.iotdb.cluster.config.ClusterConstant;
+import org.apache.iotdb.cluster.query.manager.querynode.ClusterLocalQueryManager;
+import org.apache.iotdb.cluster.rpc.raft.processor.BasicSyncUserProcessor;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.InitSeriesReaderRequest;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.InitSeriesReaderResponse;
+import org.apache.iotdb.cluster.utils.QPExecutorUtils;
+import org.apache.iotdb.cluster.utils.RaftUtils;
+import org.apache.iotdb.db.exception.ProcessorException;
+
+public class InitSeriesReaderSyncProcessor extends BasicSyncUserProcessor<InitSeriesReaderRequest> {
+
+ @Override
+ public Object handleRequest(BizContext bizContext, InitSeriesReaderRequest request)
+ throws Exception {
+ String groupId = request.getGroupID();
+ handleNullRead(request.getReadConsistencyLevel(), groupId);
+ return ClusterLocalQueryManager.getInstance().createQueryDataSet(request);
+ }
+
+ /**
+ * It's necessary to do null read while creating query data set with a strong consistency level
+ * and local node is not the leader of data group
+ *
+ * @param readConsistencyLevel read concistency level
+ * @param groupId group id
+ */
+ private void handleNullRead(int readConsistencyLevel, String groupId) throws ProcessorException {
+ if (readConsistencyLevel == ClusterConstant.STRONG_CONSISTENCY_LEVEL && !QPExecutorUtils
+ .checkDataGroupLeader(groupId)) {
+ Status nullReadTaskStatus = Status.OK();
+ RaftUtils.handleNullReadToDataGroup(nullReadTaskStatus, groupId);
+ if (!nullReadTaskStatus.isOk()) {
+ throw new ProcessorException("Null read to data group failed");
+ }
+ }
+ }
+
+ @Override
+ public String interest() {
+ return InitSeriesReaderRequest.class.getName();
+ }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/GlobalTimeExpression.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querydata/QuerySeriesDataByTimestampSyncProcessor.java
similarity index 52%
copy from tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/GlobalTimeExpression.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querydata/QuerySeriesDataByTimestampSyncProcessor.java
index d69a65a..ae3f057 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/GlobalTimeExpression.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querydata/QuerySeriesDataByTimestampSyncProcessor.java
@@ -16,37 +16,23 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.tsfile.read.expression.impl;
+package org.apache.iotdb.cluster.rpc.raft.processor.querydata;
-import org.apache.iotdb.tsfile.read.expression.ExpressionType;
-import org.apache.iotdb.tsfile.read.expression.IUnaryExpression;
-import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import com.alipay.remoting.BizContext;
+import org.apache.iotdb.cluster.query.manager.querynode.ClusterLocalQueryManager;
+import org.apache.iotdb.cluster.rpc.raft.processor.BasicSyncUserProcessor;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.QuerySeriesDataByTimestampRequest;
-public class GlobalTimeExpression implements IUnaryExpression {
-
- private Filter filter;
-
- public GlobalTimeExpression(Filter filter) {
- this.filter = filter;
- }
-
- @Override
- public Filter getFilter() {
- return filter;
- }
-
- @Override
- public void setFilter(Filter filter) {
- this.filter = filter;
- }
+public class QuerySeriesDataByTimestampSyncProcessor extends BasicSyncUserProcessor<QuerySeriesDataByTimestampRequest> {
@Override
- public ExpressionType getType() {
- return ExpressionType.GLOBAL_TIME;
+ public Object handleRequest(BizContext bizContext,
+ QuerySeriesDataByTimestampRequest request) throws Exception {
+ return ClusterLocalQueryManager.getInstance().readBatchDataByTimestamp(request);
}
@Override
- public String toString() {
- return "[" + this.filter.toString() + "]";
+ public String interest() {
+ return QuerySeriesDataByTimestampRequest.class.getName();
}
}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/GlobalTimeExpression.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querydata/QuerySeriesDataSyncProcessor.java
similarity index 51%
copy from tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/GlobalTimeExpression.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querydata/QuerySeriesDataSyncProcessor.java
index d69a65a..90dc24a 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/expression/impl/GlobalTimeExpression.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querydata/QuerySeriesDataSyncProcessor.java
@@ -16,37 +16,25 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.tsfile.read.expression.impl;
+package org.apache.iotdb.cluster.rpc.raft.processor.querydata;
-import org.apache.iotdb.tsfile.read.expression.ExpressionType;
-import org.apache.iotdb.tsfile.read.expression.IUnaryExpression;
-import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import com.alipay.remoting.BizContext;
+import org.apache.iotdb.cluster.query.manager.querynode.ClusterLocalQueryManager;
+import org.apache.iotdb.cluster.rpc.raft.processor.BasicSyncUserProcessor;
+import org.apache.iotdb.cluster.rpc.raft.request.querydata.QuerySeriesDataRequest;
+import org.apache.iotdb.cluster.rpc.raft.response.querydata.QuerySeriesDataResponse;
-public class GlobalTimeExpression implements IUnaryExpression {
-
- private Filter filter;
-
- public GlobalTimeExpression(Filter filter) {
- this.filter = filter;
- }
-
- @Override
- public Filter getFilter() {
- return filter;
- }
-
- @Override
- public void setFilter(Filter filter) {
- this.filter = filter;
- }
+public class QuerySeriesDataSyncProcessor extends
+ BasicSyncUserProcessor<QuerySeriesDataRequest> {
@Override
- public ExpressionType getType() {
- return ExpressionType.GLOBAL_TIME;
+ public Object handleRequest(BizContext bizContext, QuerySeriesDataRequest request)
+ throws Exception {
+ return ClusterLocalQueryManager.getInstance().readBatchData(request);
}
@Override
- public String toString() {
- return "[" + this.filter.toString() + "]";
+ public String interest() {
+ return QuerySeriesDataRequest.class.getName();
}
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/QueryMetadataAsyncProcessor.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querymetadata/QueryMetadataAsyncProcessor.java
similarity index 91%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/QueryMetadataAsyncProcessor.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querymetadata/QueryMetadataAsyncProcessor.java
index 176fa33..36e657c 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/QueryMetadataAsyncProcessor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querymetadata/QueryMetadataAsyncProcessor.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.processor;
+package org.apache.iotdb.cluster.rpc.raft.processor.querymetadata;
import com.alipay.remoting.AsyncContext;
import com.alipay.remoting.BizContext;
@@ -25,8 +25,9 @@ import com.alipay.sofa.jraft.closure.ReadIndexClosure;
import org.apache.iotdb.cluster.config.ClusterConstant;
import org.apache.iotdb.cluster.entity.raft.DataPartitionRaftHolder;
import org.apache.iotdb.cluster.entity.raft.RaftService;
-import org.apache.iotdb.cluster.rpc.raft.request.QueryMetadataRequest;
-import org.apache.iotdb.cluster.rpc.raft.response.QueryMetadataResponse;
+import org.apache.iotdb.cluster.rpc.raft.processor.BasicAsyncUserProcessor;
+import org.apache.iotdb.cluster.rpc.raft.request.querymetadata.QueryMetadataRequest;
+import org.apache.iotdb.cluster.rpc.raft.response.querymetadata.QueryMetadataResponse;
import org.apache.iotdb.cluster.utils.RaftUtils;
import org.apache.iotdb.db.exception.PathErrorException;
import org.apache.iotdb.db.metadata.MManager;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/QueryMetadataInStringAsyncProcessor.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querymetadata/QueryMetadataInStringAsyncProcessor.java
similarity index 90%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/QueryMetadataInStringAsyncProcessor.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querymetadata/QueryMetadataInStringAsyncProcessor.java
index b80f4ae..8771eea 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/QueryMetadataInStringAsyncProcessor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querymetadata/QueryMetadataInStringAsyncProcessor.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.processor;
+package org.apache.iotdb.cluster.rpc.raft.processor.querymetadata;
import com.alipay.remoting.AsyncContext;
import com.alipay.remoting.BizContext;
@@ -25,8 +25,9 @@ import com.alipay.sofa.jraft.closure.ReadIndexClosure;
import org.apache.iotdb.cluster.config.ClusterConstant;
import org.apache.iotdb.cluster.entity.raft.DataPartitionRaftHolder;
import org.apache.iotdb.cluster.entity.raft.RaftService;
-import org.apache.iotdb.cluster.rpc.raft.request.QueryMetadataInStringRequest;
-import org.apache.iotdb.cluster.rpc.raft.response.QueryMetadataInStringResponse;
+import org.apache.iotdb.cluster.rpc.raft.processor.BasicAsyncUserProcessor;
+import org.apache.iotdb.cluster.rpc.raft.request.querymetadata.QueryMetadataInStringRequest;
+import org.apache.iotdb.cluster.rpc.raft.response.querymetadata.QueryMetadataInStringResponse;
import org.apache.iotdb.cluster.utils.RaftUtils;
import org.apache.iotdb.db.metadata.MManager;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/QueryPathsAsyncProcessor.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querymetadata/QueryPathsAsyncProcessor.java
similarity index 92%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/QueryPathsAsyncProcessor.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querymetadata/QueryPathsAsyncProcessor.java
index f54aba0..8e1e47b 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/QueryPathsAsyncProcessor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querymetadata/QueryPathsAsyncProcessor.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.processor;
+package org.apache.iotdb.cluster.rpc.raft.processor.querymetadata;
import com.alipay.remoting.AsyncContext;
import com.alipay.remoting.BizContext;
@@ -25,8 +25,9 @@ import com.alipay.sofa.jraft.closure.ReadIndexClosure;
import org.apache.iotdb.cluster.config.ClusterConstant;
import org.apache.iotdb.cluster.entity.raft.DataPartitionRaftHolder;
import org.apache.iotdb.cluster.entity.raft.RaftService;
-import org.apache.iotdb.cluster.rpc.raft.request.QueryPathsRequest;
-import org.apache.iotdb.cluster.rpc.raft.response.QueryPathsResponse;
+import org.apache.iotdb.cluster.rpc.raft.processor.BasicAsyncUserProcessor;
+import org.apache.iotdb.cluster.rpc.raft.request.querymetadata.QueryPathsRequest;
+import org.apache.iotdb.cluster.rpc.raft.response.querymetadata.QueryPathsResponse;
import org.apache.iotdb.cluster.utils.RaftUtils;
import org.apache.iotdb.db.exception.PathErrorException;
import org.apache.iotdb.db.metadata.MManager;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/QuerySeriesTypeAsyncProcessor.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querymetadata/QuerySeriesTypeAsyncProcessor.java
similarity index 91%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/QuerySeriesTypeAsyncProcessor.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querymetadata/QuerySeriesTypeAsyncProcessor.java
index f0a4fc6..9e4b1c7 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/QuerySeriesTypeAsyncProcessor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querymetadata/QuerySeriesTypeAsyncProcessor.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.processor;
+package org.apache.iotdb.cluster.rpc.raft.processor.querymetadata;
import com.alipay.remoting.AsyncContext;
import com.alipay.remoting.BizContext;
@@ -25,8 +25,9 @@ import com.alipay.sofa.jraft.closure.ReadIndexClosure;
import org.apache.iotdb.cluster.config.ClusterConstant;
import org.apache.iotdb.cluster.entity.raft.DataPartitionRaftHolder;
import org.apache.iotdb.cluster.entity.raft.RaftService;
-import org.apache.iotdb.cluster.rpc.raft.request.QuerySeriesTypeRequest;
-import org.apache.iotdb.cluster.rpc.raft.response.QuerySeriesTypeResponse;
+import org.apache.iotdb.cluster.rpc.raft.processor.BasicAsyncUserProcessor;
+import org.apache.iotdb.cluster.rpc.raft.request.querymetadata.QuerySeriesTypeRequest;
+import org.apache.iotdb.cluster.rpc.raft.response.querymetadata.QuerySeriesTypeResponse;
import org.apache.iotdb.cluster.utils.RaftUtils;
import org.apache.iotdb.db.exception.PathErrorException;
import org.apache.iotdb.db.metadata.MManager;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/QueryTimeSeriesAsyncProcessor.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querymetadata/QueryTimeSeriesAsyncProcessor.java
similarity index 92%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/QueryTimeSeriesAsyncProcessor.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querymetadata/QueryTimeSeriesAsyncProcessor.java
index c41fdcf..593f99d 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/QueryTimeSeriesAsyncProcessor.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/processor/querymetadata/QueryTimeSeriesAsyncProcessor.java
@@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.processor;
+package org.apache.iotdb.cluster.rpc.raft.processor.querymetadata;
import com.alipay.remoting.AsyncContext;
import com.alipay.remoting.BizContext;
@@ -25,8 +25,9 @@ import com.alipay.sofa.jraft.closure.ReadIndexClosure;
import org.apache.iotdb.cluster.config.ClusterConstant;
import org.apache.iotdb.cluster.entity.raft.DataPartitionRaftHolder;
import org.apache.iotdb.cluster.entity.raft.RaftService;
-import org.apache.iotdb.cluster.rpc.raft.request.QueryTimeSeriesRequest;
-import org.apache.iotdb.cluster.rpc.raft.response.QueryTimeSeriesResponse;
+import org.apache.iotdb.cluster.rpc.raft.processor.BasicAsyncUserProcessor;
+import org.apache.iotdb.cluster.rpc.raft.request.querymetadata.QueryTimeSeriesRequest;
+import org.apache.iotdb.cluster.rpc.raft.response.querymetadata.QueryTimeSeriesResponse;
import org.apache.iotdb.cluster.utils.RaftUtils;
import org.apache.iotdb.db.exception.PathErrorException;
import org.apache.iotdb.db.metadata.MManager;
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicNonQueryRequest.java
similarity index 76%
copy from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicRequest.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicNonQueryRequest.java
index ee4d840..dc15158 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicNonQueryRequest.java
@@ -23,22 +23,20 @@ import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.writelog.transfer.PhysicalPlanLogTransfer;
+import org.apache.iotdb.db.qp.physical.transfer.PhysicalPlanLogTransfer;
-public abstract class BasicRequest implements Serializable {
-
- private static final long serialVersionUID = 8434915845259380829L;
-
- /**
- * Group ID
- */
- private String groupID;
+public abstract class BasicNonQueryRequest extends BasicRequest{
+ private static final long serialVersionUID = -3082772186451384202L;
/**
* Serialized physical plans
*/
private List<byte[]> physicalPlanBytes;
+ public BasicNonQueryRequest(String groupID) {
+ super(groupID);
+ }
+
protected void init(List<PhysicalPlan> physicalPlanBytes) throws IOException {
this.physicalPlanBytes = new ArrayList<>(physicalPlanBytes.size());
for (PhysicalPlan plan : physicalPlanBytes) {
@@ -50,15 +48,4 @@ public abstract class BasicRequest implements Serializable {
return physicalPlanBytes;
}
- public BasicRequest(String groupID) {
- this.groupID = groupID;
- }
-
- public String getGroupID() {
- return groupID;
- }
-
- public void setGroupID(String groupID) {
- this.groupID = groupID;
- }
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicQueryRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicQueryRequest.java
index 2cf613f..3ceddaf 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicQueryRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicQueryRequest.java
@@ -20,6 +20,7 @@ package org.apache.iotdb.cluster.rpc.raft.request;
public abstract class BasicQueryRequest extends BasicRequest {
+ private static final long serialVersionUID = 2993000692822502110L;
/**
* Read Consistency Level
*/
@@ -30,6 +31,10 @@ public abstract class BasicQueryRequest extends BasicRequest {
this.readConsistencyLevel = readConsistencyLevel;
}
+ public BasicQueryRequest(String groupID) {
+ super(groupID);
+ }
+
public int getReadConsistencyLevel() {
return readConsistencyLevel;
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicRequest.java
index ee4d840..dd4758a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicRequest.java
@@ -18,12 +18,7 @@
*/
package org.apache.iotdb.cluster.rpc.raft.request;
-import java.io.IOException;
import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.iotdb.db.qp.physical.PhysicalPlan;
-import org.apache.iotdb.db.writelog.transfer.PhysicalPlanLogTransfer;
public abstract class BasicRequest implements Serializable {
@@ -34,22 +29,6 @@ public abstract class BasicRequest implements Serializable {
*/
private String groupID;
- /**
- * Serialized physical plans
- */
- private List<byte[]> physicalPlanBytes;
-
- protected void init(List<PhysicalPlan> physicalPlanBytes) throws IOException {
- this.physicalPlanBytes = new ArrayList<>(physicalPlanBytes.size());
- for (PhysicalPlan plan : physicalPlanBytes) {
- this.physicalPlanBytes.add(PhysicalPlanLogTransfer.operatorToLog(plan));
- }
- }
-
- public List<byte[]> getPhysicalPlanBytes() {
- return physicalPlanBytes;
- }
-
public BasicRequest(String groupID) {
this.groupID = groupID;
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/DataGroupNonQueryRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/nonquery/DataGroupNonQueryRequest.java
similarity index 80%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/DataGroupNonQueryRequest.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/nonquery/DataGroupNonQueryRequest.java
index c1bcf5f..8413373 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/DataGroupNonQueryRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/nonquery/DataGroupNonQueryRequest.java
@@ -16,18 +16,19 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.request;
+package org.apache.iotdb.cluster.rpc.raft.request.nonquery;
import java.io.IOException;
-import java.io.Serializable;
import java.util.List;
+import org.apache.iotdb.cluster.rpc.raft.request.BasicNonQueryRequest;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
/**
* Handle request to data group
*/
-public class DataGroupNonQueryRequest extends BasicRequest implements Serializable {
+public class DataGroupNonQueryRequest extends BasicNonQueryRequest {
+ private static final long serialVersionUID = -2442407985738324604L;
public DataGroupNonQueryRequest(String groupID, List<PhysicalPlan> physicalPlanBytes)
throws IOException {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/MetaGroupNonQueryRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/nonquery/MetaGroupNonQueryRequest.java
similarity index 80%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/MetaGroupNonQueryRequest.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/nonquery/MetaGroupNonQueryRequest.java
index 69625ff..b29609a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/MetaGroupNonQueryRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/nonquery/MetaGroupNonQueryRequest.java
@@ -16,17 +16,19 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.request;
+package org.apache.iotdb.cluster.rpc.raft.request.nonquery;
import java.io.IOException;
-import java.io.Serializable;
import java.util.List;
+import org.apache.iotdb.cluster.rpc.raft.request.BasicNonQueryRequest;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
/**
* Handle request to metadata group leader
*/
-public class MetaGroupNonQueryRequest extends BasicRequest implements Serializable {
+public class MetaGroupNonQueryRequest extends BasicNonQueryRequest {
+
+ private static final long serialVersionUID = 312899249719243646L;
public MetaGroupNonQueryRequest(String groupID, List<PhysicalPlan> plans)
throws IOException {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicQueryRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querydata/CloseSeriesReaderRequest.java
similarity index 54%
copy from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicQueryRequest.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querydata/CloseSeriesReaderRequest.java
index 2cf613f..1a3cf3c 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicQueryRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querydata/CloseSeriesReaderRequest.java
@@ -16,25 +16,32 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.request;
+package org.apache.iotdb.cluster.rpc.raft.request.querydata;
-public abstract class BasicQueryRequest extends BasicRequest {
+import org.apache.iotdb.cluster.rpc.raft.request.BasicQueryRequest;
+
+/**
+ * Release series reader resource in remote query node
+ */
+public class CloseSeriesReaderRequest extends BasicQueryRequest {
+
+ private static final long serialVersionUID = 1369515842480836991L;
/**
- * Read Consistency Level
+ * Unique task id which is assigned in coordinator node
*/
- private int readConsistencyLevel;
+ private String taskId;
- public BasicQueryRequest(String groupID, int readConsistencyLevel) {
+ private CloseSeriesReaderRequest(String groupID, String taskId) {
super(groupID);
- this.readConsistencyLevel = readConsistencyLevel;
+ this.taskId = taskId;
}
- public int getReadConsistencyLevel() {
- return readConsistencyLevel;
+ public static CloseSeriesReaderRequest createReleaseResourceRequest(String groupId, String taskId) {
+ return new CloseSeriesReaderRequest(groupId, taskId);
}
- public void setReadConsistencyLevel(int readConsistencyLevel) {
- this.readConsistencyLevel = readConsistencyLevel;
+ public String getTaskId() {
+ return taskId;
}
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querydata/InitSeriesReaderRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querydata/InitSeriesReaderRequest.java
new file mode 100644
index 0000000..c974e2f
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querydata/InitSeriesReaderRequest.java
@@ -0,0 +1,91 @@
+/**
+ * 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.cluster.rpc.raft.request.querydata;
+
+import java.util.ArrayList;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.iotdb.cluster.query.PathType;
+import org.apache.iotdb.cluster.rpc.raft.request.BasicQueryRequest;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+
+/**
+ * Initially create corresponding series readers in remote query node
+ */
+public class InitSeriesReaderRequest extends BasicQueryRequest {
+
+ private static final long serialVersionUID = 8374330837710097285L;
+
+ /**
+ * Unique task id which is assigned in coordinator node
+ */
+ private String taskId;
+
+ /**
+ * Key is series type, value is query plan
+ */
+ private Map<PathType, QueryPlan> allQueryPlan = new EnumMap<>(PathType.class);
+
+ /**
+ * Represent all filter of leaf node in filter tree while executing a query with value filter.
+ */
+ private List<Filter> filterList = new ArrayList<>();
+
+
+ private InitSeriesReaderRequest(String groupID, String taskId) {
+ super(groupID);
+ this.taskId = taskId;
+ }
+
+ public static InitSeriesReaderRequest createInitialQueryRequest(String groupId, String taskId, int readConsistencyLevel,
+ Map<PathType, QueryPlan> allQueryPlan, List<Filter> filterList){
+ InitSeriesReaderRequest request = new InitSeriesReaderRequest(groupId, taskId);
+ request.setReadConsistencyLevel(readConsistencyLevel);
+ request.allQueryPlan = allQueryPlan;
+ request.filterList = filterList;
+ return request;
+ }
+
+ public String getTaskId() {
+ return taskId;
+ }
+
+ public void setTaskId(String taskId) {
+ this.taskId = taskId;
+ }
+
+ public Map<PathType, QueryPlan> getAllQueryPlan() {
+ return allQueryPlan;
+ }
+
+ public void setAllQueryPlan(
+ Map<PathType, QueryPlan> allQueryPlan) {
+ this.allQueryPlan = allQueryPlan;
+ }
+
+ public List<Filter> getFilterList() {
+ return filterList;
+ }
+
+ public void setFilterList(List<Filter> filterList) {
+ this.filterList = filterList;
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querydata/QuerySeriesDataByTimestampRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querydata/QuerySeriesDataByTimestampRequest.java
new file mode 100644
index 0000000..351e6eb
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querydata/QuerySeriesDataByTimestampRequest.java
@@ -0,0 +1,91 @@
+/**
+ * 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.cluster.rpc.raft.request.querydata;
+
+import java.util.List;
+import org.apache.iotdb.cluster.rpc.raft.request.BasicQueryRequest;
+
+public class QuerySeriesDataByTimestampRequest extends BasicQueryRequest {
+
+ private static final long serialVersionUID = 4942493162179531133L;
+ /**
+ * Rounds number of query
+ */
+ private long queryRounds;
+
+ /**
+ * Unique task id which is assigned in coordinator node
+ */
+ private String taskId;
+
+ /**
+ * Batch valid timestamp
+ */
+ private List<Long> batchTimestamp;
+
+ /**
+ * Series to fetch data from remote query node
+ */
+ private List<String> fetchDataSeries;
+
+ private QuerySeriesDataByTimestampRequest(String groupID) {
+ super(groupID);
+ }
+
+ public static QuerySeriesDataByTimestampRequest createRequest(String groupId, long queryRounds, String taskId, List<Long> batchTimestamp, List<String> fetchDataSeries){
+ QuerySeriesDataByTimestampRequest request = new QuerySeriesDataByTimestampRequest(groupId);
+ request.queryRounds = queryRounds;
+ request.taskId = taskId;
+ request.batchTimestamp = batchTimestamp;
+ request.fetchDataSeries = fetchDataSeries;
+ return request;
+ }
+
+ public long getQueryRounds() {
+ return queryRounds;
+ }
+
+ public void setQueryRounds(long queryRounds) {
+ this.queryRounds = queryRounds;
+ }
+
+ public String getTaskId() {
+ return taskId;
+ }
+
+ public void setTaskId(String taskId) {
+ this.taskId = taskId;
+ }
+
+ public List<Long> getBatchTimestamp() {
+ return batchTimestamp;
+ }
+
+ public void setBatchTimestamp(List<Long> batchTimestamp) {
+ this.batchTimestamp = batchTimestamp;
+ }
+
+ public List<String> getFetchDataSeries() {
+ return fetchDataSeries;
+ }
+
+ public void setFetchDataSeries(List<String> fetchDataSeries) {
+ this.fetchDataSeries = fetchDataSeries;
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querydata/QuerySeriesDataRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querydata/QuerySeriesDataRequest.java
new file mode 100644
index 0000000..554b8c1
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querydata/QuerySeriesDataRequest.java
@@ -0,0 +1,98 @@
+/**
+ * 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.cluster.rpc.raft.request.querydata;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.cluster.query.PathType;
+import org.apache.iotdb.cluster.rpc.raft.request.BasicQueryRequest;
+
+/**
+ * Read batch data from series reader from remote query node.
+ */
+public class QuerySeriesDataRequest extends BasicQueryRequest {
+
+ private static final long serialVersionUID = 7132891920951977625L;
+
+ /**
+ * Rounds number of query
+ */
+ private long queryRounds;
+
+ /**
+ * Unique task id which is assigned in coordinator node
+ */
+ private String taskId;
+
+ /**
+ * Series type
+ */
+ private PathType pathType;
+
+ /**
+ * Key is series type, value is series list
+ */
+ private List<String> seriesPaths = new ArrayList<>();
+
+ private QuerySeriesDataRequest(String groupID, String taskId) {
+ super(groupID);
+ this.taskId = taskId;
+ }
+
+ public static QuerySeriesDataRequest createFetchDataRequest(String groupId, String taskId,
+ PathType pathType, List<String> seriesPaths, long queryRounds) {
+ QuerySeriesDataRequest request = new QuerySeriesDataRequest(groupId, taskId);
+ request.pathType = pathType;
+ request.seriesPaths = seriesPaths;
+ request.queryRounds = queryRounds;
+ return request;
+ }
+
+ public long getQueryRounds() {
+ return queryRounds;
+ }
+
+ public void setQueryRounds(long queryRounds) {
+ this.queryRounds = queryRounds;
+ }
+
+ public String getTaskId() {
+ return taskId;
+ }
+
+ public void setTaskId(String taskId) {
+ this.taskId = taskId;
+ }
+
+ public PathType getPathType() {
+ return pathType;
+ }
+
+ public void setPathType(PathType pathType) {
+ this.pathType = pathType;
+ }
+
+ public List<String> getSeriesPaths() {
+ return seriesPaths;
+ }
+
+ public void setSeriesPaths(List<String> seriesPaths) {
+ this.seriesPaths = seriesPaths;
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QueryMetadataInStringRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QueryMetadataInStringRequest.java
similarity index 82%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QueryMetadataInStringRequest.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QueryMetadataInStringRequest.java
index 18471a6..c90cf80 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QueryMetadataInStringRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QueryMetadataInStringRequest.java
@@ -16,11 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.request;
+package org.apache.iotdb.cluster.rpc.raft.request.querymetadata;
-import java.io.Serializable;
+import org.apache.iotdb.cluster.rpc.raft.request.BasicQueryRequest;
-public class QueryMetadataInStringRequest extends BasicQueryRequest implements Serializable {
+public class QueryMetadataInStringRequest extends BasicQueryRequest {
+
+ private static final long serialVersionUID = -7037884610669129082L;
public QueryMetadataInStringRequest(String groupID, int readConsistencyLevel) {
super(groupID, readConsistencyLevel);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QueryMetadataRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QueryMetadataRequest.java
similarity index 77%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QueryMetadataRequest.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QueryMetadataRequest.java
index 2628fb6..75ae438 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QueryMetadataRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QueryMetadataRequest.java
@@ -16,11 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.request;
+package org.apache.iotdb.cluster.rpc.raft.request.querymetadata;
-import java.io.Serializable;
+import org.apache.iotdb.cluster.rpc.raft.request.BasicQueryRequest;
-public class QueryMetadataRequest extends BasicQueryRequest implements Serializable {
+public class QueryMetadataRequest extends BasicQueryRequest {
+
+ private static final long serialVersionUID = -1976805423799324348L;
public QueryMetadataRequest(String groupID, int readConsistencyLevel) {
super(groupID, readConsistencyLevel);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QueryPathsRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QueryPathsRequest.java
similarity index 80%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QueryPathsRequest.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QueryPathsRequest.java
index 2c600f4..b92a0e6 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QueryPathsRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QueryPathsRequest.java
@@ -16,13 +16,14 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.request;
+package org.apache.iotdb.cluster.rpc.raft.request.querymetadata;
-import java.io.Serializable;
import java.util.List;
+import org.apache.iotdb.cluster.rpc.raft.request.BasicQueryRequest;
-public class QueryPathsRequest extends BasicQueryRequest implements Serializable {
+public class QueryPathsRequest extends BasicQueryRequest {
+ private static final long serialVersionUID = -4334131357874435256L;
private List<String> path;
public QueryPathsRequest(String groupID, int readConsistencyLevel, List<String> path) {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QuerySeriesTypeRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QuerySeriesTypeRequest.java
similarity index 79%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QuerySeriesTypeRequest.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QuerySeriesTypeRequest.java
index c486576..e46fe66 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QuerySeriesTypeRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QuerySeriesTypeRequest.java
@@ -16,12 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.request;
+package org.apache.iotdb.cluster.rpc.raft.request.querymetadata;
-import java.io.Serializable;
+import org.apache.iotdb.cluster.rpc.raft.request.BasicQueryRequest;
-public class QuerySeriesTypeRequest extends BasicQueryRequest implements Serializable {
+public class QuerySeriesTypeRequest extends BasicQueryRequest {
+ private static final long serialVersionUID = -7917403708996214075L;
private String path;
public QuerySeriesTypeRequest(String groupID, int readConsistencyLevel, String path) {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QueryStorageGroupRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QueryStorageGroupRequest.java
similarity index 82%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QueryStorageGroupRequest.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QueryStorageGroupRequest.java
index 037924f..bb3d847 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QueryStorageGroupRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QueryStorageGroupRequest.java
@@ -16,11 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.request;
+package org.apache.iotdb.cluster.rpc.raft.request.querymetadata;
-import java.io.Serializable;
+import org.apache.iotdb.cluster.rpc.raft.request.BasicQueryRequest;
-public class QueryStorageGroupRequest extends BasicQueryRequest implements Serializable {
+public class QueryStorageGroupRequest extends BasicQueryRequest {
+
+ private static final long serialVersionUID = -1260362721166408556L;
public QueryStorageGroupRequest(String groupID, int readConsistencyLevel) {
super(groupID, readConsistencyLevel);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QueryTimeSeriesRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QueryTimeSeriesRequest.java
similarity index 80%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QueryTimeSeriesRequest.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QueryTimeSeriesRequest.java
index 0106f18..92d2f8a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/QueryTimeSeriesRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/querymetadata/QueryTimeSeriesRequest.java
@@ -16,13 +16,14 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.request;
+package org.apache.iotdb.cluster.rpc.raft.request.querymetadata;
-import java.io.Serializable;
import java.util.List;
+import org.apache.iotdb.cluster.rpc.raft.request.BasicQueryRequest;
-public class QueryTimeSeriesRequest extends BasicQueryRequest implements Serializable {
+public class QueryTimeSeriesRequest extends BasicQueryRequest {
+ private static final long serialVersionUID = -1902657459558399385L;
private List<String> path;
public QueryTimeSeriesRequest(String groupID, int readConsistencyLevel, List<String> path) {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/MetaGroupNonQueryResponse.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/BasicQueryDataResponse.java
similarity index 64%
copy from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/MetaGroupNonQueryResponse.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/BasicQueryDataResponse.java
index f662e35..53e7923 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/MetaGroupNonQueryResponse.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/BasicQueryDataResponse.java
@@ -18,22 +18,26 @@
*/
package org.apache.iotdb.cluster.rpc.raft.response;
-/**
- * Handle response from metadata group leader
- */
-public class MetaGroupNonQueryResponse extends BasicResponse {
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public abstract class BasicQueryDataResponse extends BasicResponse{
- private MetaGroupNonQueryResponse(String groupId, boolean redirected, String leaderStr,
+
+ private List<BatchData> seriesBatchData = new ArrayList<>();
+
+ public BasicQueryDataResponse(String groupId, boolean redirected, String leaderStr,
String errorMsg) {
super(groupId, redirected, leaderStr, errorMsg);
}
- public static MetaGroupNonQueryResponse createRedirectedResponse(String groupId, String leaderStr) {
- return new MetaGroupNonQueryResponse(groupId, true, leaderStr, null);
+ public List<BatchData> getSeriesBatchData() {
+ return seriesBatchData;
}
- public static MetaGroupNonQueryResponse createEmptyResponse(String groupId) {
- return new MetaGroupNonQueryResponse(groupId, false, null, null);
+ public void setSeriesBatchData(
+ List<BatchData> seriesBatchData) {
+ this.seriesBatchData = seriesBatchData;
}
-
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/DataGroupNonQueryResponse.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/nonquery/DataGroupNonQueryResponse.java
similarity index 89%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/DataGroupNonQueryResponse.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/nonquery/DataGroupNonQueryResponse.java
index 074f452..9d86398 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/DataGroupNonQueryResponse.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/nonquery/DataGroupNonQueryResponse.java
@@ -16,13 +16,17 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.response;
+package org.apache.iotdb.cluster.rpc.raft.response.nonquery;
+
+import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
/**
* Handle response from data group leader
*/
public class DataGroupNonQueryResponse extends BasicResponse {
+ private static final long serialVersionUID = -8288044965888956717L;
+
private DataGroupNonQueryResponse(String groupId, boolean redirected, String leaderStr,
String errorMsg) {
super(groupId, redirected, leaderStr, errorMsg);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/MetaGroupNonQueryResponse.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/nonquery/MetaGroupNonQueryResponse.java
similarity index 87%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/MetaGroupNonQueryResponse.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/nonquery/MetaGroupNonQueryResponse.java
index f662e35..653958a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/MetaGroupNonQueryResponse.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/nonquery/MetaGroupNonQueryResponse.java
@@ -16,13 +16,17 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.response;
+package org.apache.iotdb.cluster.rpc.raft.response.nonquery;
+
+import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
/**
* Handle response from metadata group leader
*/
public class MetaGroupNonQueryResponse extends BasicResponse {
+ private static final long serialVersionUID = -7444143717755803056L;
+
private MetaGroupNonQueryResponse(String groupId, boolean redirected, String leaderStr,
String errorMsg) {
super(groupId, redirected, leaderStr, errorMsg);
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querydata/InitSeriesReaderResponse.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querydata/InitSeriesReaderResponse.java
new file mode 100644
index 0000000..e30a288
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querydata/InitSeriesReaderResponse.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.cluster.rpc.raft.response.querydata;
+
+import java.util.EnumMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.iotdb.cluster.query.PathType;
+import org.apache.iotdb.cluster.rpc.raft.response.BasicQueryDataResponse;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class InitSeriesReaderResponse extends BasicQueryDataResponse {
+
+ private static final long serialVersionUID = 6298440729566956283L;
+
+ private Map<PathType, List<TSDataType>> seriesDataTypes = new EnumMap<>(PathType.class);
+
+ /**
+ * Series type
+ */
+ private PathType pathType;
+
+ public InitSeriesReaderResponse(String groupId) {
+ super(groupId, false, null, null);
+ }
+
+ public Map<PathType, List<TSDataType>> getSeriesDataTypes() {
+ return seriesDataTypes;
+ }
+
+ public void setSeriesDataTypes(
+ Map<PathType, List<TSDataType>> seriesDataTypes) {
+ this.seriesDataTypes = seriesDataTypes;
+ }
+
+ public PathType getPathType() {
+ return pathType;
+ }
+
+ public void setPathType(PathType pathType) {
+ this.pathType = pathType;
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/exception/FileNodeManagerException.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querydata/QuerySeriesDataByTimestampResponse.java
similarity index 67%
copy from iotdb/src/main/java/org/apache/iotdb/db/exception/FileNodeManagerException.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querydata/QuerySeriesDataByTimestampResponse.java
index 1e5e11d..f3417b7 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/exception/FileNodeManagerException.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querydata/QuerySeriesDataByTimestampResponse.java
@@ -16,22 +16,16 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.db.exception;
+package org.apache.iotdb.cluster.rpc.raft.response.querydata;
-public class FileNodeManagerException extends Exception {
+import org.apache.iotdb.cluster.rpc.raft.response.BasicQueryDataResponse;
- private static final long serialVersionUID = 9001649171768311032L;
+public class QuerySeriesDataByTimestampResponse extends BasicQueryDataResponse {
- public FileNodeManagerException() {
- super();
- }
-
- public FileNodeManagerException(String message) {
- super(message);
- }
+ private static final long serialVersionUID = -9015865924302417289L;
- public FileNodeManagerException(Throwable cause) {
- super(cause);
+ public QuerySeriesDataByTimestampResponse(String groupId) {
+ super(groupId, false, null, null);
}
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicQueryRequest.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querydata/QuerySeriesDataResponse.java
similarity index 58%
copy from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicQueryRequest.java
copy to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querydata/QuerySeriesDataResponse.java
index 2cf613f..b4586bd 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/request/BasicQueryRequest.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querydata/QuerySeriesDataResponse.java
@@ -16,25 +16,29 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.request;
+package org.apache.iotdb.cluster.rpc.raft.response.querydata;
-public abstract class BasicQueryRequest extends BasicRequest {
+import org.apache.iotdb.cluster.query.PathType;
+import org.apache.iotdb.cluster.rpc.raft.response.BasicQueryDataResponse;
+
+public class QuerySeriesDataResponse extends BasicQueryDataResponse {
+
+ private static final long serialVersionUID = -4783032133961145205L;
/**
- * Read Consistency Level
+ * Series type
*/
- private int readConsistencyLevel;
+ private PathType pathType;
- public BasicQueryRequest(String groupID, int readConsistencyLevel) {
- super(groupID);
- this.readConsistencyLevel = readConsistencyLevel;
+ public QuerySeriesDataResponse(String groupId) {
+ super(groupId, false, null, null);
}
- public int getReadConsistencyLevel() {
- return readConsistencyLevel;
+ public PathType getPathType() {
+ return pathType;
}
- public void setReadConsistencyLevel(int readConsistencyLevel) {
- this.readConsistencyLevel = readConsistencyLevel;
+ public void setPathType(PathType pathType) {
+ this.pathType = pathType;
}
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QueryMetadataInStringResponse.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QueryMetadataInStringResponse.java
similarity index 88%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QueryMetadataInStringResponse.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QueryMetadataInStringResponse.java
index a3a963a..98b8201 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QueryMetadataInStringResponse.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QueryMetadataInStringResponse.java
@@ -16,10 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.response;
+package org.apache.iotdb.cluster.rpc.raft.response.querymetadata;
+
+import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
public class QueryMetadataInStringResponse extends BasicResponse {
+ private static final long serialVersionUID = 5704333006127833921L;
private String metadata;
private QueryMetadataInStringResponse(String groupId, boolean redirected, String leaderStr,
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QueryMetadataResponse.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QueryMetadataResponse.java
similarity index 88%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QueryMetadataResponse.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QueryMetadataResponse.java
index 6c21798..20e09f2 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QueryMetadataResponse.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QueryMetadataResponse.java
@@ -16,12 +16,14 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.response;
+package org.apache.iotdb.cluster.rpc.raft.response.querymetadata;
+import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
import org.apache.iotdb.db.metadata.Metadata;
public class QueryMetadataResponse extends BasicResponse {
+ private static final long serialVersionUID = -3969749781116510054L;
private Metadata metadata;
private QueryMetadataResponse(String groupId, boolean redirected, String leaderStr,
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QueryPathsResponse.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QueryPathsResponse.java
similarity index 89%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QueryPathsResponse.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QueryPathsResponse.java
index 29d659a..171563a 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QueryPathsResponse.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QueryPathsResponse.java
@@ -16,13 +16,15 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.response;
+package org.apache.iotdb.cluster.rpc.raft.response.querymetadata;
import java.util.ArrayList;
import java.util.List;
+import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
public class QueryPathsResponse extends BasicResponse {
+ private static final long serialVersionUID = -8255822509893237195L;
private List<String> paths;
private QueryPathsResponse(String groupId, boolean redirected, boolean success, String leaderStr, String errorMsg) {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QuerySeriesTypeResponse.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QuerySeriesTypeResponse.java
similarity index 89%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QuerySeriesTypeResponse.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QuerySeriesTypeResponse.java
index e86e108..eee45c6 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QuerySeriesTypeResponse.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QuerySeriesTypeResponse.java
@@ -16,12 +16,14 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.response;
+package org.apache.iotdb.cluster.rpc.raft.response.querymetadata;
+import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
public class QuerySeriesTypeResponse extends BasicResponse {
+ private static final long serialVersionUID = 7977583965911799165L;
private TSDataType dataType;
private QuerySeriesTypeResponse(String groupId, boolean redirected, String leaderStr,
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QueryStorageGroupResponse.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QueryStorageGroupResponse.java
similarity index 89%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QueryStorageGroupResponse.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QueryStorageGroupResponse.java
index 6abff89..8a3bb11 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QueryStorageGroupResponse.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QueryStorageGroupResponse.java
@@ -16,12 +16,14 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.response;
+package org.apache.iotdb.cluster.rpc.raft.response.querymetadata;
import java.util.Set;
+import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
public class QueryStorageGroupResponse extends BasicResponse {
+ private static final long serialVersionUID = 248840631619860233L;
private Set<String> storageGroups;
private QueryStorageGroupResponse(boolean success, String leaderStr, String errorMsg) {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QueryTimeSeriesResponse.java b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QueryTimeSeriesResponse.java
similarity index 89%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QueryTimeSeriesResponse.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QueryTimeSeriesResponse.java
index edeb4c4..1e029e8 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/QueryTimeSeriesResponse.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/rpc/raft/response/querymetadata/QueryTimeSeriesResponse.java
@@ -16,13 +16,15 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.raft.response;
+package org.apache.iotdb.cluster.rpc.raft.response.querymetadata;
import java.util.ArrayList;
import java.util.List;
+import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
public class QueryTimeSeriesResponse extends BasicResponse {
+ private static final long serialVersionUID = 8313150788331085964L;
private List<List<String>> timeSeries;
private QueryTimeSeriesResponse(String groupId, boolean redirected, boolean success, String leaderStr, String errorMsg) {
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/service/TSServiceClusterImpl.java b/cluster/src/main/java/org/apache/iotdb/cluster/service/TSServiceClusterImpl.java
similarity index 67%
rename from cluster/src/main/java/org/apache/iotdb/cluster/rpc/service/TSServiceClusterImpl.java
rename to cluster/src/main/java/org/apache/iotdb/cluster/service/TSServiceClusterImpl.java
index 33e3e81..bfc74c1 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/rpc/service/TSServiceClusterImpl.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/service/TSServiceClusterImpl.java
@@ -16,32 +16,45 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.cluster.rpc.service;
+package org.apache.iotdb.cluster.service;
-import com.alipay.sofa.jraft.util.OnlyForTest;
import java.io.IOException;
import java.sql.Statement;
import java.util.Arrays;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import java.util.Set;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import org.apache.iotdb.cluster.config.ClusterConstant;
import org.apache.iotdb.cluster.exception.ConsistencyLevelException;
+import org.apache.iotdb.cluster.qp.executor.ClusterQueryProcessExecutor;
import org.apache.iotdb.cluster.qp.executor.NonQueryExecutor;
import org.apache.iotdb.cluster.qp.executor.QueryMetadataExecutor;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcQueryManager;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.IClusterRpcQueryManager;
import org.apache.iotdb.db.auth.AuthException;
import org.apache.iotdb.db.conf.IoTDBConstant;
+import org.apache.iotdb.db.exception.FileNodeManagerException;
import org.apache.iotdb.db.exception.PathErrorException;
import org.apache.iotdb.db.exception.ProcessorException;
import org.apache.iotdb.db.metadata.Metadata;
+import org.apache.iotdb.db.qp.QueryProcessor;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
import org.apache.iotdb.db.service.TSServiceImpl;
+import org.apache.iotdb.service.rpc.thrift.TSCloseOperationReq;
import org.apache.iotdb.service.rpc.thrift.TSExecuteBatchStatementReq;
import org.apache.iotdb.service.rpc.thrift.TSExecuteBatchStatementResp;
+import org.apache.iotdb.service.rpc.thrift.TSFetchResultsReq;
import org.apache.iotdb.service.rpc.thrift.TS_StatusCode;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
import org.apache.thrift.TException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -53,17 +66,50 @@ public class TSServiceClusterImpl extends TSServiceImpl {
private static final Logger LOGGER = LoggerFactory.getLogger(TSServiceClusterImpl.class);
- private ThreadLocal<NonQueryExecutor> nonQueryExecutor = new ThreadLocal<>();
- private ThreadLocal<QueryMetadataExecutor> queryMetadataExecutor = new ThreadLocal<>();
+ private ClusterQueryProcessExecutor queryDataExecutor = new ClusterQueryProcessExecutor();
+ private NonQueryExecutor nonQueryExecutor = new NonQueryExecutor();
+ private QueryMetadataExecutor queryMetadataExecutor = new QueryMetadataExecutor();
+
+ private IClusterRpcQueryManager queryManager = ClusterRpcQueryManager.getInstance();
public TSServiceClusterImpl() throws IOException {
super();
+ processor = new QueryProcessor(queryDataExecutor);
+ }
+
+
+ @Override
+ protected Set<String> getAllStorageGroups() throws InterruptedException {
+ return queryMetadataExecutor.processStorageGroupQuery();
+ }
+
+ @Override
+ protected List<List<String>> getTimeSeriesForPath(String path)
+ throws PathErrorException, InterruptedException, ProcessorException {
+ return queryMetadataExecutor.processTimeSeriesQuery(path);
+ }
+
+ @Override
+ protected String getMetadataInString()
+ throws InterruptedException, ProcessorException {
+ return queryMetadataExecutor.processMetadataInStringQuery();
}
@Override
- public void initClusterService() {
- nonQueryExecutor.set(new NonQueryExecutor());
- queryMetadataExecutor.set(new QueryMetadataExecutor());
+ protected Metadata getMetadata()
+ throws InterruptedException, ProcessorException, PathErrorException {
+ return queryMetadataExecutor.processMetadataQuery();
+ }
+
+ @Override
+ protected TSDataType getSeriesType(String path) throws PathErrorException, InterruptedException, ProcessorException {
+ return queryMetadataExecutor.processSeriesTypeQuery(path);
+ }
+
+ @Override
+ protected List<String> getPaths(String path)
+ throws PathErrorException, InterruptedException, ProcessorException {
+ return queryMetadataExecutor.processPathsQuery(path);
}
@Override
@@ -96,7 +142,7 @@ public class TSServiceClusterImpl extends TSServiceImpl {
result = resultTemp;
physicalPlans = physicalPlansTemp;
BatchResult batchResult = new BatchResult(isAllSuccessful, batchErrorMessage, result);
- nonQueryExecutor.get().processBatch(physicalPlans, batchResult);
+ nonQueryExecutor.processBatch(physicalPlans, batchResult);
return getTSBathExecuteStatementResp(TS_StatusCode.ERROR_STATUS,
"statement is query :" + statements.get(i), Arrays.stream(result).boxed().collect(
Collectors.toList()));
@@ -130,7 +176,7 @@ public class TSServiceClusterImpl extends TSServiceImpl {
}
BatchResult batchResult = new BatchResult(isAllSuccessful, batchErrorMessage, result);
- nonQueryExecutor.get().processBatch(physicalPlans, batchResult);
+ nonQueryExecutor.processBatch(physicalPlans, batchResult);
batchErrorMessage = batchResult.batchErrorMessage;
isAllSuccessful = batchResult.isAllSuccessful;
@@ -199,13 +245,13 @@ public class TSServiceClusterImpl extends TSServiceImpl {
if (Pattern.matches(ClusterConstant.SET_READ_METADATA_CONSISTENCY_LEVEL_PATTERN, statement)) {
String[] splits = statement.split("\\s+");
int level = Integer.parseInt(splits[splits.length - 1]);
- nonQueryExecutor.get().setReadMetadataConsistencyLevel(level);
+ queryMetadataExecutor.setReadMetadataConsistencyLevel(level);
return true;
} else if (Pattern
.matches(ClusterConstant.SET_READ_DATA_CONSISTENCY_LEVEL_PATTERN, statement)) {
String[] splits = statement.split("\\s+");
int level = Integer.parseInt(splits[splits.length - 1]);
- nonQueryExecutor.get().setReadDataConsistencyLevel(level);
+ queryDataExecutor.setReadDataConsistencyLevel(level);
return true;
} else {
return false;
@@ -217,54 +263,77 @@ public class TSServiceClusterImpl extends TSServiceImpl {
@Override
protected boolean executeNonQuery(PhysicalPlan plan) throws ProcessorException {
- return nonQueryExecutor.get().processNonQuery(plan);
+ return nonQueryExecutor.processNonQuery(plan);
}
/**
- * Close cluster service
+ * It's unnecessary to do this check. It has benn checked in transforming query physical plan.
*/
@Override
- public void closeClusterService() {
- nonQueryExecutor.get().shutdown();
- queryMetadataExecutor.get().shutdown();
+ public void checkFileLevelSet(List<Path> paths) throws PathErrorException {
}
@Override
- protected Set<String> getAllStorageGroups() throws InterruptedException {
- return queryMetadataExecutor.get().processStorageGroupQuery();
+ public void releaseQueryResource(TSCloseOperationReq req) throws Exception {
+ Map<Long, QueryContext> contextMap = contextMapLocal.get();
+ if (contextMap == null) {
+ return;
+ }
+ if (req == null || req.queryId == -1) {
+ // end query for all the query tokens created by current thread
+ for (QueryContext context : contextMap.values()) {
+ QueryResourceManager.getInstance().endQueryForGivenJob(context.getJobId());
+ queryManager.releaseQueryResource(context.getJobId());
+ }
+ contextMapLocal.set(new HashMap<>());
+ } else {
+ long jobId = contextMap.remove(req.queryId).getJobId();
+ QueryResourceManager.getInstance().endQueryForGivenJob(jobId);
+ queryManager.releaseQueryResource(jobId);
+ }
}
@Override
- protected List<List<String>> getTimeSeriesForPath(String path)
- throws PathErrorException, InterruptedException, ProcessorException {
- return queryMetadataExecutor.get().processTimeSeriesQuery(path);
- }
+ public QueryDataSet createNewDataSet(String statement, int fetchSize, TSFetchResultsReq req)
+ throws PathErrorException, QueryFilterOptimizationException, FileNodeManagerException,
+ ProcessorException, IOException {
+ PhysicalPlan physicalPlan = queryStatus.get().get(statement);
+ processor.getExecutor().setFetchSize(fetchSize);
+ long jobId = QueryResourceManager.getInstance().assignJobId();
+ QueryContext context = new QueryContext(jobId);
+ initContextMap();
+ contextMapLocal.get().put(req.queryId, context);
+
+ queryManager.addSingleQuery(jobId, (QueryPlan) physicalPlan);
+ QueryDataSet queryDataSet = processor.getExecutor().processQuery((QueryPlan) physicalPlan,
+ context);
+ queryRet.get().put(statement, queryDataSet);
+ return queryDataSet;
+ }
+ /**
+ * Close cluster service
+ */
@Override
- protected String getMetadataInString()
- throws InterruptedException, ProcessorException {
- return queryMetadataExecutor.get().processMetadataInStringQuery();
+ public void closeClusterService() {
+ nonQueryExecutor.shutdown();
+ queryMetadataExecutor.shutdown();
}
- @Override
- protected Metadata getMetadata()
- throws InterruptedException, ProcessorException, PathErrorException {
- return queryMetadataExecutor.get().processMetadataQuery();
+ public ClusterQueryProcessExecutor getQueryDataExecutor() {
+ return queryDataExecutor;
}
- @Override
- protected TSDataType getSeriesType(String path) throws PathErrorException, InterruptedException, ProcessorException {
- return queryMetadataExecutor.get().processSeriesTypeQuery(path);
+ public void setQueryDataExecutor(
+ ClusterQueryProcessExecutor queryDataExecutor) {
+ this.queryDataExecutor = queryDataExecutor;
}
- @Override
- protected List<String> getPaths(String path)
- throws PathErrorException, InterruptedException, ProcessorException {
- return queryMetadataExecutor.get().processPathsQuery(path);
+ public QueryMetadataExecutor getQueryMetadataExecutor() {
+ return queryMetadataExecutor;
}
- @OnlyForTest
- public NonQueryExecutor getNonQueryExecutor() {
- return nonQueryExecutor.get();
+ public void setNonQueryExecutor(NonQueryExecutor nonQueryExecutor) {
+ this.nonQueryExecutor = nonQueryExecutor;
}
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/utils/QPExecutorUtils.java b/cluster/src/main/java/org/apache/iotdb/cluster/utils/QPExecutorUtils.java
new file mode 100644
index 0000000..809a01c
--- /dev/null
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/utils/QPExecutorUtils.java
@@ -0,0 +1,156 @@
+/**
+ * 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.cluster.utils;
+
+import com.alipay.sofa.jraft.util.OnlyForTest;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iotdb.cluster.config.ClusterConfig;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.entity.Server;
+import org.apache.iotdb.cluster.entity.raft.MetadataRaftHolder;
+import org.apache.iotdb.cluster.utils.hash.PhysicalNode;
+import org.apache.iotdb.cluster.utils.hash.Router;
+import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.metadata.MManager;
+
+/**
+ * Utils for QP executor
+ */
+public class QPExecutorUtils {
+
+ private static final ClusterConfig CLUSTER_CONFIG = ClusterDescriptor.getInstance().getConfig();
+
+ private static final Router router = Router.getInstance();
+
+ private static final PhysicalNode localNode = new PhysicalNode(CLUSTER_CONFIG.getIp(),
+ CLUSTER_CONFIG.getPort());
+
+ private static final MManager mManager = MManager.getInstance();
+
+ private static final Server server = Server.getInstance();
+
+ /**
+ * Get Storage Group Name by device name
+ */
+ public static String getStroageGroupByDevice(String device) throws PathErrorException {
+ String storageGroup;
+ try {
+ storageGroup = MManager.getInstance().getFileNameByPath(device);
+ } catch (PathErrorException e) {
+ throw new PathErrorException(String.format("File level of %s doesn't exist.", device));
+ }
+ return storageGroup;
+ }
+
+ /**
+ * Get all Storage Group Names by path
+ */
+ public static List<String> getAllStroageGroupsByPath(String path) throws PathErrorException {
+ List<String> storageGroupList;
+ try {
+ storageGroupList = mManager.getAllFileNamesByPath(path);
+ } catch (PathErrorException e) {
+ throw new PathErrorException(String.format("File level of %s doesn't exist.", path));
+ }
+ return storageGroupList;
+ }
+
+ /**
+ * Classify the input storage group list by which data group it belongs to.
+ *
+ * @return key is groupId, value is all SGs belong to this data group
+ */
+ public static Map<String, Set<String>> classifySGByGroupId(List<String> sgList) {
+ Map<String, Set<String>> map = new HashMap<>();
+ for (int i = 0; i < sgList.size(); i++) {
+ String sg = sgList.get(i);
+ String groupId = router.getGroupIdBySG(sg);
+ if (map.containsKey(groupId)) {
+ map.get(groupId).add(sg);
+ } else {
+ Set<String> set = new HashSet<>();
+ set.add(sg);
+ map.put(groupId, set);
+ }
+ }
+ return map;
+ }
+
+ /**
+ * Check if the non query command can execute in local. 1. If this node belongs to the storage
+ * group 2. If this node is leader.
+ */
+ public static boolean canHandleNonQueryByGroupId(String groupId) {
+ boolean canHandle = false;
+ if(groupId.equals(ClusterConfig.METADATA_GROUP_ID)){
+ canHandle = ((MetadataRaftHolder) (server.getMetadataHolder())).getFsm().isLeader();
+ }else {
+ if (checkDataGroupLeader(groupId)) {
+ canHandle = true;
+ }
+ }
+ return canHandle;
+ }
+
+ /**
+ * Check whether local node is leader of data group.
+ *
+ * @param groupId data group
+ * @return true if local node is leader of data group, else return false
+ */
+ public static boolean checkDataGroupLeader(String groupId) {
+ boolean isLeader = false;
+ if (router.containPhysicalNodeByGroupId(groupId, localNode) && RaftUtils
+ .getPhysicalNodeFrom(RaftUtils.getLeaderPeerID(groupId)).equals(localNode)) {
+ isLeader = true;
+ }
+ return isLeader;
+ }
+
+ /**
+ * Check if the query command can execute in local. Check if this node belongs to the group id
+ */
+ public static boolean canHandleQueryByGroupId(String groupId) {
+ return router.containPhysicalNodeByGroupId(groupId, localNode);
+ }
+
+ /**
+ * Get group id by device
+ *
+ * @param device device
+ */
+ public static String getGroupIdByDevice(String device) throws PathErrorException {
+ String storageGroup = QPExecutorUtils.getStroageGroupByDevice(device);
+ String groupId = Router.getInstance().getGroupIdBySG(storageGroup);
+ return groupId;
+ }
+
+ /**
+ * Change address of local node for test
+ */
+ @OnlyForTest
+ public static void setLocalNodeAddr(String ip, int port) {
+ localNode.setIp(ip);
+ localNode.setPort(port);
+ }
+}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/utils/RaftUtils.java b/cluster/src/main/java/org/apache/iotdb/cluster/utils/RaftUtils.java
index 744730f..be6eea0 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/utils/RaftUtils.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/utils/RaftUtils.java
@@ -27,23 +27,28 @@ import com.alipay.sofa.jraft.entity.PeerId;
import com.alipay.sofa.jraft.entity.Task;
import com.alipay.sofa.jraft.util.Bits;
import com.alipay.sofa.jraft.util.OnlyForTest;
-
import java.nio.ByteBuffer;
import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.iotdb.cluster.qp.callback.QPTask;
-import org.apache.iotdb.cluster.qp.callback.SingleQPTask;
import org.apache.iotdb.cluster.config.ClusterConfig;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
import org.apache.iotdb.cluster.entity.Server;
import org.apache.iotdb.cluster.entity.raft.DataPartitionRaftHolder;
import org.apache.iotdb.cluster.entity.raft.MetadataRaftHolder;
import org.apache.iotdb.cluster.entity.raft.RaftService;
+import org.apache.iotdb.cluster.exception.RaftConnectionException;
+import org.apache.iotdb.cluster.qp.task.QPTask;
+import org.apache.iotdb.cluster.qp.task.SingleQPTask;
+import org.apache.iotdb.cluster.rpc.raft.NodeAsClient;
import org.apache.iotdb.cluster.rpc.raft.closure.ResponseClosure;
+import org.apache.iotdb.cluster.rpc.raft.impl.RaftNodeAsClientManager;
+import org.apache.iotdb.cluster.rpc.raft.request.BasicNonQueryRequest;
import org.apache.iotdb.cluster.rpc.raft.request.BasicRequest;
import org.apache.iotdb.cluster.rpc.raft.response.BasicResponse;
-import org.apache.iotdb.cluster.rpc.raft.response.MetaGroupNonQueryResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.nonquery.DataGroupNonQueryResponse;
+import org.apache.iotdb.cluster.rpc.raft.response.nonquery.MetaGroupNonQueryResponse;
import org.apache.iotdb.cluster.utils.hash.PhysicalNode;
import org.apache.iotdb.cluster.utils.hash.Router;
import org.slf4j.Logger;
@@ -51,10 +56,17 @@ import org.slf4j.LoggerFactory;
public class RaftUtils {
+ private static final ClusterConfig CLUSTER_CONFIG = ClusterDescriptor.getInstance().getConfig();
+
private static final Logger LOGGER = LoggerFactory.getLogger(RaftUtils.class);
private static final Server server = Server.getInstance();
private static final Router router = Router.getInstance();
private static final AtomicInteger requestId = new AtomicInteger(0);
+ /**
+ * Raft as client manager.
+ */
+ private static final RaftNodeAsClientManager CLIENT_MANAGER = RaftNodeAsClientManager
+ .getInstance();
/**
* The cache will be update in two case: 1. When @onLeaderStart() method of state machine is
@@ -110,7 +122,7 @@ public class RaftUtils {
}
public static PeerId getPeerIDFrom(PhysicalNode node) {
- return new PeerId(node.ip, node.port);
+ return new PeerId(node.getIp(), node.getPort());
}
public static PhysicalNode getPhysicalNodeFrom(PeerId peer) {
@@ -165,6 +177,15 @@ public class RaftUtils {
LOGGER.info("group leader cache:{}", groupLeaderCache);
}
+ /**
+ * Remove cached raft group leader if occurs exception in the process of executing qp task.
+ *
+ * @param groupId data group id
+ */
+ public static void removeCachedRaftGroupLeader(String groupId) {
+ groupLeaderCache.remove(groupId);
+ }
+
@OnlyForTest
public static void clearRaftGroupLeader() {
groupLeaderCache.clear();
@@ -206,7 +227,7 @@ public class RaftUtils {
* @param service raft service
*/
public static void executeRaftTaskForRpcProcessor(RaftService service, AsyncContext asyncContext,
- BasicRequest request, BasicResponse response) {
+ BasicNonQueryRequest request, BasicResponse response) {
final Task task = new Task();
ResponseClosure closure = new ResponseClosure(response, status -> {
response.addResult(status.isOk());
@@ -262,8 +283,6 @@ public class RaftUtils {
/**
* Handle null-read process in metadata group if the request is to set path.
- *
- * @param status status to return result if this node is leader of the data group
*/
public static void handleNullReadToMetaGroup(Status status) {
SingleQPTask nullReadTask = new SingleQPTask(false, null);
@@ -273,7 +292,7 @@ public class RaftUtils {
public static void handleNullReadToMetaGroup(Status status, Server server,
SingleQPTask nullReadTask) {
try {
- LOGGER.debug("Handle null-read in meta group for adding path request.");
+ LOGGER.debug("Handle null-read in meta group for metadata request.");
final byte[] reqContext = RaftUtils.createRaftRequestContext();
MetadataRaftHolder metadataRaftHolder = (MetadataRaftHolder) server.getMetadataHolder();
((RaftService) metadataRaftHolder.getService()).getNode()
@@ -296,10 +315,58 @@ public class RaftUtils {
}
}
+ /**
+ * Handle null-read process in data group while reading process
+ */
+ public static void handleNullReadToDataGroup(Status status, String groupId) {
+ SingleQPTask nullReadTask = new SingleQPTask(false, null);
+ handleNullReadToDataGroup(status, server, nullReadTask, groupId);
+ }
+
+ private static void handleNullReadToDataGroup(Status status, Server server,
+ SingleQPTask nullReadTask, String groupId) {
+ try {
+ LOGGER.debug("Handle null-read in data group for reading.");
+ final byte[] reqContext = RaftUtils.createRaftRequestContext();
+ DataPartitionRaftHolder dataPartitionRaftHolder = (DataPartitionRaftHolder) server.getDataPartitionHolder(groupId);
+ ((RaftService) dataPartitionRaftHolder.getService()).getNode()
+ .readIndex(reqContext, new ReadIndexClosure() {
+ @Override
+ public void run(Status status, long index, byte[] reqCtx) {
+ BasicResponse response = DataGroupNonQueryResponse
+ .createEmptyResponse(groupId);
+ if (!status.isOk()) {
+ status.setCode(-1);
+ status.setErrorMsg(status.getErrorMsg());
+ }
+ nullReadTask.run(response);
+ }
+ });
+ nullReadTask.await();
+ } catch (InterruptedException e) {
+ status.setCode(-1);
+ status.setErrorMsg(e.getMessage());
+ }
+ }
+
public static Status createErrorStatus(String errorMsg){
Status status = new Status();
status.setErrorMsg(errorMsg);
status.setCode(-1);
return status;
}
+
+ /**
+ * try to get raft rpc client
+ */
+ public static NodeAsClient getRaftNodeAsClient() throws RaftConnectionException {
+ NodeAsClient client = CLIENT_MANAGER.getRaftNodeAsClient();
+ if (client == null) {
+ throw new RaftConnectionException(String
+ .format("Raft inner rpc clients have reached the max numbers %s",
+ CLUSTER_CONFIG.getMaxNumOfInnerRpcClient() + CLUSTER_CONFIG
+ .getMaxQueueNumOfInnerRpcClient()));
+ }
+ return client;
+ }
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/utils/hash/PhysicalNode.java b/cluster/src/main/java/org/apache/iotdb/cluster/utils/hash/PhysicalNode.java
index b8c30c9..66544a8 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/utils/hash/PhysicalNode.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/utils/hash/PhysicalNode.java
@@ -18,10 +18,12 @@
*/
package org.apache.iotdb.cluster.utils.hash;
+import com.alipay.sofa.jraft.util.OnlyForTest;
+
public class PhysicalNode {
- public final String ip;
- public final int port;
+ private String ip;
+ private int port;
public PhysicalNode(String ip, int port) {
this.ip = ip;
@@ -74,4 +76,14 @@ public class PhysicalNode {
public int getPort() {
return port;
}
+
+ @OnlyForTest
+ public void setIp(String ip) {
+ this.ip = ip;
+ }
+
+ @OnlyForTest
+ public void setPort(int port) {
+ this.port = port;
+ }
}
diff --git a/cluster/src/main/java/org/apache/iotdb/cluster/utils/hash/Router.java b/cluster/src/main/java/org/apache/iotdb/cluster/utils/hash/Router.java
index 7c7b2be..544c0fc 100644
--- a/cluster/src/main/java/org/apache/iotdb/cluster/utils/hash/Router.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/utils/hash/Router.java
@@ -189,6 +189,7 @@ public class Router {
sgRouter.clear();
dataPartitionCache.clear();
nodeMapGroupIdCache.clear();
+ groupIdMapNodeCache.clear();
}
@OnlyForTest
@@ -236,4 +237,11 @@ public class Router {
public Set<String> getAllGroupId() {
return groupIdMapNodeCache.keySet();
}
+
+ /**
+ * Get raft group id by storage group name
+ */
+ public String getGroupIdBySG(String storageGroup) {
+ return getGroupID(routeGroup(storageGroup));
+ }
}
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/config/ClusterDescriptorTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/config/ClusterDescriptorTest.java
index 1e8650c..a03ee99 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/config/ClusterDescriptorTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/config/ClusterDescriptorTest.java
@@ -78,7 +78,6 @@ public class ClusterDescriptorTest {
private int testMetadataConsistencyOld;
private int testDataConsistencyOld;
private int testConcurrentQPTaskThreadOld;
- private int testConcurrentRaftTaskThreadOld;
private Map<String, String> testConfigMap = new HashMap<String, String>() {
private static final long serialVersionUID = 7832408957178621132L;
@@ -121,6 +120,7 @@ public class ClusterDescriptorTest {
@Test
public void test() throws IOException {
+ String oldConfig = System.getProperty(IoTDBConstant.IOTDB_CONF);
System.setProperty(IoTDBConstant.IOTDB_CONF, absoultePath);
ClusterDescriptor.getInstance().loadProps();
ClusterConfig config = ClusterDescriptor.getInstance().getConfig();
@@ -149,8 +149,11 @@ public class ClusterDescriptorTest {
assertEquals(testDataConsistencyNew, config.getReadDataConsistencyLevel() + "");
assertEquals(testConcurrentQPTaskThreadNew, config.getConcurrentQPSubTaskThread() + "");
-
- System.setProperty(IoTDBConstant.IOTDB_CONF, "");
+ if (oldConfig == null) {
+ System.clearProperty(IoTDBConstant.IOTDB_CONF);
+ } else {
+ System.setProperty(IoTDBConstant.IOTDB_CONF, oldConfig);
+ }
config.deleteAllPath();
}
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/integration/IoTDBMetadataFetchRemoteIT.java b/cluster/src/test/java/org/apache/iotdb/cluster/integration/IoTDBMetadataFetchRemoteIT.java
index f601f14..76eaca0 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/integration/IoTDBMetadataFetchRemoteIT.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/integration/IoTDBMetadataFetchRemoteIT.java
@@ -29,7 +29,6 @@ import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
-
public class IoTDBMetadataFetchRemoteIT extends IoTDBMetadataFetchAbstract{
@BeforeClass
public static void setUp() throws Exception {
@@ -44,11 +43,11 @@ public class IoTDBMetadataFetchRemoteIT extends IoTDBMetadataFetchAbstract{
@Test
public void test() throws IOException {
- String dir = Utils.getCurrentPath("pwd");
- String node = "3";
- String replicator = "3";
- startScript("sh", dir + File.separator + "script" + File.separator + "deploy.sh", node, replicator, dir);
- startScript("sh", dir + File.separator + "script" + File.separator + "stop.sh", node, replicator, dir);
+// String dir = Utils.getCurrentPath("pwd");
+// String node = "3";
+// String replicator = "3";
+// startScript("sh", dir + File.separator + "script" + File.separator + "deploy.sh", node, replicator, dir);
+// startScript("sh", dir + File.separator + "script" + File.separator + "stop.sh", node, replicator, dir);
}
private void startScript(String... command) throws IOException {
@@ -68,4 +67,4 @@ public class IoTDBMetadataFetchRemoteIT extends IoTDBMetadataFetchAbstract{
r.close();
p.destroy();
}
-}
\ No newline at end of file
+}
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/qp/ClusterQPExecutorTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/qp/AbstractQPExecutorTest.java
similarity index 69%
rename from cluster/src/test/java/org/apache/iotdb/cluster/qp/ClusterQPExecutorTest.java
rename to cluster/src/test/java/org/apache/iotdb/cluster/qp/AbstractQPExecutorTest.java
index 298dd4d..02b0311 100644
--- a/cluster/src/test/java/org/apache/iotdb/cluster/qp/ClusterQPExecutorTest.java
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/qp/AbstractQPExecutorTest.java
@@ -24,28 +24,30 @@ import static org.junit.Assert.assertTrue;
import org.apache.iotdb.cluster.config.ClusterConfig;
import org.apache.iotdb.cluster.config.ClusterDescriptor;
-import org.apache.iotdb.cluster.qp.executor.NonQueryExecutor;
-import org.apache.iotdb.cluster.rpc.service.TSServiceClusterImpl;
+import org.apache.iotdb.cluster.qp.executor.ClusterQueryProcessExecutor;
+import org.apache.iotdb.cluster.qp.executor.QueryMetadataExecutor;
+import org.apache.iotdb.cluster.service.TSServiceClusterImpl;
import org.apache.iotdb.cluster.utils.EnvironmentUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
-public class ClusterQPExecutorTest {
+public class AbstractQPExecutorTest {
private static final ClusterConfig CLUSTER_CONFIG = ClusterDescriptor.getInstance().getConfig();
private TSServiceClusterImpl impl;
- private NonQueryExecutor executor;
+ private ClusterQueryProcessExecutor queryExecutor;
+
+ private QueryMetadataExecutor queryMetadataExecutor;
@Before
public void setUp() throws Exception {
EnvironmentUtils.envSetUp();
impl = new TSServiceClusterImpl();
- impl.initClusterService();
-
- executor = impl.getNonQueryExecutor();
+ queryMetadataExecutor = impl.getQueryMetadataExecutor();
+ queryExecutor = impl.getQueryDataExecutor();
}
@After
@@ -57,22 +59,22 @@ public class ClusterQPExecutorTest {
@Test
public void setReadMetadataConsistencyLevel() throws Exception {
assertEquals(CLUSTER_CONFIG.getReadMetadataConsistencyLevel(),
- executor.getReadMetadataConsistencyLevel());
+ queryMetadataExecutor.getReadMetadataConsistencyLevel());
boolean exec;
exec= impl.execSetConsistencyLevel("set read metadata level to 1");
assertTrue(exec);
- assertEquals(1, executor.getReadMetadataConsistencyLevel());
+ assertEquals(1, queryMetadataExecutor.getReadMetadataConsistencyLevel());
exec= impl.execSetConsistencyLevel("show timeseries");
- assertEquals(1, executor.getReadMetadataConsistencyLevel());
+ assertEquals(1, queryMetadataExecutor.getReadMetadataConsistencyLevel());
assertFalse(exec);
exec= impl.execSetConsistencyLevel("set read metadata level to 2");
assertTrue(exec);
- assertEquals(2, executor.getReadMetadataConsistencyLevel());
+ assertEquals(2, queryMetadataExecutor.getReadMetadataConsistencyLevel());
exec = impl.execSetConsistencyLevel("set read metadata level to -2");
- assertEquals(2, executor.getReadMetadataConsistencyLevel());
+ assertEquals(2, queryMetadataExecutor.getReadMetadataConsistencyLevel());
assertFalse(exec);
try {
@@ -80,28 +82,28 @@ public class ClusterQPExecutorTest {
} catch (Exception e) {
assertEquals("Consistency level 90 not support", e.getMessage());
}
- assertEquals(2, executor.getReadMetadataConsistencyLevel());
+ assertEquals(2, queryMetadataExecutor.getReadMetadataConsistencyLevel());
}
@Test
public void setReadDataConsistencyLevel() throws Exception {
assertEquals(CLUSTER_CONFIG.getReadDataConsistencyLevel(),
- executor.getReadDataConsistencyLevel());
+ queryMetadataExecutor.getReadDataConsistencyLevel());
boolean exec;
exec= impl.execSetConsistencyLevel("set read data level to 1");
assertTrue(exec);
- assertEquals(1, executor.getReadDataConsistencyLevel());
+ assertEquals(1, queryExecutor.getReadDataConsistencyLevel());
exec= impl.execSetConsistencyLevel("show timeseries");
- assertEquals(1, executor.getReadDataConsistencyLevel());
+ assertEquals(1, queryExecutor.getReadDataConsistencyLevel());
assertFalse(exec);
exec= impl.execSetConsistencyLevel("set read data level to 2");
assertTrue(exec);
- assertEquals(2, executor.getReadDataConsistencyLevel());
+ assertEquals(2, queryExecutor.getReadDataConsistencyLevel());
exec = impl.execSetConsistencyLevel("set read data level to -2");
- assertEquals(2, executor.getReadDataConsistencyLevel());
+ assertEquals(2, queryExecutor.getReadDataConsistencyLevel());
assertFalse(exec);
try {
@@ -109,6 +111,6 @@ public class ClusterQPExecutorTest {
} catch (Exception e) {
assertEquals("Consistency level 90 not support", e.getMessage());
}
- assertEquals(2, executor.getReadDataConsistencyLevel());
+ assertEquals(2, queryExecutor.getReadDataConsistencyLevel());
}
}
\ No newline at end of file
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterQueryLargeDataTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterQueryLargeDataTest.java
new file mode 100644
index 0000000..223f0dc
--- /dev/null
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterQueryLargeDataTest.java
@@ -0,0 +1,507 @@
+/**
+ * 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.cluster.query;
+
+import static org.apache.iotdb.cluster.utils.Utils.insertData;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.iotdb.cluster.config.ClusterConfig;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.entity.Server;
+import org.apache.iotdb.cluster.utils.EnvironmentUtils;
+import org.apache.iotdb.cluster.utils.hash.PhysicalNode;
+import org.apache.iotdb.jdbc.Config;
+import org.apache.iotdb.jdbc.IoTDBResultMetadata;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class ClusterQueryLargeDataTest {
+
+
+ private Server server;
+ private static final ClusterConfig CLUSTER_CONFIG = ClusterDescriptor.getInstance().getConfig();
+ private static final PhysicalNode localNode = new PhysicalNode(CLUSTER_CONFIG.getIp(),
+ CLUSTER_CONFIG.getPort());
+
+ private static final String URL = "127.0.0.1:6667/";
+
+ private static final String[] createSQLs1 = {
+ "SET STORAGE GROUP TO root.vehicle",
+ "SET STORAGE GROUP TO root.test",
+ "CREATE TIMESERIES root.vehicle.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d0.s1 WITH DATATYPE=TEXT, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.vehicle.d1.s2 WITH DATATYPE=FLOAT, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d1.s3 WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.test.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+ "CREATE TIMESERIES root.test.d0.s1 WITH DATATYPE=TEXT, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.test.d1.g0.s0 WITH DATATYPE=INT32, ENCODING=RLE"
+ };
+ private static final String[] insertSQLs1 = {
+ "insert into root.vehicle.d0(timestamp,s0) values(10,100)",
+ "insert into root.vehicle.d0(timestamp,s0,s1) values(12,101,'102')",
+ "insert into root.vehicle.d0(timestamp,s1) values(19,'103')",
+ "insert into root.vehicle.d1(timestamp,s2) values(11,104.0)",
+ "insert into root.vehicle.d1(timestamp,s2,s3) values(15,105.0,true)",
+ "insert into root.vehicle.d1(timestamp,s3) values(17,false)",
+ "insert into root.vehicle.d0(timestamp,s0) values(20,1000)",
+ "insert into root.vehicle.d0(timestamp,s0,s1) values(22,1001,'1002')",
+ "insert into root.vehicle.d0(timestamp,s1) values(29,'1003')",
+ "insert into root.vehicle.d1(timestamp,s2) values(21,1004.0)",
+ "insert into root.vehicle.d1(timestamp,s2,s3) values(25,1005.0,true)",
+ "insert into root.vehicle.d1(timestamp,s3) values(27,true)",
+ "insert into root.test.d0(timestamp,s0) values(10,106)",
+ "insert into root.test.d0(timestamp,s0,s1) values(14,107,'108')",
+ "insert into root.test.d0(timestamp,s1) values(16,'109')",
+ "insert into root.test.d1.g0(timestamp,s0) values(1,110)",
+ "insert into root.test.d0(timestamp,s0) values(30,1006)",
+ "insert into root.test.d0(timestamp,s0,s1) values(34,1007,'1008')",
+ "insert into root.test.d0(timestamp,s1) values(36,'1090')",
+ "insert into root.test.d1.g0(timestamp,s0) values(10,1100)"};
+ private static final String[] insertSqls2 = {
+ "insert into root.vehicle.d0(timestamp,s0) values(6,120)",
+ "insert into root.vehicle.d0(timestamp,s0,s1) values(38,121,'122')",
+ "insert into root.vehicle.d0(timestamp,s1) values(9,'123')",
+ "insert into root.vehicle.d0(timestamp,s0) values(16,128)",
+ "insert into root.vehicle.d0(timestamp,s0,s1) values(18,189,'198')",
+ "insert into root.vehicle.d0(timestamp,s1) values(99,'1234')",
+ "insert into root.vehicle.d1(timestamp,s2) values(14,1024.0)",
+ "insert into root.vehicle.d1(timestamp,s2,s3) values(29,1205.0,true)",
+ "insert into root.vehicle.d1(timestamp,s3) values(33,true)",
+ "insert into root.test.d0(timestamp,s0) values(15,126)",
+ "insert into root.test.d0(timestamp,s0,s1) values(8,127,'128')",
+ "insert into root.test.d0(timestamp,s1) values(20,'129')",
+ "insert into root.test.d1.g0(timestamp,s0) values(14,430)",
+ "insert into root.test.d0(timestamp,s0) values(150,426)",
+ "insert into root.test.d0(timestamp,s0,s1) values(80,427,'528')",
+ "insert into root.test.d0(timestamp,s1) values(2,'1209')",
+ "insert into root.test.d1.g0(timestamp,s0) values(4,330)"};
+ private static final String[] createSQLs3 = {
+ "SET STORAGE GROUP TO root.iotdb",
+ "SET STORAGE GROUP TO root.cluster",
+ "CREATE TIMESERIES root.iotdb.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+ "CREATE TIMESERIES root.iotdb.d0.s1 WITH DATATYPE=TEXT, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.iotdb.d1.s2 WITH DATATYPE=FLOAT, ENCODING=RLE",
+ "CREATE TIMESERIES root.iotdb.d1.s3 WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.cluster.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+ "CREATE TIMESERIES root.cluster.d0.s1 WITH DATATYPE=TEXT, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.cluster.d1.g0.s0 WITH DATATYPE=INT32, ENCODING=RLE"
+ };
+ private static final String[] insertSQLs3 = {
+ "DELETE FROM root.vehicle WHERE time < 20",
+ "DELETE FROM root.test WHERE time < 20",
+ "insert into root.iotdb.d0(timestamp,s0) values(3,100)",
+ "insert into root.iotdb.d0(timestamp,s0,s1) values(22,101,'102')",
+ "insert into root.iotdb.d0(timestamp,s1) values(24,'103')",
+ "insert into root.iotdb.d1(timestamp,s2) values(21,104.0)",
+ "insert into root.iotdb.d1(timestamp,s2,s3) values(25,105.0,true)",
+ "insert into root.iotdb.d1(timestamp,s3) values(27,false)",
+ "insert into root.iotdb.d0(timestamp,s0) values(30,1000)",
+ "insert into root.iotdb.d0(timestamp,s0,s1) values(202,101,'102')",
+ "insert into root.iotdb.d0(timestamp,s1) values(44,'103')",
+ "insert into root.iotdb.d1(timestamp,s2) values(1,404.0)",
+ "insert into root.iotdb.d1(timestamp,s2,s3) values(250,10.0,true)",
+ "insert into root.iotdb.d1(timestamp,s3) values(207,false)",
+ "insert into root.cluster.d0(timestamp,s0) values(20,106)",
+ "insert into root.cluster.d0(timestamp,s0,s1) values(14,107,'108')",
+ "insert into root.cluster.d1.g0(timestamp,s0) values(1,110)",
+ "insert into root.cluster.d0(timestamp,s0) values(200,1006)",
+ "insert into root.cluster.d0(timestamp,s0,s1) values(1004,1007,'1080')",
+ "insert into root.cluster.d1.g0(timestamp,s0) values(1000,910)",
+ "insert into root.vehicle.d0(timestamp,s0) values(209,130)",
+ "insert into root.vehicle.d0(timestamp,s0,s1) values(206,131,'132')",
+ "insert into root.vehicle.d0(timestamp,s1) values(70,'33')",
+ "insert into root.vehicle.d1(timestamp,s2) values(204,14.0)",
+ "insert into root.vehicle.d1(timestamp,s2,s3) values(29,135.0,false)",
+ "insert into root.vehicle.d1(timestamp,s3) values(14,false)",
+ "insert into root.test.d0(timestamp,s0) values(19,136)",
+ "insert into root.test.d0(timestamp,s0,s1) values(7,137,'138')",
+ "insert into root.test.d0(timestamp,s1) values(30,'139')",
+ "insert into root.test.d1.g0(timestamp,s0) values(4,150)",
+ "insert into root.test.d0(timestamp,s0) values(1900,1316)",
+ "insert into root.test.d0(timestamp,s0,s1) values(700,1307,'1038')",
+ "insert into root.test.d0(timestamp,s1) values(3000,'1309')",
+ "insert into root.test.d1.g0(timestamp,s0) values(400,1050)"
+ };
+
+ private static final String[] querys1 ={
+ "select * from root.vehicle",
+ "select * from root.test",
+ "select * from root.vehicle,root.test where time = 11 or time = 12",
+ "select * from root.vehicle,root.test where d0.s0 > 10 and d0.s1 < 301 or time = 12",
+ "select * from root"
+ };
+ private static final String[] querys2 ={
+ "select * from root.vehicle",
+ "select * from root.test",
+ "select * from root.vehicle,root.test where time = 11 or time = 16",
+ "select * from root.vehicle,root.test where d0.s0 > 10 and d0.s1 < 301 or time = 20",
+ "select * from root"
+ };
+ private static final String[] querys3 ={
+ "select * from root.vehicle",
+ "select * from root.test",
+ "select * from root.cluster",
+ "select * from root.vehicle,root.test where time = 11 or time = 14",
+ "select * from root.vehicle,root.test where d0.s0 > 0 and d0.s1 < 1001 or time = 14",
+ "select * from root"
+ };
+
+ private Map<Integer, List<String>> queryCorrentResults = new HashMap<>();
+
+ @Before
+ public void setUp() throws Exception {
+ EnvironmentUtils.cleanEnv();
+ EnvironmentUtils.closeStatMonitor();
+ EnvironmentUtils.closeMemControl();
+ CLUSTER_CONFIG.createAllPath();
+ server = Server.getInstance();
+ server.start();
+ EnvironmentUtils.envSetUp();
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ server.stop();
+ EnvironmentUtils.cleanEnv();
+ }
+
+ private void initCorrectResults1(){
+ queryCorrentResults.clear();
+ queryCorrentResults.put(0, new ArrayList<>());
+ queryCorrentResults.put(1, new ArrayList<>());
+ queryCorrentResults.put(2, new ArrayList<>());
+ queryCorrentResults.put(3, new ArrayList<>());
+ queryCorrentResults.put(4, new ArrayList<>());
+ List<String> firstQueryRes = queryCorrentResults.get(0);
+ firstQueryRes.add("10,100,null,null,null");
+ firstQueryRes.add("11,null,null,104.0,null");
+ firstQueryRes.add("12,101,102,null,null");
+ firstQueryRes.add("15,null,null,105.0,true");
+ firstQueryRes.add("17,null,null,null,false");
+ firstQueryRes.add("19,null,103,null,null");
+ firstQueryRes.add("20,1000,null,null,null");
+ firstQueryRes.add("21,null,null,1004.0,null");
+ firstQueryRes.add("22,1001,1002,null,null");
+ firstQueryRes.add("25,null,null,1005.0,true");
+ firstQueryRes.add("27,null,null,null,true");
+ firstQueryRes.add("29,null,1003,null,null");
+
+ List<String> secondQueryRes = queryCorrentResults.get(1);
+ secondQueryRes.add("1,null,null,110");
+ secondQueryRes.add("10,106,null,1100");
+ secondQueryRes.add("14,107,108,null");
+ secondQueryRes.add("16,null,109,null");
+ secondQueryRes.add("30,1006,null,null");
+ secondQueryRes.add("34,1007,1008,null");
+ secondQueryRes.add("36,null,1090,null");
+
+ List<String> thirdQueryRes = queryCorrentResults.get(2);
+ thirdQueryRes.add("11,null,null,104.0,null,null,null,null");
+ thirdQueryRes.add("12,101,102,null,null,null,null,null");
+
+ List<String> forthQueryRes = queryCorrentResults.get(3);
+ forthQueryRes.add("12,101,102,null,null,null,null,null");
+
+ List<String> fifthQueryRes = queryCorrentResults.get(4);
+ fifthQueryRes.add("1,null,null,null,null,null,null,110");
+ fifthQueryRes.add("10,100,null,null,null,106,null,1100");
+ fifthQueryRes.add("11,null,null,104.0,null,null,null,null");
+ fifthQueryRes.add("12,101,102,null,null,null,null,null");
+ fifthQueryRes.add("14,null,null,null,null,107,108,null");
+ fifthQueryRes.add("15,null,null,105.0,true,null,null,null");
+ fifthQueryRes.add("16,null,null,null,null,null,109,null");
+ fifthQueryRes.add("17,null,null,null,false,null,null,null");
+ fifthQueryRes.add("19,null,103,null,null,null,null,null");
+ fifthQueryRes.add("20,1000,null,null,null,null,null,null");
+ fifthQueryRes.add("21,null,null,1004.0,null,null,null,null");
+ fifthQueryRes.add("22,1001,1002,null,null,null,null,null");
+ fifthQueryRes.add("25,null,null,1005.0,true,null,null,null");
+ fifthQueryRes.add("27,null,null,null,true,null,null,null");
+ fifthQueryRes.add("29,null,1003,null,null,null,null,null");
+ fifthQueryRes.add("30,null,null,null,null,1006,null,null");
+ fifthQueryRes.add("34,null,null,null,null,1007,1008,null");
+ fifthQueryRes.add("36,null,null,null,null,null,1090,null");
+ }
+
+ private void initCorrectResults2(){
+ queryCorrentResults.clear();
+ queryCorrentResults.put(0, new ArrayList<>());
+ queryCorrentResults.put(1, new ArrayList<>());
+ queryCorrentResults.put(2, new ArrayList<>());
+ queryCorrentResults.put(3, new ArrayList<>());
+ queryCorrentResults.put(4, new ArrayList<>());
+ List<String> firstQueryRes = queryCorrentResults.get(0);
+ firstQueryRes.add("6,120,null,null,null");
+ firstQueryRes.add("9,null,123,null,null");
+ firstQueryRes.add("10,100,null,null,null");
+ firstQueryRes.add("11,null,null,104.0,null");
+ firstQueryRes.add("12,101,102,null,null");
+ firstQueryRes.add("14,null,null,1024.0,null");
+ firstQueryRes.add("15,null,null,105.0,true");
+ firstQueryRes.add("16,128,null,null,null");
+ firstQueryRes.add("17,null,null,null,false");
+ firstQueryRes.add("18,189,198,null,null");
+ firstQueryRes.add("19,null,103,null,null");
+ firstQueryRes.add("20,1000,null,null,null");
+ firstQueryRes.add("21,null,null,1004.0,null");
+ firstQueryRes.add("22,1001,1002,null,null");
+ firstQueryRes.add("25,null,null,1005.0,true");
+ firstQueryRes.add("27,null,null,null,true");
+ firstQueryRes.add("29,null,1003,1205.0,true");
+ firstQueryRes.add("33,null,null,null,true");
+ firstQueryRes.add("38,121,122,null,null");
+ firstQueryRes.add("99,null,1234,null,null");
+
+ List<String> secondQueryRes = queryCorrentResults.get(1);
+ secondQueryRes.add("1,null,null,110");
+ secondQueryRes.add("2,null,1209,null");
+ secondQueryRes.add("4,null,null,330");
+ secondQueryRes.add("8,127,128,null");
+ secondQueryRes.add("10,106,null,1100");
+ secondQueryRes.add("14,107,108,430");
+ secondQueryRes.add("15,126,null,null");
+ secondQueryRes.add("16,null,109,null");
+ secondQueryRes.add("20,null,129,null");
+ secondQueryRes.add("30,1006,null,null");
+ secondQueryRes.add("34,1007,1008,null");
+ secondQueryRes.add("36,null,1090,null");
+ secondQueryRes.add("80,427,528,null");
+ secondQueryRes.add("150,426,null,null");
+
+ List<String> thirdQueryRes = queryCorrentResults.get(2);
+ thirdQueryRes.add("11,null,null,104.0,null,null,null,null");
+ thirdQueryRes.add("16,128,null,null,null,null,109,null");
+
+ List<String> forthQueryRes = queryCorrentResults.get(3);
+ forthQueryRes.add("20,1000,null,null,null,null,129,null");
+
+ List<String> fifthQueryRes = queryCorrentResults.get(4);
+ fifthQueryRes.add("1,null,null,null,null,null,null,110");
+ fifthQueryRes.add("2,null,null,null,null,null,1209,null");
+ fifthQueryRes.add("4,null,null,null,null,null,null,330");
+ fifthQueryRes.add("6,120,null,null,null,null,null,null");
+ fifthQueryRes.add("8,null,null,null,null,127,128,null");
+ fifthQueryRes.add("9,null,123,null,null,null,null,null");
+ fifthQueryRes.add("10,100,null,null,null,106,null,1100");
+ fifthQueryRes.add("11,null,null,104.0,null,null,null,null");
+ fifthQueryRes.add("12,101,102,null,null,null,null,null");
+ fifthQueryRes.add("14,null,null,1024.0,null,107,108,430");
+ fifthQueryRes.add("15,null,null,105.0,true,126,null,null");
+ fifthQueryRes.add("16,128,null,null,null,null,109,null");
+ fifthQueryRes.add("17,null,null,null,false,null,null,null");
+ fifthQueryRes.add("18,189,198,null,null,null,null,null");
+ fifthQueryRes.add("19,null,103,null,null,null,null,null");
+ fifthQueryRes.add("20,1000,null,null,null,null,129,null");
+ fifthQueryRes.add("21,null,null,1004.0,null,null,null,null");
+ fifthQueryRes.add("22,1001,1002,null,null,null,null,null");
+ fifthQueryRes.add("25,null,null,1005.0,true,null,null,null");
+ fifthQueryRes.add("27,null,null,null,true,null,null,null");
+ fifthQueryRes.add("29,null,1003,1205.0,true,null,null,null");
+ fifthQueryRes.add("30,null,null,null,null,1006,null,null");
+ fifthQueryRes.add("33,null,null,null,true,null,null,null");
+ fifthQueryRes.add("34,null,null,null,null,1007,1008,null");
+ fifthQueryRes.add("36,null,null,null,null,null,1090,null");
+ fifthQueryRes.add("38,121,122,null,null,null,null,null");
+ fifthQueryRes.add("80,null,null,null,null,427,528,null");
+ fifthQueryRes.add("99,null,1234,null,null,null,null,null");
+ fifthQueryRes.add("150,null,null,null,null,426,null,null");
+ }
+
+ private void initCorrectResults3(){
+ queryCorrentResults.clear();
+ queryCorrentResults.put(0, new ArrayList<>());
+ queryCorrentResults.put(1, new ArrayList<>());
+ queryCorrentResults.put(2, new ArrayList<>());
+ queryCorrentResults.put(3, new ArrayList<>());
+ queryCorrentResults.put(4, new ArrayList<>());
+ queryCorrentResults.put(5, new ArrayList<>());
+ List<String> zeroQueryRes = queryCorrentResults.get(0);
+ zeroQueryRes.add("14,null,null,null,false");
+ zeroQueryRes.add("20,1000,null,null,null");
+ zeroQueryRes.add("21,null,null,1004.0,null");
+ zeroQueryRes.add("22,1001,1002,null,null");
+ zeroQueryRes.add("25,null,null,1005.0,true");
+ zeroQueryRes.add("27,null,null,null,true");
+ zeroQueryRes.add("29,null,1003,135.0,false");
+ zeroQueryRes.add("33,null,null,null,true");
+ zeroQueryRes.add("38,121,122,null,null");
+ zeroQueryRes.add("70,null,33,null,null");
+ zeroQueryRes.add("99,null,1234,null,null");
+ zeroQueryRes.add("204,null,null,14.0,null");
+ zeroQueryRes.add("206,131,132,null,null");
+ zeroQueryRes.add("209,130,null,null,null");
+
+ List<String> firstQueryRes = queryCorrentResults.get(1);
+ firstQueryRes.add("4,null,null,150");
+ firstQueryRes.add("7,137,138,null");
+ firstQueryRes.add("19,136,null,null");
+ firstQueryRes.add("20,null,129,null");
+ firstQueryRes.add("30,1006,139,null");
+ firstQueryRes.add("34,1007,1008,null");
+ firstQueryRes.add("36,null,1090,null");
+ firstQueryRes.add("80,427,528,null");
+ firstQueryRes.add("150,426,null,null");
+ firstQueryRes.add("400,null,null,1050");
+ firstQueryRes.add("700,1307,1038,null");
+ firstQueryRes.add("1900,1316,null,null");
+ firstQueryRes.add("3000,null,1309,null");
+
+ List<String> secondQueryRes = queryCorrentResults.get(2);
+ secondQueryRes.add("1,null,null,110");
+ secondQueryRes.add("14,107,108,null");
+ secondQueryRes.add("20,106,null,null");
+ secondQueryRes.add("200,1006,null,null");
+ secondQueryRes.add("1000,null,null,910");
+ secondQueryRes.add("1004,1007,1080,null");
+
+ List<String> thirdQueryRes = queryCorrentResults.get(3);
+ thirdQueryRes.add("14,null,null,null,false,null,null,null");
+
+ List<String> forthQueryRes = queryCorrentResults.get(4);
+ forthQueryRes.add("14,null,null,null,false,null,null,null");
+
+ List<String> fifthQueryRes = queryCorrentResults.get(5);
+ fifthQueryRes.add("1,null,null,404.0,null,null,null,null,null,null,null,null,null,null,110");
+ fifthQueryRes.add("3,100,null,null,null,null,null,null,null,null,null,null,null,null,null");
+ fifthQueryRes.add("4,null,null,null,null,null,null,null,null,null,null,150,null,null,null");
+ fifthQueryRes.add("7,null,null,null,null,null,null,null,null,137,138,null,null,null,null");
+ fifthQueryRes.add("14,null,null,null,null,null,null,null,false,null,null,null,107,108,null");
+ fifthQueryRes.add("19,null,null,null,null,null,null,null,null,136,null,null,null,null,null");
+ fifthQueryRes.add("20,null,null,null,null,1000,null,null,null,null,129,null,106,null,null");
+ fifthQueryRes.add("21,null,null,104.0,null,null,null,1004.0,null,null,null,null,null,null,null");
+ fifthQueryRes.add("22,101,102,null,null,1001,1002,null,null,null,null,null,null,null,null");
+ fifthQueryRes.add("24,null,103,null,null,null,null,null,null,null,null,null,null,null,null");
+ fifthQueryRes.add("25,null,null,105.0,true,null,null,1005.0,true,null,null,null,null,null,null");
+ fifthQueryRes.add("27,null,null,null,false,null,null,null,true,null,null,null,null,null,null");
+ fifthQueryRes.add("29,null,null,null,null,null,1003,135.0,false,null,null,null,null,null,null");
+ fifthQueryRes.add("30,1000,null,null,null,null,null,null,null,1006,139,null,null,null,null");
+ fifthQueryRes.add("33,null,null,null,null,null,null,null,true,null,null,null,null,null,null");
+ fifthQueryRes.add("34,null,null,null,null,null,null,null,null,1007,1008,null,null,null,null");
+ fifthQueryRes.add("36,null,null,null,null,null,null,null,null,null,1090,null,null,null,null");
+ fifthQueryRes.add("38,null,null,null,null,121,122,null,null,null,null,null,null,null,null");
+ fifthQueryRes.add("44,null,103,null,null,null,null,null,null,null,null,null,null,null,null");
+ fifthQueryRes.add("70,null,null,null,null,null,33,null,null,null,null,null,null,null,null");
+ fifthQueryRes.add("80,null,null,null,null,null,null,null,null,427,528,null,null,null,null");
+ fifthQueryRes.add("99,null,null,null,null,null,1234,null,null,null,null,null,null,null,null");
+ fifthQueryRes.add("150,null,null,null,null,null,null,null,null,426,null,null,null,null,null");
+ fifthQueryRes.add("200,null,null,null,null,null,null,null,null,null,null,null,1006,null,null");
+ fifthQueryRes.add("202,101,102,null,null,null,null,null,null,null,null,null,null,null,null");
+ fifthQueryRes.add("204,null,null,null,null,null,null,14.0,null,null,null,null,null,null,null");
+ fifthQueryRes.add("206,null,null,null,null,131,132,null,null,null,null,null,null,null,null");
+ fifthQueryRes.add("207,null,null,null,false,null,null,null,null,null,null,null,null,null,null");
+ fifthQueryRes.add("209,null,null,null,null,130,null,null,null,null,null,null,null,null,null");
+ fifthQueryRes.add("250,null,null,10.0,true,null,null,null,null,null,null,null,null,null,null");
+ fifthQueryRes.add("400,null,null,null,null,null,null,null,null,null,null,1050,null,null,null");
+ fifthQueryRes.add("700,null,null,null,null,null,null,null,null,1307,1038,null,null,null,null");
+ fifthQueryRes.add("1000,null,null,null,null,null,null,null,null,null,null,null,null,null,910");
+ fifthQueryRes.add("1004,null,null,null,null,null,null,null,null,null,null,null,1007,1080,null");
+ fifthQueryRes.add("1900,null,null,null,null,null,null,null,null,1316,null,null,null,null,null");
+ fifthQueryRes.add("3000,null,null,null,null,null,null,null,null,null,1309,null,null,null,null");
+ }
+
+ @Test
+ public void testClusterQueryWithLargeData() throws Exception {
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ Statement statement = connection.createStatement();
+
+ //first round
+ insertData(connection, createSQLs1, insertSQLs1);
+ initCorrectResults1();
+ for(int i =0 ; i < querys1.length; i++) {
+ String queryStatement = querys1[i];
+ boolean hasResultSet = statement.execute(queryStatement);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ IoTDBResultMetadata resultSetMetaData = (IoTDBResultMetadata) resultSet.getMetaData();
+ int columnCount = resultSetMetaData.getColumnCount();
+ List<String> correctResult = queryCorrentResults.get(i);
+ int count = 0;
+ while (resultSet.next()) {
+ String correctRow = correctResult.get(count++);
+ StringBuilder rowRecordBuilder = new StringBuilder();
+ for (int j = 1; j < columnCount; j++) {
+ rowRecordBuilder.append(resultSet.getString(j)).append(",");
+ }
+ rowRecordBuilder.append(resultSet.getString(columnCount));
+ assertEquals(correctRow, rowRecordBuilder.toString());
+ }
+ }
+
+ // second round
+ insertData(connection, new String[]{}, insertSqls2);
+ initCorrectResults2();
+ for(int i =0 ; i < querys2.length; i++) {
+ String queryStatement = querys2[i];
+ boolean hasResultSet = statement.execute(queryStatement);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ IoTDBResultMetadata resultSetMetaData = (IoTDBResultMetadata) resultSet.getMetaData();
+ int columnCount = resultSetMetaData.getColumnCount();
+ List<String> correctResult = queryCorrentResults.get(i);
+ int count = 0;
+ while (resultSet.next()) {
+ String correctRow = correctResult.get(count++);
+ StringBuilder rowRecordBuilder = new StringBuilder();
+ for (int j = 1; j < columnCount; j++) {
+ rowRecordBuilder.append(resultSet.getString(j)).append(",");
+ }
+ rowRecordBuilder.append(resultSet.getString(columnCount));
+ assertEquals(correctRow, rowRecordBuilder.toString());
+ }
+ }
+
+ // third round
+ insertData(connection, createSQLs3, insertSQLs3);
+ initCorrectResults3();
+ for(int i =0 ; i < querys3.length; i++) {
+ String queryStatement = querys3[i];
+ boolean hasResultSet = statement.execute(queryStatement);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ IoTDBResultMetadata resultSetMetaData = (IoTDBResultMetadata) resultSet.getMetaData();
+ int columnCount = resultSetMetaData.getColumnCount();
+ List<String> correctResult = queryCorrentResults.get(i);
+ int count = 0;
+ while (resultSet.next()) {
+ String correctRow = correctResult.get(count++);
+ StringBuilder rowRecordBuilder = new StringBuilder();
+ for (int j = 1; j < columnCount; j++) {
+ rowRecordBuilder.append(resultSet.getString(j)).append(",");
+ }
+ rowRecordBuilder.append(resultSet.getString(columnCount));
+ assertEquals(correctRow, rowRecordBuilder.toString());
+ }
+ }
+
+ statement.close();
+ }
+ }
+}
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterQueryTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterQueryTest.java
new file mode 100644
index 0000000..f5cc295
--- /dev/null
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/query/ClusterQueryTest.java
@@ -0,0 +1,550 @@
+/**
+ * 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.cluster.query;
+
+import static org.apache.iotdb.cluster.utils.Utils.insertBatchData;
+import static org.apache.iotdb.cluster.utils.Utils.insertData;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.iotdb.cluster.config.ClusterConfig;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.entity.Server;
+import org.apache.iotdb.cluster.utils.EnvironmentUtils;
+import org.apache.iotdb.cluster.utils.QPExecutorUtils;
+import org.apache.iotdb.cluster.utils.hash.PhysicalNode;
+import org.apache.iotdb.jdbc.Config;
+import org.apache.iotdb.jdbc.IoTDBResultMetadata;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class ClusterQueryTest {
+
+ private Server server;
+ private static final ClusterConfig CLUSTER_CONFIG = ClusterDescriptor.getInstance().getConfig();
+ private static final PhysicalNode localNode = new PhysicalNode(CLUSTER_CONFIG.getIp(),
+ CLUSTER_CONFIG.getPort());
+
+ private static final String URL = "127.0.0.1:6667/";
+
+ private String[] createSQLs = {
+ "SET STORAGE GROUP TO root.vehicle",
+ "SET STORAGE GROUP TO root.test",
+ "CREATE TIMESERIES root.vehicle.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d0.s1 WITH DATATYPE=TEXT, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.vehicle.d1.s2 WITH DATATYPE=FLOAT, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d1.s3 WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.test.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+ "CREATE TIMESERIES root.test.d0.s1 WITH DATATYPE=TEXT, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.test.d1.g0.s0 WITH DATATYPE=INT32, ENCODING=RLE"
+
+ };
+ private String[] insertSQLs = {
+ "insert into root.vehicle.d0(timestamp,s0) values(10,100)",
+ "insert into root.vehicle.d0(timestamp,s0,s1) values(12,101,'102')",
+ "insert into root.vehicle.d0(timestamp,s1) values(19,'103')",
+ "insert into root.vehicle.d1(timestamp,s2) values(11,104.0)",
+ "insert into root.vehicle.d1(timestamp,s2,s3) values(15,105.0,true)",
+ "insert into root.vehicle.d1(timestamp,s3) values(17,false)",
+ "insert into root.vehicle.d0(timestamp,s0) values(20,1000)",
+ "insert into root.vehicle.d0(timestamp,s0,s1) values(22,1001,'1002')",
+ "insert into root.vehicle.d0(timestamp,s1) values(29,'1003')",
+ "insert into root.vehicle.d1(timestamp,s2) values(21,1004.0)",
+ "insert into root.vehicle.d1(timestamp,s2,s3) values(25,1005.0,true)",
+ "insert into root.vehicle.d1(timestamp,s3) values(27,true)",
+ "insert into root.test.d0(timestamp,s0) values(10,106)",
+ "insert into root.test.d0(timestamp,s0,s1) values(14,107,'108')",
+ "insert into root.test.d0(timestamp,s1) values(16,'109')",
+ "insert into root.test.d1.g0(timestamp,s0) values(1,110)",
+ "insert into root.test.d0(timestamp,s0) values(30,1006)",
+ "insert into root.test.d0(timestamp,s0,s1) values(34,1007,'1008')",
+ "insert into root.test.d0(timestamp,s1) values(36,'1090')",
+ "insert into root.test.d1.g0(timestamp,s0) values(10,1100)",
+ "insert into root.vehicle.d0(timestamp,s0) values(6,120)",
+ "insert into root.vehicle.d0(timestamp,s0,s1) values(38,121,'122')",
+ "insert into root.vehicle.d0(timestamp,s1) values(9,'123')",
+ "insert into root.vehicle.d0(timestamp,s0) values(16,128)",
+ "insert into root.vehicle.d0(timestamp,s0,s1) values(18,189,'198')",
+ "insert into root.vehicle.d0(timestamp,s1) values(99,'1234')",
+ "insert into root.vehicle.d1(timestamp,s2) values(14,1024.0)",
+ "insert into root.vehicle.d1(timestamp,s2,s3) values(29,1205.0,true)",
+ "insert into root.vehicle.d1(timestamp,s3) values(33,true)",
+ "insert into root.test.d0(timestamp,s0) values(15,126)",
+ "insert into root.test.d0(timestamp,s0,s1) values(8,127,'128')",
+ "insert into root.test.d0(timestamp,s1) values(20,'129')",
+ "insert into root.test.d1.g0(timestamp,s0) values(14,430)",
+ "insert into root.test.d0(timestamp,s0) values(150,426)",
+ "insert into root.test.d0(timestamp,s0,s1) values(80,427,'528')",
+ "insert into root.test.d0(timestamp,s1) values(2,'1209')",
+ "insert into root.test.d1.g0(timestamp,s0) values(4,330)"
+ };
+ private String[] queryStatementsWithoutFilter = {
+ "select * from root",
+ "select * from root.vehicle",
+ "select * from root.test",
+ "select vehicle.d0.s0, test.d0.s1 from root",
+ "select vehicle.d1.s2, vehicle.d1.s3 ,test.d1.g0.s0 from root"
+ };
+
+ private Map<Integer, List<String>> queryCorrentResultsWithoutFilter = new HashMap<>();
+
+ private String[] queryStatementsWithFilter = {
+ "select * from root.vehicle where d0.s0 > 10",
+ "select * from root.vehicle where d0.s0 < 101",
+ "select * from root.vehicle where d0.s0 > 10 and d0.s0 < 101 or time = 12",
+ "select * from root.test where d0.s0 > 10",
+ "select * from root.test where d0.s0 > 10 and d0.s0 < 200 or d0.s0 = 3",
+ "select * from root where vehicle.d0.s0 > 10 and test.d0.s0 < 101 or time = 20",
+ };
+
+ private Map<Integer, List<String>> queryCorrentResultsWithFilter = new HashMap<>();
+
+ @Before
+ public void setUp() throws Exception {
+ EnvironmentUtils.cleanEnv();
+ EnvironmentUtils.closeStatMonitor();
+ EnvironmentUtils.closeMemControl();
+ CLUSTER_CONFIG.createAllPath();
+ server = Server.getInstance();
+ server.start();
+ EnvironmentUtils.envSetUp();
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ server.stop();
+ EnvironmentUtils.cleanEnv();
+ }
+
+ private void initCorrectResultsWithoutFilter(){
+ queryCorrentResultsWithoutFilter.put(0, new ArrayList<>());
+ queryCorrentResultsWithoutFilter.put(1, new ArrayList<>());
+ queryCorrentResultsWithoutFilter.put(2, new ArrayList<>());
+ queryCorrentResultsWithoutFilter.put(3, new ArrayList<>());
+ queryCorrentResultsWithoutFilter.put(4, new ArrayList<>());
+ List<String> firstQueryRes = queryCorrentResultsWithoutFilter.get(0);
+ firstQueryRes.add("1,null,null,null,null,null,null,110");
+ firstQueryRes.add("2,null,null,null,null,null,1209,null");
+ firstQueryRes.add("4,null,null,null,null,null,null,330");
+ firstQueryRes.add("6,120,null,null,null,null,null,null");
+ firstQueryRes.add("8,null,null,null,null,127,128,null");
+ firstQueryRes.add("9,null,123,null,null,null,null,null");
+ firstQueryRes.add("10,100,null,null,null,106,null,1100");
+ firstQueryRes.add("11,null,null,104.0,null,null,null,null");
+ firstQueryRes.add("12,101,102,null,null,null,null,null");
+ firstQueryRes.add("14,null,null,1024.0,null,107,108,430");
+ firstQueryRes.add("15,null,null,105.0,true,126,null,null");
+ firstQueryRes.add("16,128,null,null,null,null,109,null");
+ firstQueryRes.add("17,null,null,null,false,null,null,null");
+ firstQueryRes.add("18,189,198,null,null,null,null,null");
+ firstQueryRes.add("19,null,103,null,null,null,null,null");
+ firstQueryRes.add("20,1000,null,null,null,null,129,null");
+ firstQueryRes.add("21,null,null,1004.0,null,null,null,null");
+ firstQueryRes.add("22,1001,1002,null,null,null,null,null");
+ firstQueryRes.add("25,null,null,1005.0,true,null,null,null");
+ firstQueryRes.add("27,null,null,null,true,null,null,null");
+ firstQueryRes.add("29,null,1003,1205.0,true,null,null,null");
+ firstQueryRes.add("30,null,null,null,null,1006,null,null");
+ firstQueryRes.add("33,null,null,null,true,null,null,null");
+ firstQueryRes.add("34,null,null,null,null,1007,1008,null");
+ firstQueryRes.add("36,null,null,null,null,null,1090,null");
+ firstQueryRes.add("38,121,122,null,null,null,null,null");
+ firstQueryRes.add("80,null,null,null,null,427,528,null");
+ firstQueryRes.add("99,null,1234,null,null,null,null,null");
+ firstQueryRes.add("150,null,null,null,null,426,null,null");
+
+ List<String> secondQueryRes = queryCorrentResultsWithoutFilter.get(1);
+ secondQueryRes.add("6,120,null,null,null");
+ secondQueryRes.add("9,null,123,null,null");
+ secondQueryRes.add("10,100,null,null,null");
+ secondQueryRes.add("11,null,null,104.0,null");
+ secondQueryRes.add("12,101,102,null,null");
+ secondQueryRes.add("14,null,null,1024.0,null");
+ secondQueryRes.add("15,null,null,105.0,true");
+ secondQueryRes.add("16,128,null,null,null");
+ secondQueryRes.add("17,null,null,null,false");
+ secondQueryRes.add("18,189,198,null,null");
+ secondQueryRes.add("19,null,103,null,null");
+ secondQueryRes.add("20,1000,null,null,null");
+ secondQueryRes.add("21,null,null,1004.0,null");
+ secondQueryRes.add("22,1001,1002,null,null");
+ secondQueryRes.add("25,null,null,1005.0,true");
+ secondQueryRes.add("27,null,null,null,true");
+ secondQueryRes.add("29,null,1003,1205.0,true");
+ secondQueryRes.add("33,null,null,null,true");
+ secondQueryRes.add("38,121,122,null,null");
+ secondQueryRes.add("99,null,1234,null,null");
+ List<String> thirdQueryRes = queryCorrentResultsWithoutFilter.get(2);
+ thirdQueryRes.add("1,null,null,110");
+ thirdQueryRes.add("2,null,1209,null");
+ thirdQueryRes.add("4,null,null,330");
+ thirdQueryRes.add("8,127,128,null");
+ thirdQueryRes.add("10,106,null,1100");
+ thirdQueryRes.add("14,107,108,430");
+ thirdQueryRes.add("15,126,null,null");
+ thirdQueryRes.add("16,null,109,null");
+ thirdQueryRes.add("20,null,129,null");
+ thirdQueryRes.add("30,1006,null,null");
+ thirdQueryRes.add("34,1007,1008,null");
+ thirdQueryRes.add("36,null,1090,null");
+ thirdQueryRes.add("80,427,528,null");
+ thirdQueryRes.add("150,426,null,null");
+ List<String> forthQueryRes = queryCorrentResultsWithoutFilter.get(3);
+ forthQueryRes.add("2,null,1209");
+ forthQueryRes.add("6,120,null");
+ forthQueryRes.add("8,null,128");
+ forthQueryRes.add("10,100,null");
+ forthQueryRes.add("12,101,null");
+ forthQueryRes.add("14,null,108");
+ forthQueryRes.add("16,128,109");
+ forthQueryRes.add("18,189,null");
+ forthQueryRes.add("20,1000,129");
+ forthQueryRes.add("22,1001,null");
+ forthQueryRes.add("34,null,1008");
+ forthQueryRes.add("36,null,1090");
+ forthQueryRes.add("38,121,null");
+ forthQueryRes.add("80,null,528");
+ List<String> fifthQueryRes = queryCorrentResultsWithoutFilter.get(4);
+ fifthQueryRes.add("1,null,null,110");
+ fifthQueryRes.add("4,null,null,330");
+ fifthQueryRes.add("10,null,null,1100");
+ fifthQueryRes.add("11,104.0,null,null");
+ fifthQueryRes.add("14,1024.0,null,430");
+ fifthQueryRes.add("15,105.0,true,null");
+ fifthQueryRes.add("17,null,false,null");
+ fifthQueryRes.add("21,1004.0,null,null");
+ fifthQueryRes.add("25,1005.0,true,null");
+ fifthQueryRes.add("27,null,true,null");
+ fifthQueryRes.add("29,1205.0,true,null");
+ fifthQueryRes.add("33,null,true,null");
+ }
+
+ private void initCorrectResultsWithFilter(){
+ queryCorrentResultsWithFilter.put(0, new ArrayList<>());
+ queryCorrentResultsWithFilter.put(1, new ArrayList<>());
+ queryCorrentResultsWithFilter.put(2, new ArrayList<>());
+ queryCorrentResultsWithFilter.put(3, new ArrayList<>());
+ queryCorrentResultsWithFilter.put(4, new ArrayList<>());
+ queryCorrentResultsWithFilter.put(5, new ArrayList<>());
+ List<String> firstQueryRes = queryCorrentResultsWithFilter.get(0);
+ firstQueryRes.add("6,120,null,null,null");
+ firstQueryRes.add("10,100,null,null,null");
+ firstQueryRes.add("12,101,102,null,null");
+ firstQueryRes.add("16,128,null,null,null");
+ firstQueryRes.add("18,189,198,null,null");
+ firstQueryRes.add("20,1000,null,null,null");
+ firstQueryRes.add("22,1001,1002,null,null");
+ firstQueryRes.add("38,121,122,null,null");
+
+ List<String> secondQueryRes = queryCorrentResultsWithFilter.get(1);
+ secondQueryRes.add("10,100,null,null,null");
+ List<String> thirdQueryRes = queryCorrentResultsWithFilter.get(2);
+ thirdQueryRes.add("10,100,null,null,null");
+ thirdQueryRes.add("12,101,102,null,null");
+ List<String> forthQueryRes = queryCorrentResultsWithFilter.get(3);
+ forthQueryRes.add("8,127,128,null");
+ forthQueryRes.add("10,106,null,1100");
+ forthQueryRes.add("14,107,108,430");
+ forthQueryRes.add("15,126,null,null");
+ forthQueryRes.add("30,1006,null,null");
+ forthQueryRes.add("34,1007,1008,null");
+ forthQueryRes.add("80,427,528,null");
+ forthQueryRes.add("150,426,null,null");
+ List<String> fifthQueryRes = queryCorrentResultsWithFilter.get(4);
+ fifthQueryRes.add("8,127,128,null");
+ fifthQueryRes.add("10,106,null,1100");
+ fifthQueryRes.add("14,107,108,430");
+ fifthQueryRes.add("15,126,null,null");
+ List<String> sixthQueryRes = queryCorrentResultsWithFilter.get(5);
+ sixthQueryRes.add("20,1000,null,null,null,null,129,null");
+ }
+
+ @Test
+ public void testLocalQueryWithoutFilter() throws Exception {
+ initCorrectResultsWithoutFilter();
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+
+ for(int i =0 ; i < queryStatementsWithoutFilter.length; i++) {
+ String queryStatement = queryStatementsWithoutFilter[i];
+ boolean hasResultSet = statement.execute(queryStatement);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ IoTDBResultMetadata resultSetMetaData = (IoTDBResultMetadata) resultSet.getMetaData();
+ int columnCount = resultSetMetaData.getColumnCount();
+ List<String> correctResult = queryCorrentResultsWithoutFilter.get(i);
+ int count = 0;
+ while (resultSet.next()) {
+ String correctRow = correctResult.get(count++);
+ StringBuilder rowRecordBuilder = new StringBuilder();
+ for (int j = 1; j < columnCount; j++) {
+ rowRecordBuilder.append(resultSet.getString(j)).append(",");
+ }
+ rowRecordBuilder.append(resultSet.getString(columnCount));
+ assertEquals(correctRow, rowRecordBuilder.toString());
+ }
+ }
+ statement.close();
+ }
+ }
+
+ @Test
+ public void testLocalQueryWithFilter() throws Exception {
+
+ initCorrectResultsWithFilter();
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+
+ for(int i =0 ; i < queryStatementsWithFilter.length; i++) {
+ String queryStatement = queryStatementsWithFilter[i];
+ boolean hasResultSet = statement.execute(queryStatement);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ IoTDBResultMetadata resultSetMetaData = (IoTDBResultMetadata) resultSet.getMetaData();
+ int columnCount = resultSetMetaData.getColumnCount();
+ List<String> correctResult = queryCorrentResultsWithFilter.get(i);
+ int count = 0;
+ while (resultSet.next()) {
+ String correctRow = correctResult.get(count++);
+ StringBuilder rowRecordBuilder = new StringBuilder();
+ for (int j = 1; j < columnCount; j++) {
+ rowRecordBuilder.append(resultSet.getString(j)).append(",");
+ }
+ rowRecordBuilder.append(resultSet.getString(columnCount));
+ assertEquals(correctRow, rowRecordBuilder.toString());
+ }
+ }
+ statement.close();
+ }
+ }
+
+ @Test
+ public void testRemoteQueryWithoutFilter() throws Exception {
+
+ QPExecutorUtils.setLocalNodeAddr("0.0.0.0", 0);
+ initCorrectResultsWithoutFilter();
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+
+ for(int i =0 ; i < queryStatementsWithoutFilter.length; i++) {
+ String queryStatement = queryStatementsWithoutFilter[i];
+ boolean hasResultSet = statement.execute(queryStatement);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ IoTDBResultMetadata resultSetMetaData = (IoTDBResultMetadata) resultSet.getMetaData();
+ int columnCount = resultSetMetaData.getColumnCount();
+ List<String> correctResult = queryCorrentResultsWithoutFilter.get(i);
+ int count = 0;
+ while (resultSet.next()) {
+ String correctRow = correctResult.get(count++);
+ StringBuilder rowRecordBuilder = new StringBuilder();
+ for (int j = 1; j < columnCount; j++) {
+ rowRecordBuilder.append(resultSet.getString(j)).append(",");
+ }
+ rowRecordBuilder.append(resultSet.getString(columnCount));
+ assertEquals(correctRow, rowRecordBuilder.toString());
+ }
+ }
+ QPExecutorUtils.setLocalNodeAddr(localNode.getIp(), localNode.getPort());
+ statement.close();
+ }
+ }
+
+ @Test
+ public void testRemoteQueryWithFilter() throws Exception {
+
+ QPExecutorUtils.setLocalNodeAddr("0.0.0.0", 0);
+ initCorrectResultsWithFilter();
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+
+ for(int i =0 ; i < queryStatementsWithFilter.length; i++) {
+ String queryStatement = queryStatementsWithFilter[i];
+ boolean hasResultSet = statement.execute(queryStatement);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ IoTDBResultMetadata resultSetMetaData = (IoTDBResultMetadata) resultSet.getMetaData();
+ int columnCount = resultSetMetaData.getColumnCount();
+ List<String> correctResult = queryCorrentResultsWithFilter.get(i);
+ int count = 0;
+ while (resultSet.next()) {
+ String correctRow = correctResult.get(count++);
+ StringBuilder rowRecordBuilder = new StringBuilder();
+ for (int j = 1; j < columnCount; j++) {
+ rowRecordBuilder.append(resultSet.getString(j)).append(",");
+ }
+ rowRecordBuilder.append(resultSet.getString(columnCount));
+ assertEquals(correctRow, rowRecordBuilder.toString());
+ }
+ }
+ QPExecutorUtils.setLocalNodeAddr(localNode.getIp(), localNode.getPort());
+ statement.close();
+ }
+ }
+
+ @Test
+ public void testLocalQueryWithoutFilterByBatch() throws Exception {
+ initCorrectResultsWithoutFilter();
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertBatchData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+
+ for(int i =0 ; i < queryStatementsWithoutFilter.length; i++) {
+ String queryStatement = queryStatementsWithoutFilter[i];
+ boolean hasResultSet = statement.execute(queryStatement);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ IoTDBResultMetadata resultSetMetaData = (IoTDBResultMetadata) resultSet.getMetaData();
+ int columnCount = resultSetMetaData.getColumnCount();
+ List<String> correctResult = queryCorrentResultsWithoutFilter.get(i);
+ int count = 0;
+ while (resultSet.next()) {
+ String correctRow = correctResult.get(count++);
+ StringBuilder rowRecordBuilder = new StringBuilder();
+ for (int j = 1; j < columnCount; j++) {
+ rowRecordBuilder.append(resultSet.getString(j)).append(",");
+ }
+ rowRecordBuilder.append(resultSet.getString(columnCount));
+ assertEquals(correctRow, rowRecordBuilder.toString());
+ }
+ }
+ statement.close();
+ }
+ }
+
+ @Test
+ public void testLocalQueryWithFilterByBatch() throws Exception {
+
+ initCorrectResultsWithFilter();
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertBatchData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+
+ for(int i =0 ; i < queryStatementsWithFilter.length; i++) {
+ String queryStatement = queryStatementsWithFilter[i];
+ boolean hasResultSet = statement.execute(queryStatement);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ IoTDBResultMetadata resultSetMetaData = (IoTDBResultMetadata) resultSet.getMetaData();
+ int columnCount = resultSetMetaData.getColumnCount();
+ List<String> correctResult = queryCorrentResultsWithFilter.get(i);
+ int count = 0;
+ while (resultSet.next()) {
+ String correctRow = correctResult.get(count++);
+ StringBuilder rowRecordBuilder = new StringBuilder();
+ for (int j = 1; j < columnCount; j++) {
+ rowRecordBuilder.append(resultSet.getString(j)).append(",");
+ }
+ rowRecordBuilder.append(resultSet.getString(columnCount));
+ assertEquals(correctRow, rowRecordBuilder.toString());
+ }
+ }
+ statement.close();
+ }
+ }
+
+ @Test
+ public void testRemoteQueryWithoutFilterByBatch() throws Exception {
+
+ QPExecutorUtils.setLocalNodeAddr("0.0.0.0", 0);
+ initCorrectResultsWithoutFilter();
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertBatchData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+
+ for(int i =0 ; i < queryStatementsWithoutFilter.length; i++) {
+ String queryStatement = queryStatementsWithoutFilter[i];
+ boolean hasResultSet = statement.execute(queryStatement);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ IoTDBResultMetadata resultSetMetaData = (IoTDBResultMetadata) resultSet.getMetaData();
+ int columnCount = resultSetMetaData.getColumnCount();
+ List<String> correctResult = queryCorrentResultsWithoutFilter.get(i);
+ int count = 0;
+ while (resultSet.next()) {
+ String correctRow = correctResult.get(count++);
+ StringBuilder rowRecordBuilder = new StringBuilder();
+ for (int j = 1; j < columnCount; j++) {
+ rowRecordBuilder.append(resultSet.getString(j)).append(",");
+ }
+ rowRecordBuilder.append(resultSet.getString(columnCount));
+ assertEquals(correctRow, rowRecordBuilder.toString());
+ }
+ }
+ QPExecutorUtils.setLocalNodeAddr(localNode.getIp(), localNode.getPort());
+ statement.close();
+ }
+ }
+
+ @Test
+ public void testRemoteQueryWithFilterByBatch() throws Exception {
+
+ QPExecutorUtils.setLocalNodeAddr("0.0.0.0", 0);
+ initCorrectResultsWithFilter();
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertBatchData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+
+ for(int i =0 ; i < queryStatementsWithFilter.length; i++) {
+ String queryStatement = queryStatementsWithFilter[i];
+ boolean hasResultSet = statement.execute(queryStatement);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ IoTDBResultMetadata resultSetMetaData = (IoTDBResultMetadata) resultSet.getMetaData();
+ int columnCount = resultSetMetaData.getColumnCount();
+ List<String> correctResult = queryCorrentResultsWithFilter.get(i);
+ int count = 0;
+ while (resultSet.next()) {
+ String correctRow = correctResult.get(count++);
+ StringBuilder rowRecordBuilder = new StringBuilder();
+ for (int j = 1; j < columnCount; j++) {
+ rowRecordBuilder.append(resultSet.getString(j)).append(",");
+ }
+ rowRecordBuilder.append(resultSet.getString(columnCount));
+ assertEquals(correctRow, rowRecordBuilder.toString());
+ }
+ }
+ QPExecutorUtils.setLocalNodeAddr(localNode.getIp(), localNode.getPort());
+ statement.close();
+ }
+ }
+
+}
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/query/manager/ClusterLocalManagerTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/query/manager/ClusterLocalManagerTest.java
new file mode 100644
index 0000000..c09aaa5
--- /dev/null
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/query/manager/ClusterLocalManagerTest.java
@@ -0,0 +1,406 @@
+/**
+ * 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.cluster.query.manager;
+
+import static org.apache.iotdb.cluster.utils.Utils.insertData;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.iotdb.cluster.config.ClusterConfig;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.entity.Server;
+import org.apache.iotdb.cluster.query.manager.querynode.ClusterLocalQueryManager;
+import org.apache.iotdb.cluster.query.manager.querynode.ClusterLocalSingleQueryManager;
+import org.apache.iotdb.cluster.query.reader.querynode.ClusterBatchReaderByTimestamp;
+import org.apache.iotdb.cluster.query.reader.querynode.ClusterBatchReaderWithoutTimeGenerator;
+import org.apache.iotdb.cluster.query.reader.querynode.ClusterFilterSeriesBatchReader;
+import org.apache.iotdb.cluster.query.reader.querynode.AbstractClusterBatchReader;
+import org.apache.iotdb.cluster.utils.EnvironmentUtils;
+import org.apache.iotdb.cluster.utils.QPExecutorUtils;
+import org.apache.iotdb.cluster.utils.hash.PhysicalNode;
+import org.apache.iotdb.jdbc.Config;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class ClusterLocalManagerTest {
+
+ private Server server;
+ private static final ClusterConfig CLUSTER_CONFIG = ClusterDescriptor.getInstance().getConfig();
+ private static ClusterLocalQueryManager manager = ClusterLocalQueryManager.getInstance();
+ private static final PhysicalNode localNode = new PhysicalNode(CLUSTER_CONFIG.getIp(),
+ CLUSTER_CONFIG.getPort());
+ private static final String URL = "127.0.0.1:6667/";
+
+ private String[] createSQLs = {
+ "set storage group to root.vehicle",
+ "CREATE TIMESERIES root.vehicle.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d0.s1 WITH DATATYPE=TEXT, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.vehicle.d0.s3 WITH DATATYPE=TEXT, ENCODING=PLAIN"
+ };
+ private String[] insertSQLs = {
+ "insert into root.vehicle.d0(timestamp,s0) values(10,100)",
+ "insert into root.vehicle.d0(timestamp,s0,s1) values(12,101,'102')",
+ "insert into root.vehicle.d0(timestamp,s3) values(19,'103')",
+ "insert into root.vehicle.d0(timestamp,s0,s1) values(22,1031,'3102')",
+ "insert into root.vehicle.d0(timestamp,s1) values(192,'1033')"
+ };
+ private String queryStatementsWithoutFilter = "select * from root.vehicle";
+ private String queryStatementsWithFilter = "select * from root.vehicle where d0.s0 > 10 and d0.s0 < 101 or d0.s0 = 3";
+
+ @Before
+ public void setUp() throws Exception {
+ EnvironmentUtils.cleanEnv();
+ EnvironmentUtils.closeStatMonitor();
+ EnvironmentUtils.closeMemControl();
+ QPExecutorUtils.setLocalNodeAddr("0.0.0.0", 0);
+ CLUSTER_CONFIG.createAllPath();
+ server = Server.getInstance();
+ server.start();
+ EnvironmentUtils.envSetUp();
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ server.stop();
+ QPExecutorUtils.setLocalNodeAddr(localNode.getIp(), localNode.getPort());
+ EnvironmentUtils.cleanEnv();
+ }
+
+ @Test
+ public void testClusterLocalQueryManagerWithoutFilter() throws Exception {
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+
+ // first query
+ boolean hasResultSet = statement.execute(queryStatementsWithoutFilter);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ ConcurrentHashMap<String, Long> map = ClusterLocalQueryManager.getTaskIdMapJobId();
+ assertEquals(1, map.size());
+ for (String taskId : map.keySet()) {
+ assertNotNull(manager.getSingleQuery(taskId));
+ }
+
+ // second query
+ hasResultSet = statement.execute(queryStatementsWithoutFilter);
+ assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ map = ClusterLocalQueryManager.getTaskIdMapJobId();
+ assertEquals(2, map.size());
+ for (String taskId : map.keySet()) {
+ assertNotNull(manager.getSingleQuery(taskId));
+ }
+
+ // third query
+ hasResultSet = statement.execute(queryStatementsWithoutFilter);
+ assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ map = ClusterLocalQueryManager.getTaskIdMapJobId();
+ assertEquals(3, map.size());
+ for (String taskId : map.keySet()) {
+ assertNotNull(manager.getSingleQuery(taskId));
+ }
+ statement.close();
+ }
+ }
+
+ @Test
+ public void testClusterLocalQueryManagerWithFilter() throws Exception {
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+
+ // first query
+ boolean hasResultSet = statement.execute(queryStatementsWithFilter);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ assertEquals(10, resultSet.getLong(1));
+ assertEquals(100, resultSet.getInt(2));
+ assertNull(resultSet.getString(3));
+ assertNull(resultSet.getString(4));
+ ConcurrentHashMap<String, Long> map = ClusterLocalQueryManager.getTaskIdMapJobId();
+ assertEquals(1, map.size());
+ for (String taskId : map.keySet()) {
+ assertNotNull(manager.getSingleQuery(taskId));
+ }
+ assertFalse(resultSet.next());
+
+ // second query
+ hasResultSet = statement.execute(queryStatementsWithFilter);
+ assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ assertEquals(10, resultSet.getLong(1));
+ assertEquals(100, resultSet.getInt(2));
+ assertNull(resultSet.getString(3));
+ assertNull(resultSet.getString(4));
+ map = ClusterLocalQueryManager.getTaskIdMapJobId();
+ assertEquals(2, map.size());
+ for (String taskId : map.keySet()) {
+ assertNotNull(manager.getSingleQuery(taskId));
+ }
+ assertFalse(resultSet.next());
+
+ // third query
+ hasResultSet = statement.execute(queryStatementsWithFilter);
+ assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ assertEquals(10, resultSet.getLong(1));
+ assertEquals(100, resultSet.getInt(2));
+ assertNull(resultSet.getString(3));
+ assertNull(resultSet.getString(4));
+ map = ClusterLocalQueryManager.getTaskIdMapJobId();
+ assertEquals(3, map.size());
+ for (String taskId : map.keySet()) {
+ assertNotNull(manager.getSingleQuery(taskId));
+ }
+ assertFalse(resultSet.next());
+
+ statement.close();
+ }
+ }
+
+ @Test
+ public void testClusterLocalSingleQueryWithoutFilterManager() throws Exception {
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+
+ // first query
+ boolean hasResultSet = statement.execute(queryStatementsWithoutFilter);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ ConcurrentHashMap<String, Long> map = ClusterLocalQueryManager.getTaskIdMapJobId();
+ assertEquals(1, map.size());
+ for (String taskId : map.keySet()) {
+ ClusterLocalSingleQueryManager singleQueryManager = manager.getSingleQuery(taskId);
+ assertNotNull(singleQueryManager);
+ assertEquals((long) map.get(taskId), singleQueryManager.getJobId());
+ assertEquals(0, singleQueryManager.getQueryRound());
+ assertNull(singleQueryManager.getFilterReader());
+ Map<String, AbstractClusterBatchReader> selectSeriesReaders = singleQueryManager
+ .getSelectSeriesReaders();
+ assertEquals(3, selectSeriesReaders.size());
+ Map<String, TSDataType> typeMap = singleQueryManager.getDataTypeMap();
+ for (Entry<String, AbstractClusterBatchReader> entry : selectSeriesReaders.entrySet()) {
+ String path = entry.getKey();
+ TSDataType dataType = typeMap.get(path);
+ AbstractClusterBatchReader clusterBatchReader = entry.getValue();
+ assertNotNull(((ClusterBatchReaderWithoutTimeGenerator) clusterBatchReader).getReader());
+ assertEquals(dataType,
+ ((ClusterBatchReaderWithoutTimeGenerator) clusterBatchReader).getDataType());
+ }
+ }
+
+ // second query
+ hasResultSet = statement.execute(queryStatementsWithoutFilter);
+ assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ map = ClusterLocalQueryManager.getTaskIdMapJobId();
+ assertEquals(2, map.size());
+ for (String taskId : map.keySet()) {
+ ClusterLocalSingleQueryManager singleQueryManager = manager.getSingleQuery(taskId);
+ assertNotNull(singleQueryManager);
+ assertEquals((long) map.get(taskId), singleQueryManager.getJobId());
+ assertEquals(0, singleQueryManager.getQueryRound());
+ assertNull(singleQueryManager.getFilterReader());
+ Map<String, AbstractClusterBatchReader> selectSeriesReaders = singleQueryManager
+ .getSelectSeriesReaders();
+ assertEquals(3, selectSeriesReaders.size());
+ Map<String, TSDataType> typeMap = singleQueryManager.getDataTypeMap();
+ for (Entry<String, AbstractClusterBatchReader> entry : selectSeriesReaders.entrySet()) {
+ String path = entry.getKey();
+ TSDataType dataType = typeMap.get(path);
+ AbstractClusterBatchReader clusterBatchReader = entry.getValue();
+ assertNotNull(((ClusterBatchReaderWithoutTimeGenerator) clusterBatchReader).getReader());
+ assertEquals(dataType,
+ ((ClusterBatchReaderWithoutTimeGenerator) clusterBatchReader).getDataType());
+ }
+ }
+
+ // third query
+ hasResultSet = statement.execute(queryStatementsWithoutFilter);
+ assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ map = ClusterLocalQueryManager.getTaskIdMapJobId();
+ assertEquals(3, map.size());
+ for (String taskId : map.keySet()) {
+ ClusterLocalSingleQueryManager singleQueryManager = manager.getSingleQuery(taskId);
+ assertNotNull(singleQueryManager);
+ assertEquals((long) map.get(taskId), singleQueryManager.getJobId());
+ assertEquals(0, singleQueryManager.getQueryRound());
+ assertNull(singleQueryManager.getFilterReader());
+ Map<String, AbstractClusterBatchReader> selectSeriesReaders = singleQueryManager
+ .getSelectSeriesReaders();
+ assertEquals(3, selectSeriesReaders.size());
+ Map<String, TSDataType> typeMap = singleQueryManager.getDataTypeMap();
+ for (Entry<String, AbstractClusterBatchReader> entry : selectSeriesReaders.entrySet()) {
+ String path = entry.getKey();
+ TSDataType dataType = typeMap.get(path);
+ AbstractClusterBatchReader clusterBatchReader = entry.getValue();
+ assertNotNull(((ClusterBatchReaderWithoutTimeGenerator) clusterBatchReader).getReader());
+ assertEquals(dataType,
+ ((ClusterBatchReaderWithoutTimeGenerator) clusterBatchReader).getDataType());
+ }
+ }
+ statement.close();
+ }
+ }
+
+ @Test
+ public void testClusterLocalSingleQueryWithFilterManager() throws Exception {
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+
+ // first query
+ boolean hasResultSet = statement.execute(queryStatementsWithFilter);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ ConcurrentHashMap<String, Long> map = ClusterLocalQueryManager.getTaskIdMapJobId();
+ assertEquals(1, map.size());
+ for (String taskId : map.keySet()) {
+ ClusterLocalSingleQueryManager singleQueryManager = manager.getSingleQuery(taskId);
+ assertNotNull(singleQueryManager);
+ assertEquals((long) map.get(taskId), singleQueryManager.getJobId());
+ assertEquals(3, singleQueryManager.getQueryRound());
+ ClusterFilterSeriesBatchReader filterReader = (ClusterFilterSeriesBatchReader) singleQueryManager.getFilterReader();
+ assertNotNull(filterReader);
+ List<Path> allFilterPaths = new ArrayList<>();
+ allFilterPaths.add(new Path("root.vehicle.d0.s0"));
+ assertTrue(allFilterPaths.containsAll(filterReader.getAllFilterPath()));
+ assertNotNull(filterReader.getQueryDataSet());
+
+ Map<String, AbstractClusterBatchReader> selectSeriesReaders = singleQueryManager
+ .getSelectSeriesReaders();
+ assertNotNull(selectSeriesReaders);
+ assertEquals(3, selectSeriesReaders.size());
+ Map<String, TSDataType> typeMap = singleQueryManager.getDataTypeMap();
+ for (Entry<String, AbstractClusterBatchReader> entry : selectSeriesReaders.entrySet()) {
+ String path = entry.getKey();
+ TSDataType dataType = typeMap.get(path);
+ AbstractClusterBatchReader clusterBatchReader = entry.getValue();
+ assertNotNull(((ClusterBatchReaderByTimestamp) clusterBatchReader).getReaderByTimeStamp());
+ assertEquals(dataType,
+ ((ClusterBatchReaderByTimestamp) clusterBatchReader).getDataType());
+ }
+ }
+
+ // second query
+ hasResultSet = statement.execute(queryStatementsWithFilter);
+ assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ map = ClusterLocalQueryManager.getTaskIdMapJobId();
+ assertEquals(2, map.size());
+ for (String taskId : map.keySet()) {
+ ClusterLocalSingleQueryManager singleQueryManager = manager.getSingleQuery(taskId);
+ assertNotNull(singleQueryManager);
+ assertEquals((long) map.get(taskId), singleQueryManager.getJobId());
+ assertEquals(3, singleQueryManager.getQueryRound());
+ ClusterFilterSeriesBatchReader filterReader = (ClusterFilterSeriesBatchReader) singleQueryManager.getFilterReader();
+ assertNotNull(filterReader);
+ List<Path> allFilterPaths = new ArrayList<>();
+ allFilterPaths.add(new Path("root.vehicle.d0.s0"));
+ assertTrue(allFilterPaths.containsAll(filterReader.getAllFilterPath()));
+ assertNotNull(filterReader.getQueryDataSet());
+
+ Map<String, AbstractClusterBatchReader> selectSeriesReaders = singleQueryManager
+ .getSelectSeriesReaders();
+ assertNotNull(selectSeriesReaders);
+ assertEquals(3, selectSeriesReaders.size());
+ Map<String, TSDataType> typeMap = singleQueryManager.getDataTypeMap();
+ for (Entry<String, AbstractClusterBatchReader> entry : selectSeriesReaders.entrySet()) {
+ String path = entry.getKey();
+ TSDataType dataType = typeMap.get(path);
+ AbstractClusterBatchReader clusterBatchReader = entry.getValue();
+ assertNotNull(((ClusterBatchReaderByTimestamp) clusterBatchReader).getReaderByTimeStamp());
+ assertEquals(dataType,
+ ((ClusterBatchReaderByTimestamp) clusterBatchReader).getDataType());
+ }
+ }
+
+ // third query
+ hasResultSet = statement.execute(queryStatementsWithFilter);
+ assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ map = ClusterLocalQueryManager.getTaskIdMapJobId();
+ assertEquals(3, map.size());
+ for (String taskId : map.keySet()) {
+ ClusterLocalSingleQueryManager singleQueryManager = manager.getSingleQuery(taskId);
+ assertNotNull(singleQueryManager);
+ assertEquals((long) map.get(taskId), singleQueryManager.getJobId());
+ assertEquals(3, singleQueryManager.getQueryRound());
+ ClusterFilterSeriesBatchReader filterReader = (ClusterFilterSeriesBatchReader) singleQueryManager.getFilterReader();
+ assertNotNull(filterReader);
+ List<Path> allFilterPaths = new ArrayList<>();
+ allFilterPaths.add(new Path("root.vehicle.d0.s0"));
+ assertTrue(allFilterPaths.containsAll(filterReader.getAllFilterPath()));
+ assertNotNull(filterReader.getQueryDataSet());
+
+ Map<String, AbstractClusterBatchReader> selectSeriesReaders = singleQueryManager
+ .getSelectSeriesReaders();
+ assertNotNull(selectSeriesReaders);
+ assertEquals(3, selectSeriesReaders.size());
+ Map<String, TSDataType> typeMap = singleQueryManager.getDataTypeMap();
+ for (Entry<String, AbstractClusterBatchReader> entry : selectSeriesReaders.entrySet()) {
+ String path = entry.getKey();
+ TSDataType dataType = typeMap.get(path);
+ AbstractClusterBatchReader clusterBatchReader = entry.getValue();
+ assertNotNull(((ClusterBatchReaderByTimestamp) clusterBatchReader).getReaderByTimeStamp());
+ assertEquals(dataType,
+ ((ClusterBatchReaderByTimestamp) clusterBatchReader).getDataType());
+ }
+ }
+ statement.close();
+ }
+ }
+
+}
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/query/manager/ClusterRpcManagerTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/query/manager/ClusterRpcManagerTest.java
new file mode 100644
index 0000000..b800fbf
--- /dev/null
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/query/manager/ClusterRpcManagerTest.java
@@ -0,0 +1,334 @@
+/**
+ * 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.cluster.query.manager;
+
+import static org.apache.iotdb.cluster.utils.Utils.insertData;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.iotdb.cluster.config.ClusterConfig;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.entity.Server;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcQueryManager;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcSingleQueryManager;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.FilterGroupEntity;
+import org.apache.iotdb.cluster.utils.EnvironmentUtils;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.jdbc.Config;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class ClusterRpcManagerTest {
+
+
+ private Server server;
+ private static final ClusterConfig CLUSTER_CONFIG = ClusterDescriptor.getInstance().getConfig();
+ private static final String LOCAL_ADDR = String
+ .format("%s:%d", CLUSTER_CONFIG.getIp(), CLUSTER_CONFIG.getPort());
+ private static ClusterRpcQueryManager manager = ClusterRpcQueryManager.getInstance();
+
+ private static final String URL = "127.0.0.1:6667/";
+
+ private String[] createSQLs = {
+ "set storage group to root.vehicle",
+ "CREATE TIMESERIES root.vehicle.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d0.s1 WITH DATATYPE=TEXT, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.vehicle.d0.s3 WITH DATATYPE=TEXT, ENCODING=PLAIN"
+ };
+ private String[] insertSQLs = {
+ "insert into root.vehicle.d0(timestamp,s0) values(10,100)",
+ "insert into root.vehicle.d0(timestamp,s0,s1) values(12,101,'102')",
+ "insert into root.vehicle.d0(timestamp,s3) values(19,'103')",
+ "insert into root.vehicle.d0(timestamp,s0,s1) values(22,1031,'3102')",
+ "insert into root.vehicle.d0(timestamp,s1) values(192,'1033')"
+ };
+ private String queryStatementsWithoutFilter = "select * from root.vehicle";
+ private String queryStatementsWithFilter = "select * from root.vehicle where d0.s0 > 10 and d0.s0 < 101 or d0.s0 = 3";
+
+ @Before
+ public void setUp() throws Exception {
+ EnvironmentUtils.cleanEnv();
+ EnvironmentUtils.closeStatMonitor();
+ EnvironmentUtils.closeMemControl();
+ CLUSTER_CONFIG.createAllPath();
+ server = Server.getInstance();
+ server.start();
+ EnvironmentUtils.envSetUp();
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ server.stop();
+ EnvironmentUtils.cleanEnv();
+ }
+
+ @Test
+ public void testClusterRpcQueryManagerWithoutFilter() throws Exception {
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+
+ // first query
+ boolean hasResultSet = statement.execute(queryStatementsWithoutFilter);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ ConcurrentHashMap<Long, String> map = ClusterRpcQueryManager.getJobIdMapTaskId();
+ assertEquals(1, map.size());
+ for (String taskId : map.values()) {
+ assertNotNull(manager.getSingleQuery(taskId));
+ }
+ for (long jobId : map.keySet()) {
+ assertNotNull(manager.getSingleQuery(jobId));
+ }
+ for (Entry<Long, String> entry : map.entrySet()) {
+ long jobId = entry.getKey();
+ String taskId = entry.getValue();
+ assertEquals(taskId, String.format("%s:%d", LOCAL_ADDR, jobId));
+ }
+
+ // second query
+ hasResultSet = statement.execute(queryStatementsWithoutFilter);
+ assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ map = ClusterRpcQueryManager.getJobIdMapTaskId();
+ assertEquals(2, map.size());
+ for (String taskId : map.values()) {
+ assertNotNull(manager.getSingleQuery(taskId));
+ }
+ for (long jobId : map.keySet()) {
+ assertNotNull(manager.getSingleQuery(jobId));
+ }
+ for (Entry<Long, String> entry : map.entrySet()) {
+ long jobId = entry.getKey();
+ String taskId = entry.getValue();
+ assertEquals(taskId, String.format("%s:%d", LOCAL_ADDR, jobId));
+ }
+
+ // third query
+ hasResultSet = statement.execute(queryStatementsWithoutFilter);
+ assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ map = ClusterRpcQueryManager.getJobIdMapTaskId();
+ assertEquals(3, map.size());
+ for (String taskId : map.values()) {
+ assertNotNull(manager.getSingleQuery(taskId));
+ }
+ for (long jobId : map.keySet()) {
+ assertNotNull(manager.getSingleQuery(jobId));
+ }
+ for (Entry<Long, String> entry : map.entrySet()) {
+ long jobId = entry.getKey();
+ String taskId = entry.getValue();
+ assertEquals(taskId, String.format("%s:%d", LOCAL_ADDR, jobId));
+ }
+ statement.close();
+ }
+ }
+
+ @Test
+ public void testClusterRpcQueryManagerWithFilter() throws Exception {
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+
+ // first query
+ boolean hasResultSet = statement.execute(queryStatementsWithFilter);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ assertEquals(10, resultSet.getLong(1));
+ assertEquals(100, resultSet.getInt(2));
+ assertNull(resultSet.getString(3));
+ assertNull(resultSet.getString(4));
+ ConcurrentHashMap<Long, String> map = ClusterRpcQueryManager.getJobIdMapTaskId();
+ assertEquals(1, map.size());
+ for (String taskId : map.values()) {
+ assertNotNull(manager.getSingleQuery(taskId));
+ }
+ for (long jobId : map.keySet()) {
+ assertNotNull(manager.getSingleQuery(jobId));
+ }
+ for (Entry<Long, String> entry : map.entrySet()) {
+ long jobId = entry.getKey();
+ String taskId = entry.getValue();
+ assertEquals(taskId, String.format("%s:%d", LOCAL_ADDR, jobId));
+ }
+ assertFalse(resultSet.next());
+
+ // second query
+ hasResultSet = statement.execute(queryStatementsWithFilter);
+ assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ assertEquals(10, resultSet.getLong(1));
+ assertEquals(100, resultSet.getInt(2));
+ assertNull(resultSet.getString(3));
+ assertNull(resultSet.getString(4));
+ map = ClusterRpcQueryManager.getJobIdMapTaskId();
+ assertEquals(2, map.size());
+ for (String taskId : map.values()) {
+ assertNotNull(manager.getSingleQuery(taskId));
+ }
+ for (long jobId : map.keySet()) {
+ assertNotNull(manager.getSingleQuery(jobId));
+ }
+ for (Entry<Long, String> entry : map.entrySet()) {
+ long jobId = entry.getKey();
+ String taskId = entry.getValue();
+ assertEquals(taskId, String.format("%s:%d", LOCAL_ADDR, jobId));
+ }
+ assertFalse(resultSet.next());
+
+ // third query
+ hasResultSet = statement.execute(queryStatementsWithFilter);
+ assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ assertEquals(10, resultSet.getLong(1));
+ assertEquals(100, resultSet.getInt(2));
+ assertNull(resultSet.getString(3));
+ assertNull(resultSet.getString(4));
+ map = ClusterRpcQueryManager.getJobIdMapTaskId();
+ assertEquals(3, map.size());
+ for (String taskId : map.values()) {
+ assertNotNull(manager.getSingleQuery(taskId));
+ }
+ for (long jobId : map.keySet()) {
+ assertNotNull(manager.getSingleQuery(jobId));
+ }
+ for (Entry<Long, String> entry : map.entrySet()) {
+ long jobId = entry.getKey();
+ String taskId = entry.getValue();
+ assertEquals(taskId, String.format("%s:%d", LOCAL_ADDR, jobId));
+ }
+ assertFalse(resultSet.next());
+ statement.close();
+ }
+ }
+
+ @Test
+ public void testClusterRpcSingleQueryWithoutFilterManager() throws Exception {
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute(queryStatementsWithoutFilter);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ ConcurrentHashMap<Long, String> map = ClusterRpcQueryManager.getJobIdMapTaskId();
+ assertEquals(1, map.size());
+ for (String taskId : map.values()) {
+ ClusterRpcSingleQueryManager singleManager = manager.getSingleQuery(taskId);
+ assertNotNull(singleManager);
+ assertEquals(0, singleManager.getQueryRounds());
+ assertEquals(taskId, singleManager.getTaskId());
+
+ // select path plans
+ Map<String, QueryPlan> selectPathPlans = singleManager.getSelectPathPlans();
+ assertEquals(1, selectPathPlans.size());
+ for (QueryPlan queryPlan : selectPathPlans.values()) {
+ List<Path> paths = queryPlan.getPaths();
+ List<Path> correctPaths = new ArrayList<>();
+ correctPaths.add(new Path("root.vehicle.d0.s0"));
+ correctPaths.add(new Path("root.vehicle.d0.s1"));
+ correctPaths.add(new Path("root.vehicle.d0.s3"));
+ assertEquals(correctPaths, paths);
+ assertNull(queryPlan.getExpression());
+ }
+
+ // select series by group id
+ assertEquals(0, singleManager.getSelectSeriesByGroupId().size());
+
+ // select series reader
+ assertTrue(singleManager
+ .getSelectSeriesReaders().isEmpty());
+
+ }
+ statement.close();
+ }
+ }
+
+ @Test
+ public void testClusterRpcSingleQueryWithFilterManager() throws Exception {
+ try (Connection connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + URL, "root", "root")) {
+ insertData(connection, createSQLs, insertSQLs);
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute(queryStatementsWithFilter);
+ assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ assertTrue(resultSet.next());
+ ConcurrentHashMap<Long, String> map = ClusterRpcQueryManager.getJobIdMapTaskId();
+ assertEquals(1, map.size());
+ for (String taskId : map.values()) {
+ ClusterRpcSingleQueryManager singleManager = manager.getSingleQuery(taskId);
+ assertNotNull(singleManager);
+ assertEquals(0, singleManager.getQueryRounds());
+ assertEquals(taskId, singleManager.getTaskId());
+
+ // select path plans
+ Map<String, QueryPlan> selectPathPlans = singleManager.getSelectPathPlans();
+ assertEquals(1, selectPathPlans.size());
+ for (QueryPlan queryPlan : selectPathPlans.values()) {
+ List<Path> paths = queryPlan.getPaths();
+ List<Path> correctPaths = new ArrayList<>();
+ correctPaths.add(new Path("root.vehicle.d0.s0"));
+ correctPaths.add(new Path("root.vehicle.d0.s1"));
+ correctPaths.add(new Path("root.vehicle.d0.s3"));
+ assertEquals(correctPaths, paths);
+ assertNotNull(queryPlan.getExpression());
+ }
+
+ // select series by group id
+ assertTrue(singleManager.getSelectSeriesByGroupId().isEmpty());
+
+ // select series reader
+ assertTrue(singleManager
+ .getSelectSeriesReaders().isEmpty());
+
+ // filter path plans
+ Map<String, FilterGroupEntity> filterGroupEntityMap = singleManager.getFilterGroupEntityMap();
+ assertTrue(filterGroupEntityMap.isEmpty());
+
+ }
+ statement.close();
+ }
+ }
+}
\ No newline at end of file
diff --git a/cluster/src/test/java/org/apache/iotdb/cluster/query/utils/ExpressionUtilsTest.java b/cluster/src/test/java/org/apache/iotdb/cluster/query/utils/ExpressionUtilsTest.java
new file mode 100644
index 0000000..84f8f5f
--- /dev/null
+++ b/cluster/src/test/java/org/apache/iotdb/cluster/query/utils/ExpressionUtilsTest.java
@@ -0,0 +1,230 @@
+/**
+ * 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.cluster.query.utils;
+
+import static org.apache.iotdb.cluster.utils.Utils.insertData;
+import static org.junit.Assert.*;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.iotdb.cluster.config.ClusterConfig;
+import org.apache.iotdb.cluster.config.ClusterDescriptor;
+import org.apache.iotdb.cluster.entity.Server;
+import org.apache.iotdb.cluster.qp.executor.ClusterQueryProcessExecutor;
+import org.apache.iotdb.cluster.query.expression.TrueExpression;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcQueryManager;
+import org.apache.iotdb.cluster.query.manager.coordinatornode.ClusterRpcSingleQueryManager;
+import org.apache.iotdb.cluster.utils.EnvironmentUtils;
+import org.apache.iotdb.db.qp.QueryProcessor;
+import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
+import org.apache.iotdb.jdbc.Config;
+import org.apache.iotdb.tsfile.read.common.Path;
... 3401 lines suppressed ...