You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by su...@apache.org on 2019/04/04 02:51:47 UTC
[incubator-iotdb] branch master updated:
[IOTDB-47][IOTDB-54][IOTDB-59][IOTDB-60]Aggregate+GroupBy+Fill (#97)
This is an automated email from the ASF dual-hosted git repository.
suyue pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git
The following commit(s) were added to refs/heads/master by this push:
new e0d4e33 [IOTDB-47][IOTDB-54][IOTDB-59][IOTDB-60]Aggregate+GroupBy+Fill (#97)
e0d4e33 is described below
commit e0d4e33517657c69871e1056b4dcb7f37c6eae84
Author: suyue <23...@qq.com>
AuthorDate: Thu Apr 4 10:51:43 2019 +0800
[IOTDB-47][IOTDB-54][IOTDB-59][IOTDB-60]Aggregate+GroupBy+Fill (#97)
* realize reader for aggregation
* realize aggregate function interface
* add aggregation in qp
* add test
* optimization last and max_time aggregate query that has only time filter or no filter
* add aggregate function with value filter
* modify for pr review suggestion
* fix bug of IOTDB-54
* group by
* add groupby test and fix groupBy bug
* fill query
* modify pr review
* provide unified query resource control interface in QueryResourceManager.
* bug fix: multiple queries in the same statement will cause resource leak.
* optimization of query with value filter by push down filter
---
.../db/engine/querycontext/ReadOnlyMemChunk.java | 7 -
.../java/org/apache/iotdb/db/metadata/MTree.java | 2 +-
.../iotdb/db/qp/executor/OverflowQPExecutor.java | 44 +-
.../iotdb/db/qp/executor/QueryProcessExecutor.java | 99 ++-
.../iotdb/db/qp/physical/sys/MetadataPlan.java | 2 +-
.../iotdb/db/qp/strategy/PhysicalGenerator.java | 1 +
.../db/query/aggregation/AggreFuncFactory.java | 79 ++
.../db/query/aggregation/AggreResultData.java | 192 +++++
.../db/query/aggregation/AggregateFunction.java | 135 ++++
.../db/query/aggregation/impl/CountAggrFunc.java | 155 ++++
.../db/query/aggregation/impl/FirstAggrFunc.java | 154 ++++
.../db/query/aggregation/impl/LastAggrFunc.java | 151 ++++
.../db/query/aggregation/impl/MaxTimeAggrFunc.java | 135 ++++
.../query/aggregation/impl/MaxValueAggrFunc.java | 170 ++++
.../db/query/aggregation/impl/MeanAggrFunc.java | 163 ++++
.../db/query/aggregation/impl/MinTimeAggrFunc.java | 156 ++++
.../query/aggregation/impl/MinValueAggrFunc.java | 166 ++++
.../db/query/aggregation/impl/SumAggrFunc.java} | 66 +-
.../iotdb/db/query/context/QueryContext.java | 17 +
...edFilePathsManager.java => JobFileManager.java} | 78 +-
.../db/query/control/QueryDataSourceManager.java | 53 --
...TokenManager.java => QueryResourceManager.java} | 86 +-
.../query/dataset/AggreResultDataPointReader.java | 52 ++
.../dataset/EngineDataSetWithTimeGenerator.java | 7 +-
.../dataset/EngineDataSetWithoutTimeGenerator.java | 16 +-
.../db/query/executor/AggregateEngineExecutor.java | 330 ++++++++
.../executor/EngineExecutorWithTimeGenerator.java | 55 +-
.../EngineExecutorWithoutTimeGenerator.java | 41 +-
.../iotdb/db/query/executor/EngineQueryRouter.java | 180 ++++-
.../db/query/executor/FillEngineExecutor.java | 91 +++
.../executor/groupby/GroupByEngineDataSet.java | 167 ++++
.../GroupByWithOnlyTimeFilterDataSetDataSet.java | 314 +++++++
.../GroupByWithValueFilterDataSetDataSet.java | 160 ++++
.../db/query/factory/SeriesReaderFactory.java | 129 ++-
.../java/org/apache/iotdb/db/query/fill/IFill.java | 79 +-
.../org/apache/iotdb/db/query/fill/LinearFill.java | 90 ++-
.../apache/iotdb/db/query/fill/PreviousFill.java | 36 +-
.../iotdb/db/query/reader/AllDataReader.java | 129 +++
...eaderByTimeStamp.java => IAggregateReader.java} | 16 +-
.../iotdb/db/query/reader/IBatchReader.java} | 61 +-
.../iotdb/db/query/reader/IPointReader.java} | 57 +-
.../org/apache/iotdb/db/query/reader/IReader.java | 14 +-
...nkReaderWithFilter.java => MemChunkReader.java} | 57 +-
.../reader/mem/MemChunkReaderByTimestamp.java | 37 +-
.../reader/mem/MemChunkReaderWithoutFilter.java | 70 --
.../reader/merge/EngineReaderByTimeStamp.java | 8 +-
.../db/query/reader/merge/PriorityMergeReader.java | 56 +-
.../merge/PriorityMergeReaderByTimestamp.java | 60 +-
.../query/reader/sequence/SealedTsFilesReader.java | 151 ++--
.../sequence/SealedTsFilesReaderByTimestamp.java | 136 ++++
.../query/reader/sequence/SequenceDataReader.java | 106 ++-
.../sequence/SequenceDataReaderByTimestamp.java | 108 +++
.../reader/sequence/UnSealedTsFileReader.java | 65 +-
.../sequence/UnSealedTsFilesReaderByTimestamp.java | 64 ++
.../query/reader/unsequence/EngineChunkReader.java | 39 +-
.../unsequence/EngineChunkReaderByTimestamp.java | 77 ++
.../db/query/timegenerator/EngineLeafNode.java | 11 +-
.../query/timegenerator/EngineNodeConstructor.java | 22 +-
.../query/timegenerator/EngineTimeGenerator.java | 10 +-
.../org/apache/iotdb/db/rescon/package-info.java} | 53 +-
.../java/org/apache/iotdb/db/service/IoTDB.java | 1 -
.../org/apache/iotdb/db/service/TSServiceImpl.java | 76 +-
.../java/org/apache/iotdb/db/utils/QueryUtils.java | 3 +
.../apache/iotdb/db/utils/TimeValuePairUtils.java | 26 +
.../org/apache/iotdb/db/utils/TsPrimitiveType.java | 2 +-
.../engine/modification/DeletionFileNodeTest.java | 25 +-
.../db/engine/modification/DeletionQueryTest.java | 16 +-
.../apache/iotdb/db/integration/IOTDBFillIT.java | 351 ++++++++
.../iotdb/db/integration/IOTDBGroupByIT.java | 532 ++++++++++++
.../iotdb/db/integration/IoTDBAggregationIT.java | 550 +++++++++++++
.../integration/IoTDBAggregationLargeDataIT.java | 900 +++++++++++++++++++++
.../integration/IoTDBAggregationSmallDataIT.java | 757 +++++++++++++++++
.../iotdb/db/integration/IoTDBCompleteIT.java | 38 +
.../db/integration/IoTDBEngineTimeGeneratorIT.java | 20 +-
.../iotdb/db/integration/IoTDBLargeDataIT.java | 4 -
.../db/integration/IoTDBSequenceDataQueryIT.java | 23 +-
.../iotdb/db/integration/IoTDBSeriesReaderIT.java | 29 +-
.../apache/iotdb/db/qp/plan/PhysicalPlanTest.java | 2 +-
.../org/apache/iotdb/db/qp/plan/QPUpdateTest.java | 31 +-
.../apache/iotdb/db/qp/utils/MemIntQpExecutor.java | 27 +-
.../db/query/control/FileReaderManagerTest.java | 11 +-
...agerTest.java => QueryResourceManagerTest.java} | 2 +-
.../query/executor/GroupByEngineDataSetTest.java | 153 ++++
.../iotdb/db/query/reader/AllDataReaderTest.java | 76 ++
.../iotdb/db/query/reader/FakedIBatchPoint.java | 108 +++
.../iotdb/db/query/reader/FakedIPointReader.java | 74 ++
.../query/reader/FakedSeriesReaderByTimestamp.java | 76 ++
.../merge/PriorityMergeReaderByTimestampTest.java | 72 +-
.../reader/merge/PriorityMergeReaderTest.java | 28 +-
.../reader/merge/SeriesMergeSortReaderTest.java | 27 +-
.../SequenceDataReaderByTimestampTest.java | 91 +++
.../apache/iotdb/db/utils/EnvironmentUtils.java | 10 +-
.../org/apache/iotdb/jdbc/IoTDBQueryResultSet.java | 8 +-
.../java/org/apache/iotdb/jdbc/IoTDBStatement.java | 9 +-
service-rpc/src/main/thrift/rpc.thrift | 2 +
.../apache/iotdb/tsfile/read/common/BatchData.java | 97 ++-
.../iotdb/tsfile/read/filter/basic/Filter.java | 8 +
.../tsfile/read/filter/operator/AndFilter.java | 6 +
.../iotdb/tsfile/read/filter/operator/Eq.java | 14 +
.../iotdb/tsfile/read/filter/operator/Gt.java | 17 +-
.../iotdb/tsfile/read/filter/operator/GtEq.java | 17 +-
.../iotdb/tsfile/read/filter/operator/Lt.java | 14 +
.../iotdb/tsfile/read/filter/operator/LtEq.java | 14 +
.../iotdb/tsfile/read/filter/operator/NotEq.java | 14 +
.../tsfile/read/filter/operator/NotFilter.java | 5 +
.../tsfile/read/filter/operator/OrFilter.java | 6 +
.../tsfile/read/query/dataset/QueryDataSet.java | 7 +
.../tsfile/read/reader/chunk/ChunkReader.java | 54 +-
.../read/reader/series/FileSeriesReader.java | 45 +-
.../reader/series/SeriesReaderByTimestamp.java | 33 +-
.../read/filter/MinTimeMaxTimeFilterTest.java | 52 +-
.../iotdb/tsfile/read/filter/OperatorTest.java | 1 +
112 files changed, 8701 insertions(+), 1128 deletions(-)
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
index d42eef7..5086c6e 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
@@ -48,13 +48,6 @@ public class ReadOnlyMemChunk implements TimeValuePairSorter {
/**
* init by TSDataType and TimeValuePairSorter.
*/
-// public ReadOnlyMemChunk(TSDataType dataType, TimeValuePairSorter memSeries) {
-// this(dataType, memSeries, Collections.emptyMap());
-// }
-
- /**
- * init by TSDataType and TimeValuePairSorter.
- */
public ReadOnlyMemChunk(TSDataType dataType, TimeValuePairSorter memSeries, Map<String, String> props) {
this.dataType = dataType;
this.memSeries = memSeries;
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/metadata/MTree.java b/iotdb/src/main/java/org/apache/iotdb/db/metadata/MTree.java
index 6444b4a..2c7e5b0 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/metadata/MTree.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/metadata/MTree.java
@@ -970,7 +970,7 @@ public class MTree implements Serializable {
if (node.isLeaf()) {
if (nodes.length <= idx) {
String nodePath = parent + node;
- List<String> tsRow = new ArrayList<>(4);// get [name,storage group,dataType,encoding]
+ List<String> tsRow = new ArrayList<>(4);// get [name,storage group,resultDataType,encoding]
tsRow.add(nodePath);
MeasurementSchema measurementSchema = node.getSchema();
tsRow.add(node.getDataFileName());
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java
index 061bb47..d477e2f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java
@@ -51,8 +51,11 @@ import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
import org.apache.iotdb.db.qp.physical.sys.LoadDataPlan;
import org.apache.iotdb.db.qp.physical.sys.MetadataPlan;
import org.apache.iotdb.db.qp.physical.sys.PropertyPlan;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.fill.IFill;
import org.apache.iotdb.db.utils.AuthUtils;
import org.apache.iotdb.db.utils.LoadDataUtils;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
@@ -176,22 +179,26 @@ public class OverflowQPExecutor extends QueryProcessExecutor {
}
@Override
- public QueryDataSet aggregate(List<Pair<Path, String>> aggres, IExpression expression)
- throws ProcessorException {
- throw new ProcessorException("not support");
+ public QueryDataSet aggregate(List<Path> paths, List<String> aggres, IExpression expression,
+ QueryContext context)
+ throws ProcessorException, FileNodeManagerException, QueryFilterOptimizationException,
+ PathErrorException, IOException {
+ return queryRouter.aggregate(paths, aggres, expression, context);
}
- // @Override
- // public QueryDataSet fill(List<Path> fillPaths, long queryTime, Map<TSDataType, IFill> fillTypes) throws
- // ProcessorException, IOException, PathErrorException {
- // return queryEngine.fill(fillPaths, queryTime, fillTypes);
- // }
+ @Override
+ public QueryDataSet fill(List<Path> fillPaths, long queryTime, Map<TSDataType, IFill> fillTypes,
+ QueryContext context)
+ throws ProcessorException, IOException, PathErrorException, FileNodeManagerException {
+ return queryRouter.fill(fillPaths, queryTime, fillTypes, context);
+ }
@Override
- public QueryDataSet groupBy(List<Pair<Path, String>> aggres, IExpression expression, long unit,
- long origin,
- List<Pair<Long, Long>> intervals, int fetchSize) throws ProcessorException {
- throw new ProcessorException("not support");
+ public QueryDataSet groupBy(List<Path> paths, List<String> aggres, IExpression expression,
+ long unit, long origin, List<Pair<Long, Long>> intervals, QueryContext context)
+ throws ProcessorException, FileNodeManagerException, QueryFilterOptimizationException,
+ PathErrorException, IOException {
+ return queryRouter.groupBy(paths, aggres, expression, unit, origin, intervals, context);
}
@Override
@@ -502,7 +509,8 @@ public class OverflowQPExecutor extends QueryProcessExecutor {
// optimize the speed of adding timeseries
String fileNodePath = mManager.getFileNameByPath(path.getFullPath());
// the two map is stored in the storage group node
- Map<String, MeasurementSchema> schemaMap = mManager.getSchemaMapForOneFileNode(fileNodePath);
+ Map<String, MeasurementSchema> schemaMap = mManager
+ .getSchemaMapForOneFileNode(fileNodePath);
Map<String, Integer> numSchemaMap = mManager.getNumSchemaMapForOneFileNode(fileNodePath);
String lastNode = path.getMeasurement();
boolean isNewMeasurement = true;
@@ -514,8 +522,8 @@ public class OverflowQPExecutor extends QueryProcessExecutor {
if (!columnSchema.getType().equals(dataType)
|| !columnSchema.getEncodingType().equals(encoding)) {
throw new ProcessorException(String.format(
- "The dataType or encoding of the last node %s is conflicting in the storage group %s",
- lastNode, fileNodePath));
+ "The resultDataType or encoding of the last node %s is conflicting "
+ + "in the storage group %s", lastNode, fileNodePath));
}
mManager.addPathToMTree(path.getFullPath(), dataType, encoding, compressor, props);
numSchemaMap.put(lastNode, numSchemaMap.get(lastNode) + 1);
@@ -529,7 +537,8 @@ public class OverflowQPExecutor extends QueryProcessExecutor {
if (isNewMeasurement) {
// add time series to schema
fileNodeManager.addTimeSeries(path, dataType, encoding, compressor, props);
- //TODO fileNodeManager.addTimeSeries(path, dataType, encoding, compressor, encodingArgs);
+ //TODO fileNodeManager.addTimeSeries(
+ //TODO path, resultDataType, encoding, compressor, encodingArgs);
}
// fileNodeManager.closeOneFileNode(namespacePath);
} catch (FileNodeManagerException e) {
@@ -627,8 +636,7 @@ public class OverflowQPExecutor extends QueryProcessExecutor {
/**
* Delete all data of time series in pathList.
*
- * @param pathList
- * deleted paths
+ * @param pathList deleted paths
*/
private void deleteDataOfTimeSeries(List<String> pathList) throws ProcessorException {
for (String p : pathList) {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java
index b735617..60f97f7 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java
@@ -22,14 +22,21 @@ 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.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.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.executor.EngineQueryRouter;
+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;
@@ -40,18 +47,40 @@ import org.apache.iotdb.tsfile.utils.Pair;
public abstract class QueryProcessExecutor {
protected ThreadLocal<Integer> fetchSize = new ThreadLocal<>();
- private EngineQueryRouter queryRouter = new EngineQueryRouter();
+ protected EngineQueryRouter queryRouter = new EngineQueryRouter();
public QueryProcessExecutor() {
}
- public QueryDataSet processQuery(PhysicalPlan plan) throws IOException, FileNodeManagerException {
- QueryPlan queryPlan = (QueryPlan) plan;
+ /**
+ * process query plan of qp layer, construct queryDataSet.
+ * @param queryPlan QueryPlan
+ * @return QueryDataSet
+ */
+ public QueryDataSet processQuery(QueryPlan queryPlan, QueryContext context)
+ throws IOException, FileNodeManagerException, PathErrorException,
+ QueryFilterOptimizationException, ProcessorException {
QueryExpression queryExpression = QueryExpression.create().setSelectSeries(queryPlan.getPaths())
.setExpression(queryPlan.getExpression());
+ if (queryPlan instanceof GroupByPlan) {
+ GroupByPlan groupByPlan = (GroupByPlan) queryPlan;
+ return groupBy(groupByPlan.getPaths(), groupByPlan.getAggregations(),
+ groupByPlan.getExpression(), groupByPlan.getUnit(), groupByPlan.getOrigin(),
+ groupByPlan.getIntervals(), context);
+ }
- return queryRouter.query(queryExpression);
+ if (queryPlan instanceof AggregationPlan) {
+ return aggregate(queryPlan.getPaths(), queryPlan.getAggregations(),
+ ((AggregationPlan) queryPlan).getExpression(), context);
+ }
+
+ if (queryPlan instanceof FillQueryPlan) {
+ FillQueryPlan fillQueryPlan = (FillQueryPlan) queryPlan;
+ return fill(queryPlan.getPaths(), fillQueryPlan.getQueryTime(),
+ fillQueryPlan.getFillType(), context);
+ }
+ return queryRouter.query(queryExpression, context);
}
public abstract TSDataType getSeriesType(Path fullPath) throws PathErrorException;
@@ -73,25 +102,26 @@ public abstract class QueryProcessExecutor {
this.fetchSize.set(fetchSize);
}
- public abstract QueryDataSet aggregate(List<Pair<Path, String>> aggres, IExpression expression)
- throws ProcessorException, IOException, PathErrorException;
+ public abstract QueryDataSet aggregate(List<Path> paths, List<String> aggres,
+ IExpression expression, QueryContext context) throws ProcessorException, IOException,
+ PathErrorException, FileNodeManagerException, QueryFilterOptimizationException;
+
+ public abstract 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;
- public abstract QueryDataSet groupBy(List<Pair<Path, String>> aggres, IExpression expression,
- long unit,
- long origin, List<Pair<Long, Long>> intervals, int fetchSize)
- throws ProcessorException, IOException, PathErrorException;
+ public abstract QueryDataSet fill(List<Path> fillPaths, long queryTime, Map<TSDataType,
+ IFill> fillTypes, QueryContext context)
+ throws ProcessorException, IOException, PathErrorException, FileNodeManagerException;
/**
* executeWithGlobalTimeFilter update command and return whether the operator is successful.
*
- * @param path
- * : update series seriesPath
- * @param startTime
- * start time in update command
- * @param endTime
- * end time in update command
- * @param value
- * - in type of string
+ * @param path : update series seriesPath
+ * @param startTime start time in update command
+ * @param endTime end time in update command
+ * @param value - in type of string
* @return - whether the operator is successful.
*/
public abstract boolean update(Path path, long startTime, long endTime, String value)
@@ -100,10 +130,8 @@ public abstract class QueryProcessExecutor {
/**
* executeWithGlobalTimeFilter delete command and return whether the operator is successful.
*
- * @param paths
- * : delete series paths
- * @param deleteTime
- * end time in delete command
+ * @param paths : delete series paths
+ * @param deleteTime end time in delete command
* @return - whether the operator is successful.
*/
public boolean delete(List<Path> paths, long deleteTime) throws ProcessorException {
@@ -137,10 +165,8 @@ public abstract class QueryProcessExecutor {
/**
* executeWithGlobalTimeFilter delete command and return whether the operator is successful.
*
- * @param path
- * : delete series seriesPath
- * @param deleteTime
- * end time in delete command
+ * @param path : delete series seriesPath
+ * @param deleteTime end time in delete command
* @return - whether the operator is successful.
*/
protected abstract boolean delete(Path path, long deleteTime) throws ProcessorException;
@@ -148,12 +174,9 @@ public abstract class QueryProcessExecutor {
/**
* insert a single value. Only used in test
*
- * @param path
- * seriesPath to be inserted
- * @param insertTime
- * - it's time point but not a range
- * @param value
- * value to be inserted
+ * @param path seriesPath to be inserted
+ * @param insertTime - it's time point but not a range
+ * @param value value to be inserted
* @return - Operate Type.
*/
public abstract int insert(Path path, long insertTime, String value) throws ProcessorException;
@@ -161,14 +184,10 @@ public abstract class QueryProcessExecutor {
/**
* executeWithGlobalTimeFilter insert command and return whether the operator is successful.
*
- * @param deviceId
- * deviceId to be inserted
- * @param insertTime
- * - it's time point but not a range
- * @param measurementList
- * measurements to be inserted
- * @param insertValues
- * values to be inserted
+ * @param deviceId deviceId to be inserted
+ * @param insertTime - it's time point but not a range
+ * @param measurementList measurements to be inserted
+ * @param insertValues values to be inserted
* @return - Operate Type.
*/
public abstract int multiInsert(String deviceId, long insertTime, List<String> measurementList,
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/qp/physical/sys/MetadataPlan.java b/iotdb/src/main/java/org/apache/iotdb/db/qp/physical/sys/MetadataPlan.java
index f612165..82b9404 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/qp/physical/sys/MetadataPlan.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/qp/physical/sys/MetadataPlan.java
@@ -114,7 +114,7 @@ public class MetadataPlan extends PhysicalPlan {
@Override
public String toString() {
- String ret = String.format("seriesPath: %s\ndataType: %s\nencoding: %s\nnamespace type: %s\nargs: ", path, dataType, encoding, namespaceType);
+ String ret = String.format("seriesPath: %s\nresultDataType: %s\nencoding: %s\nnamespace type: %s\nargs: ", path, dataType, encoding, namespaceType);
StringBuilder stringBuilder = new StringBuilder(ret.length()+50);
stringBuilder.append(ret);
for (Map.Entry prop : props.entrySet()) {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
index 2caac94..290d777 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
@@ -242,6 +242,7 @@ public class PhysicalGenerator {
((GroupByPlan) queryPlan).setUnit(queryOperator.getUnit());
((GroupByPlan) queryPlan).setOrigin(queryOperator.getOrigin());
((GroupByPlan) queryPlan).setIntervals(queryOperator.getIntervals());
+ ((GroupByPlan) queryPlan).setAggregations(queryOperator.getSelectOperator().getAggregations());
} else if (queryOperator.isFill()) {
queryPlan = new FillQueryPlan();
FilterOperator timeFilter = queryOperator.getFilterOperator();
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggreFuncFactory.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggreFuncFactory.java
new file mode 100644
index 0000000..c2d1883
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggreFuncFactory.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.aggregation;
+
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.db.query.aggregation.impl.CountAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.FirstAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.LastAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.MaxTimeAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.MaxValueAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.MeanAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.MinTimeAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.MinValueAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.SumAggrFunc;
+import org.apache.iotdb.tsfile.common.constant.StatisticConstant;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+/**
+ * Easy factory pattern to build AggregateFunction.
+ */
+public class AggreFuncFactory {
+
+ private AggreFuncFactory() {
+ }
+
+ /**
+ * construct AggregateFunction using factory pattern.
+ *
+ * @param aggrFuncName function name.
+ * @param dataType data type.
+ */
+ public static AggregateFunction getAggrFuncByName(String aggrFuncName, TSDataType dataType)
+ throws ProcessorException {
+ if (aggrFuncName == null) {
+ throw new ProcessorException("AggregateFunction Name must not be null");
+ }
+
+ switch (aggrFuncName.toLowerCase()) {
+ case StatisticConstant.MIN_TIME:
+ return new MinTimeAggrFunc();
+ case StatisticConstant.MAX_TIME:
+ return new MaxTimeAggrFunc();
+ case StatisticConstant.MIN_VALUE:
+ return new MinValueAggrFunc(dataType);
+ case StatisticConstant.MAX_VALUE:
+ return new MaxValueAggrFunc(dataType);
+ case StatisticConstant.COUNT:
+ return new CountAggrFunc();
+ case StatisticConstant.MEAN:
+ return new MeanAggrFunc(dataType);
+ case StatisticConstant.FIRST:
+ return new FirstAggrFunc(dataType);
+ case StatisticConstant.SUM:
+ return new SumAggrFunc(dataType);
+ case StatisticConstant.LAST:
+ return new LastAggrFunc(dataType);
+ default:
+ throw new ProcessorException(
+ "aggregate does not support " + aggrFuncName + " function.");
+ }
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggreResultData.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggreResultData.java
new file mode 100644
index 0000000..0efae01
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggreResultData.java
@@ -0,0 +1,192 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.aggregation;
+
+import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.utils.Binary;
+
+public class AggreResultData {
+
+ private long timestamp;
+ private TSDataType dataType;
+
+ private boolean booleanRet;
+ private int intRet;
+ private long longRet;
+ private float floatRet;
+ private double doubleRet;
+ private Binary binaryRet;
+
+ private boolean isSetValue;
+ private boolean isSetTime;
+
+ public AggreResultData(TSDataType dataType) {
+ this.dataType = dataType;
+ this.isSetTime = false;
+ this.isSetValue = false;
+ }
+
+ public void reSet() {
+ isSetValue = false;
+ isSetTime = false;
+ }
+
+ public void putTimeAndValue(long timestamp, Object v) {
+ setTimestamp(timestamp);
+ setAnObject((Comparable<?>) v);
+ }
+
+ public Object getValue() {
+ switch (dataType) {
+ case BOOLEAN:
+ return booleanRet;
+ case DOUBLE:
+ return doubleRet;
+ case TEXT:
+ return binaryRet;
+ case FLOAT:
+ return floatRet;
+ case INT32:
+ return intRet;
+ case INT64:
+ return longRet;
+ default:
+ throw new UnSupportedDataTypeException(String.valueOf(dataType));
+ }
+ }
+
+ /**
+ * set an object.
+ *
+ * @param v object value
+ */
+ public void setAnObject(Comparable<?> v) {
+ isSetValue = true;
+ switch (dataType) {
+ case BOOLEAN:
+ booleanRet = (Boolean) v;
+ break;
+ case DOUBLE:
+ doubleRet = (Double) v;
+ break;
+ case TEXT:
+ binaryRet = (Binary) v;
+ break;
+ case FLOAT:
+ floatRet = (Float) v;
+ break;
+ case INT32:
+ intRet = (Integer) v;
+ break;
+ case INT64:
+ longRet = (Long) v;
+ break;
+ default:
+ throw new UnSupportedDataTypeException(String.valueOf(dataType));
+ }
+ }
+
+ public long getTimestamp() {
+ return timestamp;
+ }
+
+ public void setTimestamp(long timestamp) {
+ isSetTime = true;
+ this.timestamp = timestamp;
+ }
+
+ public TSDataType getDataType() {
+ return dataType;
+ }
+
+ public boolean isBooleanRet() {
+ return booleanRet;
+ }
+
+ public void setBooleanRet(boolean booleanRet) {
+ this.isSetValue = true;
+ this.booleanRet = booleanRet;
+ }
+
+ public int getIntRet() {
+ return intRet;
+ }
+
+ public void setIntRet(int intRet) {
+ this.isSetValue = true;
+ this.intRet = intRet;
+ }
+
+ public long getLongRet() {
+ return longRet;
+ }
+
+ public void setLongRet(long longRet) {
+ this.isSetValue = true;
+ this.longRet = longRet;
+ }
+
+ public float getFloatRet() {
+ return floatRet;
+ }
+
+ public void setFloatRet(float floatRet) {
+ this.isSetValue = true;
+ this.floatRet = floatRet;
+ }
+
+ public double getDoubleRet() {
+ return doubleRet;
+ }
+
+ public void setDoubleRet(double doubleRet) {
+ this.isSetValue = true;
+ this.doubleRet = doubleRet;
+ }
+
+ public Binary getBinaryRet() {
+ return binaryRet;
+ }
+
+ public void setBinaryRet(Binary binaryRet) {
+ this.isSetValue = true;
+ this.binaryRet = binaryRet;
+ }
+
+ public boolean isSetValue() {
+ return isSetValue;
+ }
+
+ public boolean isSetTime() {
+ return isSetTime;
+ }
+
+ public AggreResultData deepCopy() {
+ AggreResultData aggreResultData = new AggreResultData(this.dataType);
+ if (isSetValue) {
+ aggreResultData.setAnObject((Comparable<?>) this.getValue());
+ }
+ if (isSetTime) {
+ aggreResultData.setTimestamp(this.getTimestamp());
+ }
+ return aggreResultData;
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateFunction.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateFunction.java
new file mode 100644
index 0000000..a9bdba2
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateFunction.java
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.aggregation;
+
+import java.io.IOException;
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public abstract class AggregateFunction {
+
+ protected AggreResultData resultData;
+ private TSDataType resultDataType;
+
+ /**
+ * construct.
+ *
+ * @param dataType result data type.
+ */
+ public AggregateFunction(TSDataType dataType) {
+ this.resultDataType = dataType;
+ this.resultData = new AggreResultData(dataType);
+ }
+
+ public abstract void init();
+
+ public abstract AggreResultData getResult();
+
+ /**
+ * <p>
+ * Calculate the aggregation using <code>PageHeader</code>.
+ * </p>
+ *
+ * @param pageHeader <code>PageHeader</code>
+ */
+ public abstract void calculateValueFromPageHeader(PageHeader pageHeader)
+ throws ProcessorException;
+
+ /**
+ * <p>
+ * Could not calculate using <method>calculateValueFromPageHeader</method> directly. Calculate the
+ * aggregation according to all decompressed data in this page.
+ * </p>
+ *
+ * @param dataInThisPage the data in the DataPage
+ * @param unsequenceReader unsequence data reader
+ * @throws IOException TsFile data read exception
+ * @throws ProcessorException wrong aggregation method parameter
+ */
+ public abstract void calculateValueFromPageData(BatchData dataInThisPage,
+ IPointReader unsequenceReader) throws IOException, ProcessorException;
+
+ /**
+ * <p>
+ * Could not calculate using <method>calculateValueFromPageHeader</method> directly. Calculate the
+ * aggregation according to all decompressed data in this page.
+ * </p>
+ *
+ * @param dataInThisPage the data in the DataPage
+ * @param unsequenceReader unsequence data reader
+ * @param bound the time upper bounder of data in unsequence data reader
+ * @throws IOException TsFile data read exception
+ * @throws ProcessorException wrong aggregation method parameter
+ */
+ public abstract void calculateValueFromPageData(BatchData dataInThisPage,
+ IPointReader unsequenceReader, long bound) throws IOException, ProcessorException;
+
+ /**
+ * <p>
+ * Calculate the aggregation with data in unsequenceReader.
+ * </p>
+ *
+ * @param unsequenceReader unsequence data reader
+ */
+ public abstract void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+ throws IOException, ProcessorException;
+
+ /**
+ * <p>
+ * Calculate the aggregation with data whose timestamp is less than bound in unsequenceReader.
+ * </p>
+ *
+ * @param unsequenceReader unsequence data reader
+ * @param bound the time upper bounder of data in unsequence data reader
+ * @throws IOException TsFile data read exception
+ */
+ public abstract void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+ throws IOException, ProcessorException;
+
+ /**
+ * <p>
+ * This method is calculate the aggregation using the common timestamps of cross series filter.
+ * </p>
+ *
+ * @throws IOException TsFile data read error
+ */
+ public abstract void calcAggregationUsingTimestamps(long[] timestamps, int length,
+ EngineReaderByTimeStamp dataReader) throws IOException;
+
+ /**
+ * Judge if aggregation results have been calculated. In other words, if the aggregated result
+ * does not need to compute the remaining data, it returns true.
+ *
+ * @return If the aggregation result has been calculated return true, else return false.
+ */
+ public abstract boolean isCalculatedAggregationResult();
+
+ /**
+ * Return data type of aggregation function result data.
+ */
+ public TSDataType getResultDataType() {
+ return resultDataType;
+ }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/CountAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/CountAggrFunc.java
new file mode 100644
index 0000000..020aaf1
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/CountAggrFunc.java
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CountAggrFunc extends AggregateFunction {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(CountAggrFunc.class);
+
+ public CountAggrFunc() {
+ super(TSDataType.INT64);
+ }
+
+ @Override
+ public void init() {
+ resultData.reSet();
+ resultData.setTimestamp(0);
+ resultData.setLongRet(0);
+ }
+
+ @Override
+ public AggreResultData getResult() {
+ return resultData;
+ }
+
+ @Override
+ public void calculateValueFromPageHeader(PageHeader pageHeader) {
+ if (LOGGER.isDebugEnabled()) {
+ LOGGER.debug("PageHeader>>>>>>>>>>>>num of rows:{}, minTimeStamp:{}, maxTimeStamp{}",
+ pageHeader.getNumOfValues(), pageHeader.getMinTimestamp(), pageHeader.getMaxTimestamp());
+ }
+ long preValue = resultData.getLongRet();
+ preValue += pageHeader.getNumOfValues();
+ resultData.setLongRet(preValue);
+
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader)
+ throws IOException {
+ calculateValueFromPageData(dataInThisPage, unsequenceReader, false, 0);
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+ long bound) throws IOException {
+ calculateValueFromPageData(dataInThisPage, unsequenceReader, true, bound);
+ }
+
+ private void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+ boolean hasBound, long bound) throws IOException {
+ int cnt = 0;
+ while (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+ long minTimestamp = Math
+ .min(dataInThisPage.currentTime(), unsequenceReader.current().getTimestamp());
+ if (hasBound && minTimestamp >= bound) {
+ break;
+ }
+ if (dataInThisPage.currentTime() == unsequenceReader.current().getTimestamp()) {
+ dataInThisPage.next();
+ unsequenceReader.next();
+ } else if (dataInThisPage.currentTime() < unsequenceReader.current().getTimestamp()) {
+ dataInThisPage.next();
+ } else {
+ unsequenceReader.next();
+ }
+ cnt++;
+ }
+
+ while (dataInThisPage.hasNext()) {
+ if (hasBound && dataInThisPage.currentTime() >= bound) {
+ break;
+ }
+ dataInThisPage.next();
+ cnt++;
+ }
+ long preValue = resultData.getLongRet();
+ preValue += cnt;
+ resultData.setLongRet(preValue);
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+ throws IOException {
+ int cnt = 0;
+ while (unsequenceReader.hasNext()) {
+ unsequenceReader.next();
+ cnt++;
+ }
+ long preValue = resultData.getLongRet();
+ preValue += cnt;
+ resultData.setLongRet(preValue);
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+ throws IOException {
+ int cnt = 0;
+ while (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+ unsequenceReader.next();
+ cnt++;
+ }
+ long preValue = resultData.getLongRet();
+ preValue += cnt;
+ resultData.setLongRet(preValue);
+ }
+
+ @Override
+ public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+ EngineReaderByTimeStamp dataReader) throws IOException {
+ int cnt = 0;
+ for (int i = 0; i < length; i++) {
+ Object value = dataReader.getValueInTimestamp(timestamps[i]);
+ if (value != null) {
+ cnt++;
+ }
+ }
+
+ long preValue = resultData.getLongRet();
+ preValue += cnt;
+ resultData.setLongRet(preValue);
+ }
+
+ @Override
+ public boolean isCalculatedAggregationResult() {
+ return false;
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/FirstAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/FirstAggrFunc.java
new file mode 100644
index 0000000..3e8bb05
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/FirstAggrFunc.java
@@ -0,0 +1,154 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.exception.ProcessorException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class FirstAggrFunc extends AggregateFunction {
+
+ public FirstAggrFunc(TSDataType dataType) {
+ super(dataType);
+ }
+
+ @Override
+ public void init() {
+ resultData.reSet();
+ }
+
+ @Override
+ public AggreResultData getResult() {
+ return resultData;
+ }
+
+ @Override
+ public void calculateValueFromPageHeader(PageHeader pageHeader) throws ProcessorException {
+ if (resultData.isSetTime()) {
+ return;
+ }
+
+ Object firstVal = pageHeader.getStatistics().getFirst();
+ if (firstVal == null) {
+ throw new ProcessorException("PageHeader contains no FIRST value");
+ }
+ resultData.putTimeAndValue(0, firstVal);
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader)
+ throws IOException {
+ if (resultData.isSetTime()) {
+ return;
+ }
+ if (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+ if (dataInThisPage.currentTime() >= unsequenceReader.current().getTimestamp()) {
+ resultData.putTimeAndValue(0, unsequenceReader.current().getValue().getValue());
+ unsequenceReader.next();
+ return;
+ } else {
+ resultData.putTimeAndValue(0, dataInThisPage.currentValue());
+ dataInThisPage.next();
+ return;
+ }
+ }
+
+ if (dataInThisPage.hasNext()) {
+ resultData.putTimeAndValue(0, dataInThisPage.currentValue());
+ }
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+ long bound) throws IOException {
+ if (resultData.isSetTime()) {
+ return;
+ }
+ if (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+ if (dataInThisPage.currentTime() >= unsequenceReader.current().getTimestamp()) {
+ if (unsequenceReader.current().getTimestamp() < bound) {
+ resultData.putTimeAndValue(0, unsequenceReader.current().getValue().getValue());
+ unsequenceReader.next();
+ return;
+ }
+ } else {
+ if (dataInThisPage.currentTime() < bound) {
+ resultData.putTimeAndValue(0, dataInThisPage.currentValue());
+ dataInThisPage.next();
+ return;
+ }
+ }
+ }
+
+ if (dataInThisPage.hasNext() && dataInThisPage.currentTime() < bound) {
+ resultData.putTimeAndValue(0, dataInThisPage.currentValue());
+ dataInThisPage.next();
+ }
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+ throws IOException {
+ if (resultData.isSetTime()) {
+ return;
+ }
+ if (unsequenceReader.hasNext()) {
+ resultData.putTimeAndValue(0, unsequenceReader.current().getValue().getValue());
+ }
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+ throws IOException {
+ if (resultData.isSetTime()) {
+ return;
+ }
+ if (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+ resultData.putTimeAndValue(0, unsequenceReader.current().getValue().getValue());
+ }
+ }
+
+ @Override
+ public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+ EngineReaderByTimeStamp dataReader) throws IOException {
+ if (resultData.isSetTime()) {
+ return;
+ }
+
+ for (int i = 0; i < length; i++) {
+ Object value = dataReader.getValueInTimestamp(timestamps[i]);
+ if (value != null) {
+ resultData.putTimeAndValue(0, value);
+ break;
+ }
+ }
+ }
+
+ @Override
+ public boolean isCalculatedAggregationResult() {
+ return resultData.isSetTime();
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/LastAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/LastAggrFunc.java
new file mode 100644
index 0000000..8c12728
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/LastAggrFunc.java
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class LastAggrFunc extends AggregateFunction {
+
+ public LastAggrFunc(TSDataType dataType) {
+ super(dataType);
+ }
+
+ @Override
+ public void init() {
+ resultData.reSet();
+ }
+
+ @Override
+ public AggreResultData getResult() {
+ if (resultData.isSetTime()) {
+ resultData.setTimestamp(0);
+ }
+ return resultData;
+ }
+
+ @Override
+ public void calculateValueFromPageHeader(PageHeader pageHeader) {
+ Object lastVal = pageHeader.getStatistics().getLast();
+ updateLastResult(pageHeader.getMaxTimestamp(), lastVal);
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader)
+ throws IOException {
+ calculateValueFromPageData(dataInThisPage, unsequenceReader, Long.MAX_VALUE);
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+ long bound) throws IOException {
+ long time = -1;
+ Object lastVal = null;
+ while (dataInThisPage.hasNext() && dataInThisPage.currentTime() < bound) {
+ time = dataInThisPage.currentTime();
+ lastVal = dataInThisPage.currentValue();
+ dataInThisPage.next();
+ }
+
+ while (unsequenceReader.hasNext()) {
+ if (unsequenceReader.current().getTimestamp() < time) {
+ unsequenceReader.next();
+ } else if (unsequenceReader.current().getTimestamp() == time) {
+ lastVal = unsequenceReader.current().getValue().getValue();
+ unsequenceReader.next();
+ } else {
+ break;
+ }
+ }
+
+ // has inited lastVal and time in the batch(dataInThisPage).
+ if (time != -1) {
+ updateLastResult(time, lastVal);
+ }
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+ throws IOException {
+ TimeValuePair pair = null;
+ while (unsequenceReader.hasNext()) {
+ pair = unsequenceReader.next();
+ }
+
+ if (pair != null) {
+ updateLastResult(pair.getTimestamp(), pair.getValue().getValue());
+ }
+
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+ throws IOException {
+ TimeValuePair pair = null;
+ while (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+ pair = unsequenceReader.next();
+ }
+
+ if (pair != null) {
+ updateLastResult(pair.getTimestamp(), pair.getValue().getValue());
+ }
+ }
+
+ @Override
+ public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+ EngineReaderByTimeStamp dataReader) throws IOException {
+
+ long time = -1;
+ Object lastVal = null;
+ for (int i = 0; i < length; i++) {
+ Object value = dataReader.getValueInTimestamp(timestamps[i]);
+ if (value != null) {
+ time = timestamps[i];
+ lastVal = value;
+ }
+ }
+ if (time != -1) {
+ updateLastResult(time, lastVal);
+ }
+ }
+
+ @Override
+ public boolean isCalculatedAggregationResult() {
+ return false;
+ }
+
+ private void updateLastResult(long time, Object value) {
+ if (!resultData.isSetTime()) {
+ resultData.putTimeAndValue(time, value);
+ } else {
+ if (time >= resultData.getTimestamp()) {
+ resultData.putTimeAndValue(time, value);
+ }
+ }
+ }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxTimeAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxTimeAggrFunc.java
new file mode 100644
index 0000000..73b8f1c
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxTimeAggrFunc.java
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class MaxTimeAggrFunc extends AggregateFunction {
+
+ public MaxTimeAggrFunc() {
+ super(TSDataType.INT64);
+ }
+
+ @Override
+ public void init() {
+ resultData.reSet();
+ }
+
+ @Override
+ public AggreResultData getResult() {
+ return resultData;
+ }
+
+ @Override
+ public void calculateValueFromPageHeader(PageHeader pageHeader) {
+ long maxTimestamp = pageHeader.getMaxTimestamp();
+ updateMaxTimeResult(0, maxTimestamp);
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader) {
+
+ int maxIndex = dataInThisPage.length() - 1;
+ if (maxIndex < 0) {
+ return;
+ }
+ long time = dataInThisPage.getTimeByIndex(maxIndex);
+ updateMaxTimeResult(0, time);
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+ long bound) {
+ long time = -1;
+ while (dataInThisPage.hasNext()) {
+ if (dataInThisPage.currentTime() < bound) {
+ time = dataInThisPage.currentTime();
+ dataInThisPage.next();
+ } else {
+ break;
+ }
+ }
+ if (time != -1) {
+ updateMaxTimeResult(0, time);
+ }
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+ throws IOException {
+ TimeValuePair pair = null;
+ while (unsequenceReader.hasNext()) {
+ pair = unsequenceReader.next();
+ }
+ if (pair != null) {
+ updateMaxTimeResult(0, pair.getTimestamp());
+ }
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+ throws IOException {
+ TimeValuePair pair = null;
+ while (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+ pair = unsequenceReader.next();
+ }
+ if (pair != null) {
+ updateMaxTimeResult(0, pair.getTimestamp());
+ }
+ }
+
+ //TODO Consider how to reverse order in dataReader(EngineReaderByTimeStamp)
+ @Override
+ public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+ EngineReaderByTimeStamp dataReader) throws IOException {
+ long time = -1;
+ for (int i = 0; i < length; i++) {
+ Object value = dataReader.getValueInTimestamp(timestamps[i]);
+ if (value != null) {
+ time = timestamps[i];
+ }
+ }
+
+ if (time == -1) {
+ return;
+ }
+ updateMaxTimeResult(0, time);
+ }
+
+ @Override
+ public boolean isCalculatedAggregationResult() {
+ return false;
+ }
+
+ private void updateMaxTimeResult(long time, long value) {
+ if (!resultData.isSetValue() || value >= resultData.getLongRet()) {
+ resultData.setTimestamp(time);
+ resultData.setLongRet(value);
+ }
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxValueAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxValueAggrFunc.java
new file mode 100644
index 0000000..8e2d2a7
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxValueAggrFunc.java
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class MaxValueAggrFunc extends AggregateFunction {
+
+ public MaxValueAggrFunc(TSDataType dataType) {
+ super(dataType);
+ }
+
+ @Override
+ public void init() {
+ resultData.reSet();
+ }
+
+ @Override
+ public AggreResultData getResult() {
+ return resultData;
+ }
+
+ @Override
+ public void calculateValueFromPageHeader(PageHeader pageHeader) {
+ Comparable<Object> maxVal = (Comparable<Object>) pageHeader.getStatistics().getMax();
+ updateResult(maxVal);
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader)
+ throws IOException {
+ Comparable<Object> maxVal = null;
+ Object tmpVal = null;
+ while (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+ if (dataInThisPage.currentTime() < unsequenceReader.current().getTimestamp()) {
+ tmpVal = dataInThisPage.currentValue();
+ dataInThisPage.next();
+ } else if (dataInThisPage.currentTime() > unsequenceReader.current().getTimestamp()) {
+ tmpVal = unsequenceReader.current().getValue().getValue();
+ unsequenceReader.next();
+ } else {
+ tmpVal = unsequenceReader.current().getValue().getValue();
+ dataInThisPage.next();
+ unsequenceReader.next();
+ }
+
+ if (maxVal == null || maxVal.compareTo(tmpVal) < 0) {
+ maxVal = (Comparable<Object>) tmpVal;
+ }
+ }
+
+ while (dataInThisPage.hasNext()) {
+ if (maxVal == null || maxVal.compareTo(dataInThisPage.currentValue()) < 0) {
+ maxVal = (Comparable<Object>) dataInThisPage.currentValue();
+ }
+ dataInThisPage.next();
+ }
+ updateResult(maxVal);
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+ long bound) throws IOException {
+ Object tmpVal = null;
+ while (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+ long time = Math.min(dataInThisPage.currentTime(), unsequenceReader.current().getTimestamp());
+ if (time >= bound) {
+ break;
+ }
+
+ if (dataInThisPage.currentTime() == time) {
+ tmpVal = dataInThisPage.currentValue();
+ dataInThisPage.next();
+ }
+
+ if (unsequenceReader.current().getTimestamp() == time) {
+ tmpVal = unsequenceReader.current().getValue().getValue();
+ unsequenceReader.next();
+ }
+ updateResult((Comparable<Object>) tmpVal);
+ }
+
+ while (dataInThisPage.hasNext() && dataInThisPage.currentTime() < bound) {
+ updateResult((Comparable<Object>) dataInThisPage.currentValue());
+ dataInThisPage.next();
+ }
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+ throws IOException {
+ Comparable<Object> maxVal = null;
+ while (unsequenceReader.hasNext()) {
+ if (maxVal == null
+ || maxVal.compareTo(unsequenceReader.current().getValue().getValue()) < 0) {
+ maxVal = (Comparable<Object>) unsequenceReader.current().getValue().getValue();
+ }
+ unsequenceReader.next();
+ }
+ updateResult(maxVal);
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+ throws IOException {
+ Comparable<Object> maxVal = null;
+ while (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+ if (maxVal == null
+ || maxVal.compareTo(unsequenceReader.current().getValue().getValue()) < 0) {
+ maxVal = (Comparable<Object>) unsequenceReader.current().getValue().getValue();
+ }
+ unsequenceReader.next();
+ }
+ updateResult(maxVal);
+ }
+
+ @Override
+ public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+ EngineReaderByTimeStamp dataReader) throws IOException {
+ Comparable<Object> maxVal = null;
+ for (int i = 0; i < length; i++) {
+ Object value = dataReader.getValueInTimestamp(timestamps[i]);
+ if (value == null) {
+ continue;
+ }
+ if (maxVal == null || maxVal.compareTo(value) < 0) {
+ maxVal = (Comparable<Object>) value;
+ }
+ }
+ updateResult(maxVal);
+ }
+
+ @Override
+ public boolean isCalculatedAggregationResult() {
+ return false;
+ }
+
+ private void updateResult(Comparable<Object> maxVal) {
+ if (maxVal == null) {
+ return;
+ }
+ if (!resultData.isSetValue() || maxVal.compareTo(resultData.getValue()) > 0) {
+ resultData.putTimeAndValue(0, maxVal);
+ }
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MeanAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MeanAggrFunc.java
new file mode 100644
index 0000000..da6cc95
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MeanAggrFunc.java
@@ -0,0 +1,163 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class MeanAggrFunc extends AggregateFunction {
+
+ protected double sum = 0.0;
+ private int cnt = 0;
+ private TSDataType seriesDataType;
+
+ public MeanAggrFunc(TSDataType seriesDataType) {
+ super(TSDataType.DOUBLE);
+ this.seriesDataType = seriesDataType;
+ }
+
+ @Override
+ public void init() {
+ resultData.reSet();
+ sum = 0.0;
+ cnt = 0;
+ }
+
+ @Override
+ public AggreResultData getResult() {
+ if (cnt > 0) {
+ resultData.setTimestamp(0);
+ resultData.setDoubleRet(sum / cnt);
+ }
+ return resultData;
+ }
+
+ @Override
+ public void calculateValueFromPageHeader(PageHeader pageHeader) {
+ sum += pageHeader.getStatistics().getSum();
+ cnt += pageHeader.getNumOfValues();
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader)
+ throws IOException {
+ calculateValueFromPageData(dataInThisPage, unsequenceReader, false, 0);
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+ long bound) throws IOException {
+ calculateValueFromPageData(dataInThisPage, unsequenceReader, true, bound);
+ }
+
+ private void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+ boolean hasBound, long bound) throws IOException {
+ while (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+ Object sumVal = null;
+ long time = Math.min(dataInThisPage.currentTime(), unsequenceReader.current().getTimestamp());
+ if (hasBound && time >= bound) {
+ break;
+ }
+ if (dataInThisPage.currentTime() < unsequenceReader.current().getTimestamp()) {
+ sumVal = dataInThisPage.currentValue();
+ dataInThisPage.next();
+ } else if (dataInThisPage.currentTime() == unsequenceReader.current().getTimestamp()) {
+ sumVal = unsequenceReader.current().getValue().getValue();
+ dataInThisPage.next();
+ unsequenceReader.next();
+ } else {
+ sumVal = unsequenceReader.current().getValue().getValue();
+ unsequenceReader.next();
+ }
+ updateMean(seriesDataType, sumVal);
+ }
+
+ while (dataInThisPage.hasNext()) {
+ if (hasBound && dataInThisPage.currentTime() >= bound) {
+ break;
+ }
+ updateMean(seriesDataType, dataInThisPage.currentValue());
+ dataInThisPage.next();
+ }
+ }
+
+ private void updateMean(TSDataType type, Object sumVal) throws IOException {
+ switch (type) {
+ case INT32:
+ sum += (int) sumVal;
+ break;
+ case INT64:
+ sum += (long) sumVal;
+ break;
+ case FLOAT:
+ sum += (float) sumVal;
+ break;
+ case DOUBLE:
+ sum += (double) sumVal;
+ break;
+ case TEXT:
+ case BOOLEAN:
+ default:
+ throw new IOException("Unsupported data type in aggregation MEAN : " + type);
+ }
+ cnt++;
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+ throws IOException {
+ while (unsequenceReader.hasNext()) {
+ TimeValuePair pair = unsequenceReader.next();
+ updateMean(seriesDataType, pair.getValue().getValue());
+ }
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+ throws IOException {
+ while (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+ TimeValuePair pair = unsequenceReader.next();
+ updateMean(seriesDataType, pair.getValue().getValue());
+ }
+ }
+
+ @Override
+ public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+ EngineReaderByTimeStamp dataReader) throws IOException {
+ for (int i = 0; i < length; i++) {
+ Object value = dataReader.getValueInTimestamp(timestamps[i]);
+ if (value != null) {
+ updateMean(seriesDataType, value);
+ }
+ }
+ }
+
+ @Override
+ public boolean isCalculatedAggregationResult() {
+ return false;
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MinTimeAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MinTimeAggrFunc.java
new file mode 100644
index 0000000..f083812
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MinTimeAggrFunc.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.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class MinTimeAggrFunc extends AggregateFunction {
+
+ public MinTimeAggrFunc() {
+ super(TSDataType.INT64);
+ }
+
+ @Override
+ public void init() {
+ resultData.reSet();
+ }
+
+ @Override
+ public AggreResultData getResult() {
+ return resultData;
+ }
+
+ @Override
+ public void calculateValueFromPageHeader(PageHeader pageHeader) {
+ if (resultData.isSetValue()) {
+ return;
+ }
+ long time = pageHeader.getMinTimestamp();
+ resultData.putTimeAndValue(0, time);
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader)
+ throws IOException {
+ if (resultData.isSetValue()) {
+ return;
+ }
+
+ if (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+ if (dataInThisPage.currentTime() < unsequenceReader.current().getTimestamp()) {
+ resultData.setTimestamp(0);
+ resultData.setLongRet(dataInThisPage.currentTime());
+ } else {
+ resultData.setTimestamp(0);
+ resultData.setLongRet(unsequenceReader.current().getTimestamp());
+ }
+ return;
+ }
+
+ if (dataInThisPage.hasNext()) {
+ resultData.setTimestamp(0);
+ resultData.setLongRet(dataInThisPage.currentTime());
+ }
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+ long bound) throws IOException {
+ if (resultData.isSetValue()) {
+ return;
+ }
+
+ if (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+ if (dataInThisPage.currentTime() < unsequenceReader.current().getTimestamp()) {
+ if (dataInThisPage.currentTime() >= bound) {
+ return;
+ }
+ resultData.setTimestamp(0);
+ resultData.setLongRet(dataInThisPage.currentTime());
+ } else {
+ if (unsequenceReader.current().getTimestamp() >= bound) {
+ return;
+ }
+ resultData.setTimestamp(0);
+ resultData.setLongRet(unsequenceReader.current().getTimestamp());
+ }
+ return;
+ }
+
+ if (dataInThisPage.hasNext() && dataInThisPage.currentTime() < bound) {
+ resultData.setTimestamp(0);
+ resultData.setLongRet(dataInThisPage.currentTime());
+ dataInThisPage.next();
+ }
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+ throws IOException {
+ if (resultData.isSetValue()) {
+ return;
+ }
+ if (unsequenceReader.hasNext()) {
+ resultData.setTimestamp(0);
+ resultData.setLongRet(unsequenceReader.current().getTimestamp());
+ }
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+ throws IOException {
+ if (resultData.isSetValue()) {
+ return;
+ }
+ if (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+ resultData.setTimestamp(0);
+ resultData.setLongRet(unsequenceReader.current().getTimestamp());
+ }
+ }
+
+ @Override
+ public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+ EngineReaderByTimeStamp dataReader) throws IOException {
+ if (resultData.isSetValue()) {
+ return;
+ }
+ for (int i = 0; i < length; i++) {
+ Object value = dataReader.getValueInTimestamp(timestamps[i]);
+ if (value != null) {
+ resultData.setTimestamp(0);
+ resultData.setLongRet(timestamps[i]);
+ return;
+ }
+ }
+ }
+
+ @Override
+ public boolean isCalculatedAggregationResult() {
+ return resultData.isSetValue();
+ }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MinValueAggrFunc.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MinValueAggrFunc.java
new file mode 100644
index 0000000..ad64fb3
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MinValueAggrFunc.java
@@ -0,0 +1,166 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.aggregation.impl;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class MinValueAggrFunc extends AggregateFunction {
+
+ public MinValueAggrFunc(TSDataType dataType) {
+ super(dataType);
+ }
+
+ @Override
+ public void init() {
+ resultData.reSet();
+ }
+
+ @Override
+ public AggreResultData getResult() {
+ return resultData;
+ }
+
+ @Override
+ public void calculateValueFromPageHeader(PageHeader pageHeader) {
+ Comparable<Object> minVal = (Comparable<Object>) pageHeader.getStatistics().getMin();
+ updateResult(minVal);
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader)
+ throws IOException {
+ while (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+ if (dataInThisPage.currentTime() < unsequenceReader.current().getTimestamp()) {
+ updateResult((Comparable<Object>) dataInThisPage.currentValue());
+ dataInThisPage.next();
+ } else if (dataInThisPage.currentTime() == unsequenceReader.current().getTimestamp()) {
+ updateResult((Comparable<Object>) unsequenceReader.current().getValue().getValue());
+ dataInThisPage.next();
+ unsequenceReader.next();
+ } else {
+ updateResult((Comparable<Object>) unsequenceReader.current().getValue().getValue());
+ unsequenceReader.next();
+ }
+ }
+
+ Comparable<Object> minVal = null;
+ while (dataInThisPage.hasNext()) {
+ if (minVal == null
+ || minVal.compareTo(dataInThisPage.currentValue()) > 0) {
+ minVal = (Comparable<Object>) dataInThisPage.currentValue();
+ }
+ dataInThisPage.next();
+ }
+ updateResult(minVal);
+ }
+
+ @Override
+ public void calculateValueFromPageData(BatchData dataInThisPage, IPointReader unsequenceReader,
+ long bound) throws IOException {
+ while (dataInThisPage.hasNext() && unsequenceReader.hasNext()) {
+ long time = Math.min(dataInThisPage.currentTime(), unsequenceReader.current().getTimestamp());
+ if (time >= bound) {
+ break;
+ }
+
+ if (dataInThisPage.currentTime() == time) {
+ updateResult((Comparable<Object>) dataInThisPage.currentValue());
+ dataInThisPage.next();
+ }
+
+ if (unsequenceReader.current().getTimestamp() == time) {
+ updateResult((Comparable<Object>) unsequenceReader.current().getValue().getValue());
+ unsequenceReader.next();
+ }
+
+ }
+
+ while (dataInThisPage.hasNext() && dataInThisPage.currentTime() < bound) {
+ updateResult((Comparable<Object>) dataInThisPage.currentValue());
+ dataInThisPage.next();
+ }
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader)
+ throws IOException {
+ Comparable<Object> minVal = null;
+ while (unsequenceReader.hasNext()) {
+ if (minVal == null
+ || minVal.compareTo(unsequenceReader.current().getValue().getValue()) > 0) {
+ minVal = (Comparable<Object>) unsequenceReader.current().getValue().getValue();
+ }
+ unsequenceReader.next();
+ }
+ updateResult(minVal);
+ }
+
+ @Override
+ public void calculateValueFromUnsequenceReader(IPointReader unsequenceReader, long bound)
+ throws IOException {
+ Comparable<Object> minVal = null;
+ while (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < bound) {
+ if (minVal == null
+ || minVal.compareTo(unsequenceReader.current().getValue().getValue()) > 0) {
+ minVal = (Comparable<Object>) unsequenceReader.current().getValue().getValue();
+ }
+ unsequenceReader.next();
+ }
+ updateResult(minVal);
+ }
+
+ @Override
+ public void calcAggregationUsingTimestamps(long[] timestamps, int length,
+ EngineReaderByTimeStamp dataReader) throws IOException {
+ Comparable<Object> minVal = null;
+ for (int i = 0; i < length; i++) {
+ Object value = dataReader.getValueInTimestamp(timestamps[i]);
+ if (value == null) {
+ continue;
+ }
+ if (minVal == null || minVal.compareTo(value) > 0) {
+ minVal = (Comparable<Object>) value;
+ }
+ }
+ updateResult(minVal);
+ }
+
+ @Override
+ public boolean isCalculatedAggregationResult() {
+ return false;
+ }
+
+ private void updateResult(Comparable<Object> minVal) {
+ if (minVal == null) {
+ return;
+ }
+ if (!resultData.isSetValue() || minVal.compareTo(resultData.getValue()) < 0) {
+ resultData.putTimeAndValue(0, minVal);
+ }
+ }
+
+}
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/SumAggrFunc.java
similarity index 64%
copy from iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
copy to iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/SumAggrFunc.java
index 3ac1540..3bb8d7d 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/aggregation/impl/SumAggrFunc.java
@@ -1,29 +1,37 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.query.control;
-
-import org.junit.Test;
-
-public class QueryTokenManagerTest {
-
- @Test
- public void test() {
- //TODO
- }
-}
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.aggregation.impl;
+
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+
+public class SumAggrFunc extends MeanAggrFunc {
+
+ public SumAggrFunc(TSDataType seriesDataType) {
+ super(seriesDataType);
+ }
+
+ @Override
+ public AggreResultData getResult() {
+ resultData.setDoubleRet(sum);
+ resultData.setTimestamp(0);
+ return resultData;
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java b/iotdb/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
index 2a7769a..cfb06ad 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/context/QueryContext.java
@@ -44,6 +44,15 @@ public class QueryContext {
*/
private Map<String, List<Modification>> fileModCache = new HashMap<>();
+ private long jobId;
+
+ public QueryContext() {
+ }
+
+ public QueryContext(long jobId) {
+ this.jobId = jobId;
+ }
+
/**
* Find the modifications of timeseries 'path' in 'modFile'. If they are not in the cache, read
* them from 'modFile' and put then into the cache.
@@ -75,4 +84,12 @@ public class QueryContext {
return pathModifications;
}
+
+ public long getJobId() {
+ return jobId;
+ }
+
+ public void setJobId(long jobId) {
+ this.jobId = jobId;
+ }
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/control/OpenedFilePathsManager.java b/iotdb/src/main/java/org/apache/iotdb/db/query/control/JobFileManager.java
similarity index 55%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/control/OpenedFilePathsManager.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/control/JobFileManager.java
index bc6ee38..84e93c6 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/control/OpenedFilePathsManager.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/control/JobFileManager.java
@@ -27,45 +27,35 @@ import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
/**
* <p>
- * Singleton pattern, to manage all query tokens. Each jdbc request has an unique job id, in this jdbc request,
- * OpenedFilePathsManager manage all the opened files, and store in the set of current job id.
+ * JobFileManager records the paths of files that every query job uses for QueryResourceManager.
+ * <p>
*/
-public class OpenedFilePathsManager {
-
- /**
- * Each jdbc request has an unique jod id, job id is stored in thread local variable jobIdContainer.
- */
- private ThreadLocal<Long> jobIdContainer;
+public class JobFileManager {
/**
* Map<jobId, Set<filePaths>>
*/
- private ConcurrentHashMap<Long, Set<String>> closedFilePathsMap;
- private ConcurrentHashMap<Long, Set<String>> unclosedFilePathsMap;
+ private ConcurrentHashMap<Long, Set<String>> sealedFilePathsMap;
+ private ConcurrentHashMap<Long, Set<String>> unsealedFilePathsMap;
- private OpenedFilePathsManager() {
- jobIdContainer = new ThreadLocal<>();
- closedFilePathsMap = new ConcurrentHashMap<>();
- unclosedFilePathsMap = new ConcurrentHashMap<>();
- }
-
- public static OpenedFilePathsManager getInstance() {
- return OpenedFilePathsManagerHelper.INSTANCE;
+ public JobFileManager() {
+ sealedFilePathsMap = new ConcurrentHashMap<>();
+ unsealedFilePathsMap = new ConcurrentHashMap<>();
}
/**
- * Set job id for current request thread. When a query request is created firstly, this method must be invoked.
+ * Set job id for current request thread. When a query request is created firstly,
+ * this method must be invoked.
*/
- public void setJobIdForCurrentRequestThread(long jobId) {
- jobIdContainer.set(jobId);
- closedFilePathsMap.put(jobId, new HashSet<>());
- unclosedFilePathsMap.put(jobId, new HashSet<>());
+ public void addJobId(long jobId) {
+ sealedFilePathsMap.computeIfAbsent(jobId, x -> new HashSet<>());
+ unsealedFilePathsMap.computeIfAbsent(jobId, x -> new HashSet<>());
}
/**
- * Add the unique file paths to closedFilePathsMap and unclosedFilePathsMap.
+ * Add the unique file paths to sealedFilePathsMap and unsealedFilePathsMap.
*/
- public void addUsedFilesForCurrentRequestThread(long jobId, QueryDataSource dataSource) {
+ public void addUsedFilesForGivenJob(long jobId, QueryDataSource dataSource) {
for (TsFileResource tsFileResource : dataSource.getSeqDataSource().getSealedTsFiles()) {
String sealedFilePath = tsFileResource.getFilePath();
addFilePathToMap(jobId, sealedFilePath, true);
@@ -88,42 +78,30 @@ public class OpenedFilePathsManager {
* Whenever the jdbc request is closed normally or abnormally, this method must be invoked. All file paths used by
* this jdbc request must be cleared and thus the usage reference must be decreased.
*/
- public void removeUsedFilesForCurrentRequestThread() {
- if (jobIdContainer.get() != null) {
- long jobId = jobIdContainer.get();
- jobIdContainer.remove();
-
- for (String filePath : closedFilePathsMap.get(jobId)) {
- FileReaderManager.getInstance().decreaseFileReaderReference(filePath, true);
- }
- closedFilePathsMap.remove(jobId);
- for (String filePath : unclosedFilePathsMap.get(jobId)) {
+ void removeUsedFilesForGivenJob(long jobId) {
+ for (String filePath : sealedFilePathsMap.get(jobId)) {
FileReaderManager.getInstance().decreaseFileReaderReference(filePath, false);
}
- unclosedFilePathsMap.remove(jobId);
- }
+ sealedFilePathsMap.remove(jobId);
+ for (String filePath : unsealedFilePathsMap.get(jobId)) {
+ FileReaderManager.getInstance().decreaseFileReaderReference(filePath, true);
+ }
+ unsealedFilePathsMap.remove(jobId);
}
/**
* Increase the usage reference of filePath of job id. Before the invoking of this method,
* <code>this.setJobIdForCurrentRequestThread</code> has been invoked,
- * so <code>closedFilePathsMap.get(jobId)</code> or <code>unclosedFilePathsMap.get(jobId)</code>
+ * so <code>sealedFilePathsMap.get(jobId)</code> or <code>unsealedFilePathsMap.get(jobId)</code>
* must not return null.
*/
- public void addFilePathToMap(long jobId, String filePath, boolean isClosed) {
- ConcurrentHashMap<Long, Set<String>> pathMap = !isClosed ? unclosedFilePathsMap :
- closedFilePathsMap;
+ void addFilePathToMap(long jobId, String filePath, boolean isSealed) {
+ ConcurrentHashMap<Long, Set<String>> pathMap = !isSealed ? unsealedFilePathsMap :
+ sealedFilePathsMap;
+ //TODO this is not an atomic operation, is there concurrent problem?
if (!pathMap.get(jobId).contains(filePath)) {
pathMap.get(jobId).add(filePath);
- FileReaderManager.getInstance().increaseFileReaderReference(filePath, isClosed);
- }
- }
-
- private static class OpenedFilePathsManagerHelper {
- private static final OpenedFilePathsManager INSTANCE = new OpenedFilePathsManager();
-
- private OpenedFilePathsManagerHelper() {
-
+ FileReaderManager.getInstance().increaseFileReaderReference(filePath, isSealed);
}
}
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryDataSourceManager.java b/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryDataSourceManager.java
deleted file mode 100644
index f3fd1f3..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryDataSourceManager.java
+++ /dev/null
@@ -1,53 +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.db.query.control;
-
-import org.apache.iotdb.db.engine.filenode.FileNodeManager;
-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.tsfile.read.common.Path;
-import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
-
-/**
- * <p>
- * This class is used to get query data source of a given path. See the component of
- * <code>QueryDataSource</code>
- */
-public class QueryDataSourceManager {
-
- private static FileNodeManager fileNodeManager = FileNodeManager.getInstance();
-
- private QueryDataSourceManager() {
- }
-
- public static QueryDataSource getQueryDataSource(long jobId, Path selectedPath,
- QueryContext context)
- throws FileNodeManagerException {
-
- SingleSeriesExpression singleSeriesExpression = new SingleSeriesExpression(selectedPath, null);
- QueryDataSource queryDataSource = fileNodeManager.query(singleSeriesExpression, context);
-
- // add used files to current thread request cached map
- OpenedFilePathsManager.getInstance()
- .addUsedFilesForCurrentRequestThread(jobId, queryDataSource);
-
- return queryDataSource;
- }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryTokenManager.java b/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
similarity index 64%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryTokenManager.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
index e304675..a535d87 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryTokenManager.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java
@@ -24,8 +24,11 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
import org.apache.iotdb.db.engine.filenode.FileNodeManager;
+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.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.read.expression.ExpressionType;
import org.apache.iotdb.tsfile.read.expression.IBinaryExpression;
@@ -34,18 +37,17 @@ import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
/**
* <p>
- * Singleton pattern, to manage all query tokens. Each jdbc query request can query multiple series,
- * in the processing of querying different device id, the <code>FileNodeManager.getInstance().
- * beginQuery</code> and <code>FileNodeManager.getInstance().endQuery</code> must be invoked in the
- * beginning and ending of jdbc request.
+ * QueryResourceManager manages resource (file streams) used by each query job, and assign Ids to the jobs.
+ * During the life cycle of a query, the following methods must be called in strict order:
+ * 1. assignJobId - get an Id for the new job.
+ * 2. beginQueryOfGivenQueryPaths - remind FileNodeManager that some files are being used
+ * 3. (if using filter)beginQueryOfGivenExpression
+ * - remind FileNodeManager that some files are being used
+ * 4. getQueryDataSource - open files for the job or reuse existing readers.
+ * 5. endQueryForGivenJob - release the resource used by this job.
* </p>
*/
-public class QueryTokenManager {
-
- /**
- * Each jdbc request has unique jod id, job id is stored in thread local variable jobContainer.
- */
- private ThreadLocal<Long> jobContainer;
+public class QueryResourceManager {
/**
* Map<jobId, Map<deviceId, List<token>>>.
@@ -72,35 +74,41 @@ public class QueryTokenManager {
* <code>FileNodeManager.getInstance().beginQuery(device_2)</code> will be invoked again, it
* returns result token `3` and `4` .
*
- * <code>FileNodeManager.getInstance().endQuery(device_1, 1)</code> and
- * <code>FileNodeManager.getInstance().endQuery(device_2, 2)</code> must be invoked no matter how
+ * <code>FileNodeManager.getInstance().endQueryForGivenJob(device_1, 1)</code> and
+ * <code>FileNodeManager.getInstance().endQueryForGivenJob(device_2, 2)</code> must be invoked no matter how
* query process Q1 exits normally or abnormally. So is Q2,
- * <code>FileNodeManager.getInstance().endQuery(device_1, 3)</code> and
- * <code>FileNodeManager.getInstance().endQuery(device_2, 4)</code> must be invoked
+ * <code>FileNodeManager.getInstance().endQueryForGivenJob(device_1, 3)</code> and
+ * <code>FileNodeManager.getInstance().endQueryForGivenJob(device_2, 4)</code> must be invoked
*
* Last but no least, to ensure the correctness of write process and query process of IoTDB,
* <code>FileNodeManager.getInstance().beginQuery()</code> and
- * <code>FileNodeManager.getInstance().endQuery()</code> must be executed rightly.
+ * <code>FileNodeManager.getInstance().endQueryForGivenJob()</code> must be executed rightly.
* </p>
*/
private ConcurrentHashMap<Long, ConcurrentHashMap<String, List<Integer>>> queryTokensMap;
+ private JobFileManager filePathsManager;
+ private AtomicLong maxJobId;
+
- private QueryTokenManager() {
- jobContainer = new ThreadLocal<>();
+ private QueryResourceManager() {
queryTokensMap = new ConcurrentHashMap<>();
+ filePathsManager = new JobFileManager();
+ maxJobId = new AtomicLong(0);
}
- public static QueryTokenManager getInstance() {
+ public static QueryResourceManager getInstance() {
return QueryTokenManagerHelper.INSTANCE;
}
/**
- * Set job id for current request thread. When a query request is created firstly, this method
+ * Assign a jobId for a new query job. When a query request is created firstly, this method
* must be invoked.
*/
- public void setJobIdForCurrentRequestThread(long jobId) {
- jobContainer.set(jobId);
- queryTokensMap.put(jobId, new ConcurrentHashMap<>());
+ public long assignJobId() {
+ long jobId = maxJobId.incrementAndGet();
+ queryTokensMap.computeIfAbsent(jobId, x -> new ConcurrentHashMap<>());
+ filePathsManager.addJobId(jobId);
+ return jobId;
}
/**
@@ -132,22 +140,37 @@ public class QueryTokenManager {
}
}
+ public QueryDataSource getQueryDataSource(Path selectedPath,
+ QueryContext context)
+ throws FileNodeManagerException {
+
+ SingleSeriesExpression singleSeriesExpression = new SingleSeriesExpression(selectedPath, null);
+ QueryDataSource queryDataSource = FileNodeManager.getInstance()
+ .query(singleSeriesExpression, context);
+
+ // add used files to current thread request cached map
+ filePathsManager.addUsedFilesForGivenJob(context.getJobId(), queryDataSource);
+
+ return queryDataSource;
+ }
+
/**
* Whenever the jdbc request is closed normally or abnormally, this method must be invoked. All
* query tokens created by this jdbc request must be cleared.
*/
- public void endQueryForCurrentRequestThread() throws FileNodeManagerException {
- if (jobContainer.get() != null) {
- long jobId = jobContainer.get();
- jobContainer.remove();
-
+ public void endQueryForGivenJob(long jobId) throws FileNodeManagerException {
+ if (queryTokensMap.get(jobId) == null) {
+ // no resource need to be released.
+ return;
+ }
for (Map.Entry<String, List<Integer>> entry : queryTokensMap.get(jobId).entrySet()) {
for (int token : entry.getValue()) {
FileNodeManager.getInstance().endQuery(entry.getKey(), token);
}
}
queryTokensMap.remove(jobId);
- }
+ // remove usage of opened file paths of current thread
+ filePathsManager.removeUsedFilesForGivenJob(jobId);
}
private void getUniquePaths(IExpression expression, Set<String> deviceIdSet) {
@@ -161,15 +184,12 @@ public class QueryTokenManager {
}
private void putQueryTokenForCurrentRequestThread(long jobId, String deviceId, int queryToken) {
- if (!queryTokensMap.get(jobId).containsKey(deviceId)) {
- queryTokensMap.get(jobId).put(deviceId, new ArrayList<>());
- }
- queryTokensMap.get(jobId).get(deviceId).add(queryToken);
+ queryTokensMap.get(jobId).computeIfAbsent(deviceId, x -> new ArrayList<>()).add(queryToken);
}
private static class QueryTokenManagerHelper {
- private static final QueryTokenManager INSTANCE = new QueryTokenManager();
+ private static final QueryResourceManager INSTANCE = new QueryResourceManager();
private QueryTokenManagerHelper() {
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/AggreResultDataPointReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/AggreResultDataPointReader.java
new file mode 100644
index 0000000..c2edab0
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/AggreResultDataPointReader.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with the License. You may obtain
+ * a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.dataset;
+
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.db.utils.TimeValuePairUtils;
+
+public class AggreResultDataPointReader implements IPointReader {
+
+ private AggreResultData aggreResultData;
+
+ public AggreResultDataPointReader(AggreResultData aggreResultData) {
+ this.aggreResultData = aggreResultData;
+ }
+
+ @Override
+ public boolean hasNext() {
+ return aggreResultData.isSetValue();
+ }
+
+ @Override
+ public TimeValuePair next() {
+ TimeValuePair timeValuePair = TimeValuePairUtils.getCurrentTimeValuePair(aggreResultData);
+ aggreResultData.reSet();
+ return timeValuePair;
+ }
+
+ @Override
+ public TimeValuePair current() {
+ return TimeValuePairUtils.getCurrentTimeValuePair(aggreResultData);
+ }
+
+ @Override
+ public void close() {
+ // batch data doesn't need to close.
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java
index 20d93e8..cdaa0fb 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java
@@ -22,7 +22,6 @@ import java.io.IOException;
import java.util.List;
import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
import org.apache.iotdb.db.query.timegenerator.EngineTimeGenerator;
-import org.apache.iotdb.db.utils.TsPrimitiveType;
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;
@@ -62,11 +61,11 @@ public class EngineDataSetWithTimeGenerator extends QueryDataSet {
RowRecord rowRecord = new RowRecord(timestamp);
for (int i = 0; i < readers.size(); i++) {
EngineReaderByTimeStamp reader = readers.get(i);
- TsPrimitiveType tsPrimitiveType = reader.getValueInTimestamp(timestamp);
- if (tsPrimitiveType == null) {
+ Object value = reader.getValueInTimestamp(timestamp);
+ if (value == null) {
rowRecord.addField(new Field(null));
} else {
- rowRecord.addField(getField(tsPrimitiveType.getValue(), dataTypes.get(i)));
+ rowRecord.addField(getField(value, dataTypes.get(i)));
}
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithoutTimeGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithoutTimeGenerator.java
index 4c0ccb7..73fc71f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithoutTimeGenerator.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithoutTimeGenerator.java
@@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.query.dataset;
import java.io.IOException;
@@ -23,7 +24,7 @@ import java.util.HashSet;
import java.util.List;
import java.util.PriorityQueue;
import java.util.Set;
-import org.apache.iotdb.db.query.reader.IReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
import org.apache.iotdb.db.utils.TimeValuePair;
import org.apache.iotdb.db.utils.TsPrimitiveType;
import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
@@ -34,11 +35,11 @@ import org.apache.iotdb.tsfile.read.common.RowRecord;
import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
/**
- * TODO implement this class as TsFile DataSetWithoutTimeGenerator
+ * TODO implement this class as TsFile DataSetWithoutTimeGenerator.
*/
public class EngineDataSetWithoutTimeGenerator extends QueryDataSet {
- private List<IReader> readers;
+ private List<IPointReader> readers;
private TimeValuePair[] cacheTimeValueList;
@@ -55,7 +56,7 @@ public class EngineDataSetWithoutTimeGenerator extends QueryDataSet {
* @throws IOException IOException
*/
public EngineDataSetWithoutTimeGenerator(List<Path> paths, List<TSDataType> dataTypes,
- List<IReader> readers)
+ List<IPointReader> readers)
throws IOException {
super(paths, dataTypes);
this.readers = readers;
@@ -68,7 +69,7 @@ public class EngineDataSetWithoutTimeGenerator extends QueryDataSet {
cacheTimeValueList = new TimeValuePair[readers.size()];
for (int i = 0; i < readers.size(); i++) {
- IReader reader = readers.get(i);
+ IPointReader reader = readers.get(i);
if (reader.hasNext()) {
TimeValuePair timeValuePair = reader.next();
cacheTimeValueList[i] = timeValuePair;
@@ -89,7 +90,7 @@ public class EngineDataSetWithoutTimeGenerator extends QueryDataSet {
RowRecord record = new RowRecord(minTime);
for (int i = 0; i < readers.size(); i++) {
- IReader reader = readers.get(i);
+ IPointReader reader = readers.get(i);
if (cacheTimeValueList[i] == null) {
record.addField(new Field(null));
} else {
@@ -109,6 +110,9 @@ public class EngineDataSetWithoutTimeGenerator extends QueryDataSet {
}
private Field getField(TsPrimitiveType tsPrimitiveType, TSDataType dataType) {
+ if (tsPrimitiveType == null) {
+ return new Field(null);
+ }
Field field = new Field(dataType);
switch (dataType) {
case INT32:
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/AggregateEngineExecutor.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/AggregateEngineExecutor.java
new file mode 100644
index 0000000..ba76c2f
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/AggregateEngineExecutor.java
@@ -0,0 +1,330 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.executor;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+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.query.aggregation.AggreFuncFactory;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+import org.apache.iotdb.db.query.aggregation.impl.LastAggrFunc;
+import org.apache.iotdb.db.query.aggregation.impl.MaxTimeAggrFunc;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.dataset.AggreResultDataPointReader;
+import org.apache.iotdb.db.query.dataset.EngineDataSetWithoutTimeGenerator;
+import org.apache.iotdb.db.query.factory.SeriesReaderFactory;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader;
+import org.apache.iotdb.db.query.reader.sequence.SequenceDataReader;
+import org.apache.iotdb.db.query.timegenerator.EngineTimeGenerator;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+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.IExpression;
+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 AggregateEngineExecutor {
+
+ private List<Path> selectedSeries;
+ private List<String> aggres;
+ private IExpression expression;
+
+ /**
+ * aggregation batch calculation size.
+ **/
+ private int aggregateFetchSize;
+
+ /**
+ * constructor.
+ */
+ public AggregateEngineExecutor(List<Path> selectedSeries, List<String> aggres,
+ IExpression expression) {
+ this.selectedSeries = selectedSeries;
+ this.aggres = aggres;
+ this.expression = expression;
+ this.aggregateFetchSize = 10 * IoTDBDescriptor.getInstance().getConfig().getFetchSize();
+ }
+
+ /**
+ * execute aggregate function with only time filter or no filter.
+ *
+ * @param context query context
+ */
+ public QueryDataSet executeWithOutTimeGenerator(QueryContext context)
+ throws FileNodeManagerException, IOException, PathErrorException, ProcessorException {
+ Filter timeFilter = null;
+ if (expression != null) {
+ timeFilter = ((GlobalTimeExpression) expression).getFilter();
+ }
+ QueryResourceManager
+ .getInstance().beginQueryOfGivenQueryPaths(context.getJobId(), selectedSeries);
+
+ List<SequenceDataReader> readersOfSequenceData = new ArrayList<>();
+ List<IPointReader> readersOfUnSequenceData = new ArrayList<>();
+ List<AggregateFunction> aggregateFunctions = new ArrayList<>();
+ for (int i = 0; i < selectedSeries.size(); i++) {
+ // construct AggregateFunction
+ TSDataType tsDataType = MManager.getInstance()
+ .getSeriesType(selectedSeries.get(i).getFullPath());
+ AggregateFunction function = AggreFuncFactory.getAggrFuncByName(aggres.get(i), tsDataType);
+ function.init();
+ aggregateFunctions.add(function);
+
+ QueryDataSource queryDataSource = QueryResourceManager.getInstance()
+ .getQueryDataSource(selectedSeries.get(i), context);
+
+ // sequence reader for sealed tsfile, unsealed tsfile, memory
+ SequenceDataReader sequenceReader;
+ if (function instanceof MaxTimeAggrFunc || function instanceof LastAggrFunc) {
+ sequenceReader = new SequenceDataReader(queryDataSource.getSeqDataSource(), timeFilter,
+ context, true);
+ } else {
+ sequenceReader = new SequenceDataReader(queryDataSource.getSeqDataSource(), timeFilter,
+ context, false);
+ }
+
+ // unseq reader for all chunk groups in unSeqFile, memory
+ PriorityMergeReader unSeqMergeReader = SeriesReaderFactory.getInstance()
+ .createUnSeqMergeReader(queryDataSource.getOverflowSeriesDataSource(), timeFilter);
+ readersOfSequenceData.add(sequenceReader);
+ readersOfUnSequenceData.add(unSeqMergeReader);
+ }
+ List<AggreResultData> aggreResultDataList = new ArrayList<>();
+ //TODO use multi-thread
+ for (int i = 0; i < selectedSeries.size(); i++) {
+ AggreResultData aggreResultData = aggregateWithOutTimeGenerator(aggregateFunctions.get(i),
+ readersOfSequenceData.get(i), readersOfUnSequenceData.get(i), timeFilter);
+ aggreResultDataList.add(aggreResultData);
+ }
+ return constructDataSet(aggreResultDataList);
+ }
+
+ /**
+ * calculation aggregate result with only time filter or no filter for one series.
+ *
+ * @param function aggregate function
+ * @param sequenceReader sequence data reader
+ * @param unSequenceReader unsequence data reader
+ * @param filter time filter or null
+ * @return one series aggregate result data
+ */
+ private AggreResultData aggregateWithOutTimeGenerator(AggregateFunction function,
+ SequenceDataReader sequenceReader, IPointReader unSequenceReader, Filter filter)
+ throws IOException, ProcessorException {
+ if (function instanceof MaxTimeAggrFunc || function instanceof LastAggrFunc) {
+ return handleLastMaxTimeWithOutTimeGenerator(function, sequenceReader, unSequenceReader,
+ filter);
+ }
+
+ while (sequenceReader.hasNext()) {
+ PageHeader pageHeader = sequenceReader.nextPageHeader();
+ // judge if overlap with unsequence data
+ if (canUseHeader(function, pageHeader, unSequenceReader, filter)) {
+ // cal by pageHeader
+ function.calculateValueFromPageHeader(pageHeader);
+ sequenceReader.skipPageData();
+ } else {
+ // cal by pageData
+ function.calculateValueFromPageData(sequenceReader.nextBatch(), unSequenceReader);
+ }
+
+ if (function.isCalculatedAggregationResult()) {
+ return function.getResult();
+ }
+ }
+
+ // cal with unsequence data
+ if (unSequenceReader.hasNext()) {
+ function.calculateValueFromUnsequenceReader(unSequenceReader);
+ }
+ return function.getResult();
+ }
+
+ /**
+ * determine whether pageHeader can be used to compute aggregation results.
+ */
+ private boolean canUseHeader(AggregateFunction function, PageHeader pageHeader,
+ IPointReader unSequenceReader, Filter filter)
+ throws IOException, ProcessorException {
+ // if page data is memory data.
+ if (pageHeader == null) {
+ return false;
+ }
+
+ long minTime = pageHeader.getMinTimestamp();
+ long maxTime = pageHeader.getMaxTimestamp();
+
+ // If there are points in the page that do not satisfy the time filter,
+ // page header cannot be used to calculate.
+ if (filter != null && !filter.containStartEndTime(minTime, maxTime)) {
+ return false;
+ }
+
+ // cal unsequence data with timestamps between pages.
+ function.calculateValueFromUnsequenceReader(unSequenceReader, minTime);
+
+ return !(unSequenceReader.hasNext() && unSequenceReader.current().getTimestamp() <= maxTime);
+
+ }
+
+ /**
+ * handle last and max_time aggregate function with only time filter or no filter.
+ *
+ * @param function aggregate function
+ * @param sequenceReader sequence data reader
+ * @param unSequenceReader unsequence data reader
+ * @return BatchData-aggregate result
+ */
+ private AggreResultData handleLastMaxTimeWithOutTimeGenerator(AggregateFunction function,
+ SequenceDataReader sequenceReader, IPointReader unSequenceReader, Filter timeFilter)
+ throws IOException, ProcessorException {
+ long lastBatchTimeStamp = Long.MIN_VALUE;
+ boolean isChunkEnd = false;
+ while (sequenceReader.hasNext()) {
+ PageHeader pageHeader = sequenceReader.nextPageHeader();
+ // judge if overlap with unsequence data
+ if (canUseHeader(function, pageHeader, unSequenceReader, timeFilter)) {
+ // cal by pageHeader
+ function.calculateValueFromPageHeader(pageHeader);
+ sequenceReader.skipPageData();
+
+ if (lastBatchTimeStamp > pageHeader.getMinTimestamp()) {
+ // the chunk is end.
+ isChunkEnd = true;
+ } else {
+ // current page and last page are in the same chunk.
+ lastBatchTimeStamp = pageHeader.getMinTimestamp();
+ }
+ } else {
+ // cal by pageData
+ BatchData batchData = sequenceReader.nextBatch();
+ if (lastBatchTimeStamp > batchData.currentTime()) {
+ // the chunk is end.
+ isChunkEnd = true;
+ } else {
+ // current page and last page are in the same chunk.
+ lastBatchTimeStamp = batchData.currentTime();
+ }
+ function.calculateValueFromPageData(batchData, unSequenceReader);
+ }
+ if (isChunkEnd) {
+ break;
+ }
+ }
+
+ // cal with unsequence data
+ if (unSequenceReader.hasNext()) {
+ function.calculateValueFromUnsequenceReader(unSequenceReader);
+ }
+ return function.getResult();
+ }
+
+
+ /**
+ * execute aggregate function with value filter.
+ *
+ * @param context query context.
+ */
+ public QueryDataSet executeWithTimeGenerator(QueryContext context)
+ throws FileNodeManagerException, PathErrorException, IOException, ProcessorException {
+ QueryResourceManager
+ .getInstance().beginQueryOfGivenQueryPaths(context.getJobId(), selectedSeries);
+ QueryResourceManager.getInstance().beginQueryOfGivenExpression(context.getJobId(), expression);
+
+ EngineTimeGenerator timestampGenerator = new EngineTimeGenerator(expression, context);
+ List<EngineReaderByTimeStamp> readersOfSelectedSeries = SeriesReaderFactory
+ .getByTimestampReadersOfSelectedPaths(selectedSeries, context);
+
+ List<AggregateFunction> aggregateFunctions = new ArrayList<>();
+ for (int i = 0; i < selectedSeries.size(); i++) {
+ TSDataType type = MManager.getInstance().getSeriesType(selectedSeries.get(i).getFullPath());
+ AggregateFunction function = AggreFuncFactory.getAggrFuncByName(aggres.get(i), type);
+ function.init();
+ aggregateFunctions.add(function);
+ }
+ List<AggreResultData> batchDataList = aggregateWithTimeGenerator(aggregateFunctions,
+ timestampGenerator,
+ readersOfSelectedSeries);
+ return constructDataSet(batchDataList);
+ }
+
+ /**
+ * calculation aggregate result with value filter.
+ */
+ private List<AggreResultData> aggregateWithTimeGenerator(
+ List<AggregateFunction> aggregateFunctions,
+ EngineTimeGenerator timestampGenerator,
+ List<EngineReaderByTimeStamp> readersOfSelectedSeries)
+ throws IOException {
+
+ while (timestampGenerator.hasNext()) {
+
+ // generate timestamps for aggregate
+ long[] timeArray = new long[aggregateFetchSize];
+ int timeArrayLength = 0;
+ for (int cnt = 0; cnt < aggregateFetchSize; cnt++) {
+ if (!timestampGenerator.hasNext()) {
+ break;
+ }
+ timeArray[timeArrayLength++] = timestampGenerator.next();
+ }
+
+ // cal part of aggregate result
+ for (int i = 0; i < readersOfSelectedSeries.size(); i++) {
+ aggregateFunctions.get(i).calcAggregationUsingTimestamps(timeArray, timeArrayLength,
+ readersOfSelectedSeries.get(i));
+ }
+ }
+
+ List<AggreResultData> aggreResultDataArrayList = new ArrayList<>();
+ for (AggregateFunction function : aggregateFunctions) {
+ aggreResultDataArrayList.add(function.getResult());
+ }
+ return aggreResultDataArrayList;
+ }
+
+ /**
+ * using aggregate result data list construct QueryDataSet.
+ *
+ * @param aggreResultDataList aggregate result data list
+ */
+ private QueryDataSet constructDataSet(List<AggreResultData> aggreResultDataList)
+ throws IOException {
+ List<TSDataType> dataTypes = new ArrayList<>();
+ List<IPointReader> resultDataPointReaders = new ArrayList<>();
+ for (AggreResultData resultData : aggreResultDataList) {
+ dataTypes.add(resultData.getDataType());
+ resultDataPointReaders.add(new AggreResultDataPointReader(resultData));
+ }
+ return new EngineDataSetWithoutTimeGenerator(selectedSeries, dataTypes, resultDataPointReaders);
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java
index 13f0053..3b40187 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java
@@ -16,24 +16,20 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.query.executor;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
-import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
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.QueryDataSourceManager;
-import org.apache.iotdb.db.query.control.QueryTokenManager;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
import org.apache.iotdb.db.query.dataset.EngineDataSetWithTimeGenerator;
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.PriorityMergeReader;
-import org.apache.iotdb.db.query.reader.merge.PriorityMergeReaderByTimestamp;
-import org.apache.iotdb.db.query.reader.sequence.SequenceDataReader;
import org.apache.iotdb.db.query.timegenerator.EngineTimeGenerator;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.common.Path;
@@ -46,10 +42,8 @@ import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
public class EngineExecutorWithTimeGenerator {
private QueryExpression queryExpression;
- private long jobId;
- EngineExecutorWithTimeGenerator(long jobId, QueryExpression queryExpression) {
- this.jobId = jobId;
+ EngineExecutorWithTimeGenerator(QueryExpression queryExpression) {
this.queryExpression = queryExpression;
}
@@ -62,17 +56,17 @@ public class EngineExecutorWithTimeGenerator {
*/
public QueryDataSet execute(QueryContext context) throws FileNodeManagerException {
- QueryTokenManager.getInstance()
- .beginQueryOfGivenQueryPaths(jobId, queryExpression.getSelectedSeries());
- QueryTokenManager.getInstance()
- .beginQueryOfGivenExpression(jobId, queryExpression.getExpression());
+ QueryResourceManager.getInstance()
+ .beginQueryOfGivenQueryPaths(context.getJobId(), queryExpression.getSelectedSeries());
+ QueryResourceManager.getInstance()
+ .beginQueryOfGivenExpression(context.getJobId(), queryExpression.getExpression());
EngineTimeGenerator timestampGenerator;
List<EngineReaderByTimeStamp> readersOfSelectedSeries;
try {
- timestampGenerator = new EngineTimeGenerator(jobId, queryExpression.getExpression(), context);
- readersOfSelectedSeries = getReadersOfSelectedPaths(queryExpression.getSelectedSeries(),
- context);
+ timestampGenerator = new EngineTimeGenerator(queryExpression.getExpression(), context);
+ readersOfSelectedSeries = SeriesReaderFactory
+ .getByTimestampReadersOfSelectedPaths(queryExpression.getSelectedSeries(), context);
} catch (IOException ex) {
throw new FileNodeManagerException(ex);
}
@@ -92,33 +86,4 @@ public class EngineExecutorWithTimeGenerator {
readersOfSelectedSeries);
}
- private List<EngineReaderByTimeStamp> getReadersOfSelectedPaths(List<Path> paths,
- QueryContext context)
- throws IOException, FileNodeManagerException {
-
- List<EngineReaderByTimeStamp> readersOfSelectedSeries = new ArrayList<>();
-
- for (Path path : paths) {
-
- QueryDataSource queryDataSource = QueryDataSourceManager.getQueryDataSource(jobId, path,
- context);
-
- PriorityMergeReaderByTimestamp mergeReaderByTimestamp = new PriorityMergeReaderByTimestamp();
-
- // reader for sequence data
- SequenceDataReader tsFilesReader = new SequenceDataReader(queryDataSource.getSeqDataSource(),
- null, context);
- mergeReaderByTimestamp.addReaderWithPriority(tsFilesReader, 1);
-
- // reader for unSequence data
- PriorityMergeReader unSeqMergeReader = SeriesReaderFactory.getInstance()
- .createUnSeqMergeReader(queryDataSource.getOverflowSeriesDataSource(), null);
- mergeReaderByTimestamp.addReaderWithPriority(unSeqMergeReader, 2);
-
- readersOfSelectedSeries.add(mergeReaderByTimestamp);
- }
-
- return readersOfSelectedSeries;
- }
-
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithoutTimeGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithoutTimeGenerator.java
index 11c94bc..2d07796 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithoutTimeGenerator.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithoutTimeGenerator.java
@@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.query.executor;
import java.io.IOException;
@@ -26,11 +27,11 @@ 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.QueryDataSourceManager;
-import org.apache.iotdb.db.query.control.QueryTokenManager;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
import org.apache.iotdb.db.query.dataset.EngineDataSetWithoutTimeGenerator;
import org.apache.iotdb.db.query.factory.SeriesReaderFactory;
-import org.apache.iotdb.db.query.reader.IReader;
+import org.apache.iotdb.db.query.reader.AllDataReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader;
import org.apache.iotdb.db.query.reader.sequence.SequenceDataReader;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
@@ -46,10 +47,8 @@ import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
public class EngineExecutorWithoutTimeGenerator {
private QueryExpression queryExpression;
- private long jobId;
- public EngineExecutorWithoutTimeGenerator(long jobId, QueryExpression queryExpression) {
- this.jobId = jobId;
+ public EngineExecutorWithoutTimeGenerator(QueryExpression queryExpression) {
this.queryExpression = queryExpression;
}
@@ -61,15 +60,15 @@ public class EngineExecutorWithoutTimeGenerator {
Filter timeFilter = ((GlobalTimeExpression) queryExpression.getExpression()).getFilter();
- List<IReader> readersOfSelectedSeries = new ArrayList<>();
+ List<IPointReader> readersOfSelectedSeries = new ArrayList<>();
List<TSDataType> dataTypes = new ArrayList<>();
- QueryTokenManager.getInstance()
- .beginQueryOfGivenQueryPaths(jobId, queryExpression.getSelectedSeries());
+ QueryResourceManager.getInstance()
+ .beginQueryOfGivenQueryPaths(context.getJobId(), queryExpression.getSelectedSeries());
for (Path path : queryExpression.getSelectedSeries()) {
- QueryDataSource queryDataSource = QueryDataSourceManager.getQueryDataSource(jobId, path,
+ QueryDataSource queryDataSource = QueryResourceManager.getInstance().getQueryDataSource(path,
context);
// add data type
@@ -79,14 +78,11 @@ public class EngineExecutorWithoutTimeGenerator {
throw new FileNodeManagerException(e);
}
- PriorityMergeReader priorityReader = new PriorityMergeReader();
-
// sequence reader for one sealed tsfile
SequenceDataReader tsFilesReader;
try {
tsFilesReader = new SequenceDataReader(queryDataSource.getSeqDataSource(),
timeFilter, context);
- priorityReader.addReaderWithPriority(tsFilesReader, PriorityMergeReader.LOW_PRIORITY);
} catch (IOException e) {
throw new FileNodeManagerException(e);
}
@@ -96,12 +92,12 @@ public class EngineExecutorWithoutTimeGenerator {
try {
unSeqMergeReader = SeriesReaderFactory.getInstance()
.createUnSeqMergeReader(queryDataSource.getOverflowSeriesDataSource(), timeFilter);
- priorityReader.addReaderWithPriority(unSeqMergeReader, PriorityMergeReader.HIGH_PRIORITY);
} catch (IOException e) {
throw new FileNodeManagerException(e);
}
- readersOfSelectedSeries.add(priorityReader);
+ // merge sequence data with unsequence data.
+ readersOfSelectedSeries.add(new AllDataReader(tsFilesReader, unSeqMergeReader));
}
try {
@@ -118,15 +114,15 @@ public class EngineExecutorWithoutTimeGenerator {
public QueryDataSet executeWithoutFilter(QueryContext context)
throws FileNodeManagerException {
- List<IReader> readersOfSelectedSeries = new ArrayList<>();
+ List<IPointReader> readersOfSelectedSeries = new ArrayList<>();
List<TSDataType> dataTypes = new ArrayList<>();
- QueryTokenManager.getInstance()
- .beginQueryOfGivenQueryPaths(jobId, queryExpression.getSelectedSeries());
+ QueryResourceManager.getInstance()
+ .beginQueryOfGivenQueryPaths(context.getJobId(), queryExpression.getSelectedSeries());
for (Path path : queryExpression.getSelectedSeries()) {
- QueryDataSource queryDataSource = QueryDataSourceManager.getQueryDataSource(jobId, path,
+ QueryDataSource queryDataSource = QueryResourceManager.getInstance().getQueryDataSource(path,
context);
// add data type
@@ -136,14 +132,11 @@ public class EngineExecutorWithoutTimeGenerator {
throw new FileNodeManagerException(e);
}
- PriorityMergeReader priorityReader = new PriorityMergeReader();
-
// sequence insert data
SequenceDataReader tsFilesReader;
try {
tsFilesReader = new SequenceDataReader(queryDataSource.getSeqDataSource(),
null, context);
- priorityReader.addReaderWithPriority(tsFilesReader, 1);
} catch (IOException e) {
throw new FileNodeManagerException(e);
}
@@ -153,12 +146,12 @@ public class EngineExecutorWithoutTimeGenerator {
try {
unSeqMergeReader = SeriesReaderFactory.getInstance()
.createUnSeqMergeReader(queryDataSource.getOverflowSeriesDataSource(), null);
- priorityReader.addReaderWithPriority(unSeqMergeReader, 2);
} catch (IOException e) {
throw new FileNodeManagerException(e);
}
- readersOfSelectedSeries.add(priorityReader);
+ // merge sequence data with unsequence data.
+ readersOfSelectedSeries.add(new AllDataReader(tsFilesReader, unSeqMergeReader));
}
try {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineQueryRouter.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineQueryRouter.java
index c9327f4..96ad5ad 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineQueryRouter.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineQueryRouter.java
@@ -16,22 +16,32 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.db.query.executor;
-import static org.apache.iotdb.tsfile.read.expression.ExpressionType.GLOBAL_TIME;
+package org.apache.iotdb.db.query.executor;
import java.io.IOException;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
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.control.OpenedFilePathsManager;
-import org.apache.iotdb.db.query.control.QueryTokenManager;
+import org.apache.iotdb.db.query.executor.groupby.GroupByWithOnlyTimeFilterDataSetDataSet;
+import org.apache.iotdb.db.query.executor.groupby.GroupByWithValueFilterDataSetDataSet;
+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.impl.BinaryExpression;
+import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
import org.apache.iotdb.tsfile.read.expression.util.ExpressionOptimizer;
+import org.apache.iotdb.tsfile.read.filter.TimeFilter;
import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Pair;
/**
* Query entrance class of IoTDB query process. All query clause will be transformed to physical
@@ -39,40 +49,26 @@ import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
*/
public class EngineQueryRouter {
- /**
- * Each unique jdbc request(query, aggregation or others job) has an unique job id. This job id
- * will always be maintained until the request is closed. In each job, the unique file will be
- * only opened once to avoid too many opened files error.
- */
- private AtomicLong jobIdGenerator = new AtomicLong();
+
/**
* execute physical plan.
*/
- public QueryDataSet query(QueryExpression queryExpression)
+ public QueryDataSet query(QueryExpression queryExpression, QueryContext context)
throws FileNodeManagerException {
- long nextJobId = getNextJobId();
- QueryTokenManager.getInstance().setJobIdForCurrentRequestThread(nextJobId);
- OpenedFilePathsManager.getInstance().setJobIdForCurrentRequestThread(nextJobId);
-
- QueryContext context = new QueryContext();
-
if (queryExpression.hasQueryFilter()) {
try {
IExpression optimizedExpression = ExpressionOptimizer.getInstance()
.optimize(queryExpression.getExpression(), queryExpression.getSelectedSeries());
queryExpression.setExpression(optimizedExpression);
- if (optimizedExpression.getType() == GLOBAL_TIME) {
+ if (optimizedExpression.getType() == ExpressionType.GLOBAL_TIME) {
EngineExecutorWithoutTimeGenerator engineExecutor =
- new EngineExecutorWithoutTimeGenerator(
-
- nextJobId, queryExpression);
+ new EngineExecutorWithoutTimeGenerator(queryExpression);
return engineExecutor.executeWithGlobalTimeFilter(context);
} else {
EngineExecutorWithTimeGenerator engineExecutor = new EngineExecutorWithTimeGenerator(
- nextJobId,
queryExpression);
return engineExecutor.execute(context);
}
@@ -82,13 +78,145 @@ public class EngineQueryRouter {
}
} else {
EngineExecutorWithoutTimeGenerator engineExecutor = new EngineExecutorWithoutTimeGenerator(
- nextJobId,
queryExpression);
return engineExecutor.executeWithoutFilter(context);
}
}
- private synchronized long getNextJobId() {
- return jobIdGenerator.incrementAndGet();
+ /**
+ * execute aggregation query.
+ */
+ public QueryDataSet aggregate(List<Path> selectedSeries, List<String> aggres,
+ IExpression expression, QueryContext context) throws QueryFilterOptimizationException,
+ FileNodeManagerException, IOException, PathErrorException, ProcessorException {
+
+ if (expression != null) {
+ IExpression optimizedExpression = ExpressionOptimizer.getInstance()
+ .optimize(expression, selectedSeries);
+ AggregateEngineExecutor engineExecutor = new AggregateEngineExecutor(
+ selectedSeries, aggres, optimizedExpression);
+ if (optimizedExpression.getType() == ExpressionType.GLOBAL_TIME) {
+ return engineExecutor.executeWithOutTimeGenerator(context);
+ } else {
+ return engineExecutor.executeWithTimeGenerator(context);
+ }
+ } else {
+ AggregateEngineExecutor engineExecutor = new AggregateEngineExecutor(
+ selectedSeries, aggres, null);
+ return engineExecutor.executeWithOutTimeGenerator(context);
+ }
}
+
+ /**
+ * execute groupBy query.
+ *
+ * @param selectedSeries select path list
+ * @param aggres aggregation name list
+ * @param expression filter expression
+ * @param unit time granularity for interval partitioning, unit is ms.
+ * @param origin the datum time point for interval division is divided into a time interval for
+ * each TimeUnit time from this point forward and backward.
+ * @param intervals time intervals, closed interval.
+ */
+ 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 {
+
+ long nextJobId = context.getJobId();
+
+ // check the legitimacy of intervals
+ for (Pair<Long, Long> pair : intervals) {
+ if (!(pair.left > 0 && pair.right > 0)) {
+ throw new ProcessorException(
+ String.format("Time interval<%d, %d> must be greater than 0.", pair.left, pair.right));
+ }
+ if (pair.right < pair.left) {
+ throw new ProcessorException(String.format(
+ "Interval starting time must be greater than the interval ending time, "
+ + "found error interval<%d, %d>", pair.left, pair.right));
+ }
+ }
+ // merge intervals
+ List<Pair<Long, Long>> mergedIntervalList = mergeInterval(intervals);
+
+ // construct groupBy intervals filter
+ BinaryExpression intervalFilter = null;
+ for (Pair<Long, Long> pair : mergedIntervalList) {
+ BinaryExpression pairFilter = BinaryExpression
+ .and(new GlobalTimeExpression(TimeFilter.gtEq(pair.left)),
+ new GlobalTimeExpression(TimeFilter.ltEq(pair.right)));
+ if (intervalFilter != null) {
+ intervalFilter = BinaryExpression.or(intervalFilter, pairFilter);
+ } else {
+ intervalFilter = pairFilter;
+ }
+ }
+
+ // merge interval filter and filtering conditions after where statements
+ if (expression == null) {
+ expression = intervalFilter;
+ } else {
+ expression = BinaryExpression.and(expression, intervalFilter);
+ }
+
+ IExpression optimizedExpression = ExpressionOptimizer.getInstance()
+ .optimize(expression, selectedSeries);
+ if (optimizedExpression.getType() == ExpressionType.GLOBAL_TIME) {
+ GroupByWithOnlyTimeFilterDataSetDataSet groupByEngine = new GroupByWithOnlyTimeFilterDataSetDataSet(
+ nextJobId, selectedSeries, unit, origin, mergedIntervalList);
+ groupByEngine.initGroupBy(context, aggres, optimizedExpression);
+ return groupByEngine;
+ } else {
+ GroupByWithValueFilterDataSetDataSet groupByEngine = new GroupByWithValueFilterDataSetDataSet(
+ nextJobId,
+ selectedSeries, unit, origin, mergedIntervalList);
+ groupByEngine.initGroupBy(context, aggres, optimizedExpression);
+ return groupByEngine;
+ }
+ }
+
+ /**
+ * execute fill query.
+ *
+ * @param fillPaths select path list
+ * @param queryTime timestamp
+ * @param fillType type IFill map
+ */
+ public QueryDataSet fill(List<Path> fillPaths, long queryTime, Map<TSDataType, IFill> fillType,
+ QueryContext context)
+ throws FileNodeManagerException, PathErrorException, IOException {
+
+ long nextJobId = context.getJobId();
+
+ FillEngineExecutor fillEngineExecutor = new FillEngineExecutor(nextJobId, fillPaths, queryTime,
+ fillType);
+ return fillEngineExecutor.execute(context);
+ }
+
+ /**
+ * sort intervals by start time and merge overlapping intervals.
+ *
+ * @param intervals time interval
+ */
+ private List<Pair<Long, Long>> mergeInterval(List<Pair<Long, Long>> intervals) {
+ // sort by interval start time.
+ intervals.sort(((o1, o2) -> (int) (o1.left - o2.left)));
+
+ LinkedList<Pair<Long, Long>> merged = new LinkedList<>();
+ for (Pair<Long, Long> interval : intervals) {
+ // if the list of merged intervals is empty or
+ // if the current interval does not overlap with the previous, simply append it.
+ if (merged.isEmpty() || merged.getLast().right < interval.left) {
+ merged.add(interval);
+ } else {
+ // otherwise, there is overlap, so we merge the current and previous intervals.
+ merged.getLast().right = Math.max(merged.getLast().right, interval.right);
+ }
+ }
+ return merged;
+ }
+
+
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/FillEngineExecutor.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/FillEngineExecutor.java
new file mode 100644
index 0000000..83c5fa9
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/FillEngineExecutor.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.db.query.executor;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+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.dataset.EngineDataSetWithoutTimeGenerator;
+import org.apache.iotdb.db.query.fill.IFill;
+import org.apache.iotdb.db.query.fill.PreviousFill;
+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.query.dataset.QueryDataSet;
+
+public class FillEngineExecutor {
+
+ private long jobId;
+ private List<Path> selectedSeries;
+ private long queryTime;
+ private Map<TSDataType, IFill> typeIFillMap;
+
+ public FillEngineExecutor(long jobId, List<Path> selectedSeries, long queryTime,
+ Map<TSDataType, IFill> typeIFillMap) {
+ this.jobId = jobId;
+ this.selectedSeries = selectedSeries;
+ this.queryTime = queryTime;
+ this.typeIFillMap = typeIFillMap;
+ }
+
+ /**
+ * execute fill.
+ *
+ * @param context query context
+ */
+ public QueryDataSet execute(QueryContext context)
+ throws FileNodeManagerException, PathErrorException, IOException {
+ QueryResourceManager.getInstance().beginQueryOfGivenQueryPaths(jobId, selectedSeries);
+
+ List<IFill> fillList = new ArrayList<>();
+ List<TSDataType> dataTypeList = new ArrayList<>();
+ for (Path path : selectedSeries) {
+ QueryDataSource queryDataSource = QueryResourceManager.getInstance()
+ .getQueryDataSource(path, context);
+ TSDataType dataType = MManager.getInstance().getSeriesType(path.getFullPath());
+ dataTypeList.add(dataType);
+ IFill fill = null;
+ if (!typeIFillMap.containsKey(dataType)) {
+ fill = new PreviousFill(dataType, queryTime, 0);
+ } else {
+ fill = typeIFillMap.get(dataType).copy(path);
+ }
+ fill.setDataType(dataType);
+ fill.setQueryTime(queryTime);
+ fill.constructReaders(queryDataSource, context);
+ fillList.add(fill);
+ }
+
+ List<IPointReader> readers = new ArrayList<>();
+ for (IFill fill : fillList) {
+ readers.add(fill.getFillResult());
+ }
+
+ return new EngineDataSetWithoutTimeGenerator(selectedSeries, dataTypeList, readers);
+ }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByEngineDataSet.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByEngineDataSet.java
new file mode 100644
index 0000000..2ccecf5
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByEngineDataSet.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.db.query.executor.groupby;
+
+import java.util.ArrayList;
+import java.util.List;
+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.query.aggregation.AggreFuncFactory;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+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.query.dataset.QueryDataSet;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public abstract class GroupByEngineDataSet extends QueryDataSet {
+
+ protected long jobId;
+ protected List<Path> selectedSeries;
+ private long unit;
+ private long origin;
+ private List<Pair<Long, Long>> mergedIntervals;
+
+ protected long startTime;
+ protected long endTime;
+ private int usedIndex;
+ protected List<AggregateFunction> functions;
+ protected boolean hasCachedTimeInterval;
+
+ /**
+ * groupBy query.
+ */
+ public GroupByEngineDataSet(long jobId, List<Path> paths, long unit, long origin,
+ List<Pair<Long, Long>> mergedIntervals) {
+ super(paths);
+ this.jobId = jobId;
+ this.selectedSeries = paths;
+ this.unit = unit;
+ this.origin = origin;
+ this.mergedIntervals = mergedIntervals;
+ this.functions = new ArrayList<>();
+
+ // init group by time partition
+ this.usedIndex = 0;
+ this.hasCachedTimeInterval = false;
+ this.endTime = -1;
+ }
+
+ protected void initAggreFuction(List<String> aggres)
+ throws PathErrorException, ProcessorException {
+
+ List<TSDataType> types = new ArrayList<>();
+ // construct AggregateFunctions
+ for (int i = 0; i < paths.size(); i++) {
+ TSDataType tsDataType = MManager.getInstance()
+ .getSeriesType(selectedSeries.get(i).getFullPath());
+ AggregateFunction function = AggreFuncFactory.getAggrFuncByName(aggres.get(i), tsDataType);
+ function.init();
+ functions.add(function);
+ types.add(function.getResultDataType());
+ }
+ super.setDataTypes(types);
+ }
+
+ @Override
+ public boolean hasNext() {
+ // has cached
+ if (hasCachedTimeInterval) {
+ return true;
+ }
+
+ // end
+ if (usedIndex >= mergedIntervals.size()) {
+ return false;
+ }
+
+ // skip the intervals in coverage of last time-partition
+ while (usedIndex < mergedIntervals.size() && mergedIntervals.get(usedIndex).right < endTime) {
+ usedIndex++;
+ }
+ if (usedIndex >= mergedIntervals.size()) {
+ return false;
+ }
+
+ // initialize the start-end time of next interval
+ if (endTime < mergedIntervals.get(usedIndex).left) {
+ // interval start time
+ startTime = mergedIntervals.get(usedIndex).left;
+ if (origin > startTime) {
+ endTime = origin - (origin - startTime) / unit * unit;
+ } else {
+ endTime = origin + (startTime - origin) / unit * unit + unit;
+ }
+ hasCachedTimeInterval = true;
+ return true;
+ }
+
+ // current interval is not covered yet
+ if (endTime <= mergedIntervals.get(usedIndex).right) {
+ startTime = endTime;
+ endTime += unit;
+ hasCachedTimeInterval = true;
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * this method is only used in the test class to get the next time partition.
+ */
+ public Pair<Long, Long> nextTimePartition() {
+ hasCachedTimeInterval = false;
+ return new Pair<>(startTime, endTime);
+ }
+
+ protected Field getField(AggreResultData aggreResultData) {
+ if (!aggreResultData.isSetValue()) {
+ return new Field(null);
+ }
+ Field field = new Field(aggreResultData.getDataType());
+ switch (aggreResultData.getDataType()) {
+ case INT32:
+ field.setIntV(aggreResultData.getIntRet());
+ break;
+ case INT64:
+ field.setLongV(aggreResultData.getLongRet());
+ break;
+ case FLOAT:
+ field.setFloatV(aggreResultData.getFloatRet());
+ break;
+ case DOUBLE:
+ field.setDoubleV(aggreResultData.getDoubleRet());
+ break;
+ case BOOLEAN:
+ field.setBoolV(aggreResultData.isBooleanRet());
+ break;
+ case TEXT:
+ field.setBinaryV(aggreResultData.getBinaryRet());
+ break;
+ default:
+ throw new UnSupportedDataTypeException("UnSupported: " + aggreResultData.getDataType());
+ }
+ return field;
+ }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByWithOnlyTimeFilterDataSetDataSet.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByWithOnlyTimeFilterDataSetDataSet.java
new file mode 100644
index 0000000..e82b567
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByWithOnlyTimeFilterDataSetDataSet.java
@@ -0,0 +1,314 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.executor.groupby;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+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.aggregation.AggreResultData;
+import org.apache.iotdb.db.query.aggregation.AggregateFunction;
+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.IAggregateReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader;
+import org.apache.iotdb.db.query.reader.sequence.SequenceDataReader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+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.expression.IExpression;
+import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public class GroupByWithOnlyTimeFilterDataSetDataSet extends GroupByEngineDataSet {
+
+ protected List<IPointReader> unSequenceReaderList;
+ protected List<IAggregateReader> sequenceReaderList;
+ private List<BatchData> batchDataList;
+ private List<Boolean> hasCachedSequenceDataList;
+ private Filter timeFilter;
+
+ /**
+ * constructor.
+ */
+ public GroupByWithOnlyTimeFilterDataSetDataSet(long jobId, List<Path> paths, long unit,
+ long origin, List<Pair<Long, Long>> mergedIntervals) {
+ super(jobId, paths, unit, origin, mergedIntervals);
+ this.unSequenceReaderList = new ArrayList<>();
+ this.sequenceReaderList = new ArrayList<>();
+ this.timeFilter = null;
+ this.hasCachedSequenceDataList = new ArrayList<>();
+ this.batchDataList = new ArrayList<>();
+ for (int i = 0; i < paths.size(); i++) {
+ hasCachedSequenceDataList.add(false);
+ batchDataList.add(null);
+ }
+ }
+
+ /**
+ * init reader and aggregate function.
+ */
+ public void initGroupBy(QueryContext context, List<String> aggres, IExpression expression)
+ throws FileNodeManagerException, PathErrorException, ProcessorException, IOException {
+ initAggreFuction(aggres);
+ // init reader
+ QueryResourceManager.getInstance().beginQueryOfGivenQueryPaths(jobId, selectedSeries);
+ if (expression != null) {
+ timeFilter = ((GlobalTimeExpression) expression).getFilter();
+ }
+ for (int i = 0; i < selectedSeries.size(); i++) {
+ QueryDataSource queryDataSource = QueryResourceManager.getInstance()
+ .getQueryDataSource(selectedSeries.get(i), context);
+
+ // sequence reader for sealed tsfile, unsealed tsfile, memory
+ SequenceDataReader sequenceReader = new SequenceDataReader(queryDataSource.getSeqDataSource(),
+ timeFilter, context, false);
+
+ // unseq reader for all chunk groups in unSeqFile, memory
+ PriorityMergeReader unSeqMergeReader = SeriesReaderFactory.getInstance()
+ .createUnSeqMergeReader(queryDataSource.getOverflowSeriesDataSource(), timeFilter);
+
+ sequenceReaderList.add(sequenceReader);
+ unSequenceReaderList.add(unSeqMergeReader);
+ }
+
+ }
+
+ @Override
+ public RowRecord next() throws IOException {
+ if (!hasCachedTimeInterval) {
+ throw new IOException("need to call hasNext() before calling next() "
+ + "in GroupByWithOnlyTimeFilterDataSetDataSet.");
+ }
+ hasCachedTimeInterval = false;
+ RowRecord record = new RowRecord(startTime);
+ for (int i = 0; i < functions.size(); i++) {
+ AggreResultData res = null;
+ try {
+ res = nextSeries(i);
+ } catch (ProcessorException e) {
+ throw new IOException(e);
+ }
+ if (res == null) {
+ record.addField(new Field(null));
+ } else {
+ record.addField(getField(res));
+ }
+ }
+ return record;
+ }
+
+ /**
+ * calculate the group by result of the series indexed by idx.
+ *
+ * @param idx series id
+ */
+ private AggreResultData nextSeries(int idx) throws IOException, ProcessorException {
+ IPointReader unsequenceReader = unSequenceReaderList.get(idx);
+ IAggregateReader sequenceReader = sequenceReaderList.get(idx);
+ AggregateFunction function = functions.get(idx);
+ function.init();
+
+ // skip the points with timestamp less than startTime
+ skipBeforeStartTimeData(idx, sequenceReader, unsequenceReader);
+
+ // cal group by in batch data
+ boolean finishCheckSequenceData = calGroupByInBatchData(idx, function, unsequenceReader);
+ if (finishCheckSequenceData) {
+ // check unsequence data
+ function.calculateValueFromUnsequenceReader(unsequenceReader, endTime);
+ return function.getResult().deepCopy();
+ }
+
+ // continue checking sequence data
+ while (sequenceReader.hasNext()) {
+ PageHeader pageHeader = sequenceReader.nextPageHeader();
+
+ // memory data
+ if (pageHeader == null) {
+ batchDataList.set(idx, sequenceReader.nextBatch());
+ hasCachedSequenceDataList.set(idx, true);
+ finishCheckSequenceData = calGroupByInBatchData(idx, function, unsequenceReader);
+ } else {
+ // page data
+ long minTime = pageHeader.getMinTimestamp();
+ long maxTime = pageHeader.getMaxTimestamp();
+ // no point in sequence data with a timestamp less than endTime
+ if (minTime >= endTime) {
+ finishCheckSequenceData = true;
+ } else if (canUseHeader(minTime, maxTime, unsequenceReader, function)) {
+ // cal using page header
+ function.calculateValueFromPageHeader(pageHeader);
+ sequenceReader.skipPageData();
+ } else {
+ // cal using page data
+ batchDataList.set(idx, sequenceReader.nextBatch());
+ hasCachedSequenceDataList.set(idx, true);
+ finishCheckSequenceData = calGroupByInBatchData(idx, function, unsequenceReader);
+ }
+
+ if (finishCheckSequenceData) {
+ break;
+ }
+ }
+ }
+ // cal using unsequence data
+ function.calculateValueFromUnsequenceReader(unsequenceReader, endTime);
+ return function.getResult().deepCopy();
+ }
+
+ /**
+ * calculate groupBy's result in batch data.
+ *
+ * @param idx series index
+ * @param function aggregate function of the series
+ * @param unsequenceReader unsequence reader of the series
+ * @return if all sequential data been computed
+ */
+ private boolean calGroupByInBatchData(int idx, AggregateFunction function,
+ IPointReader unsequenceReader)
+ throws IOException, ProcessorException {
+ BatchData batchData = batchDataList.get(idx);
+ boolean hasCachedSequenceData = hasCachedSequenceDataList.get(idx);
+ boolean finishCheckSequenceData = false;
+ // there was unprocessed data in last batch
+ if (hasCachedSequenceData && batchData.hasNext()) {
+ function.calculateValueFromPageData(batchData, unsequenceReader, endTime);
+ }
+
+ if (hasCachedSequenceData && batchData.hasNext()) {
+ finishCheckSequenceData = true;
+ } else {
+ hasCachedSequenceData = false;
+ }
+ batchDataList.set(idx, batchData);
+ hasCachedSequenceDataList.set(idx, hasCachedSequenceData);
+ return finishCheckSequenceData;
+ }
+
+ /**
+ * skip the points with timestamp less than startTime.
+ *
+ * @param idx the index of series
+ * @param sequenceReader sequence Reader
+ * @param unsequenceReader unsequence Reader
+ * @throws IOException exception when reading file
+ */
+ private void skipBeforeStartTimeData(int idx, IAggregateReader sequenceReader,
+ IPointReader unsequenceReader)
+ throws IOException {
+
+ // skip the unsequenceReader points with timestamp less than startTime
+ skipPointInUnsequenceData(unsequenceReader);
+
+ // skip the cached batch data points with timestamp less than startTime
+ if (skipPointInBatchData(idx)) {
+ return;
+ }
+
+ // skip the points in sequenceReader data whose timestamp are less than startTime
+ while (sequenceReader.hasNext()) {
+ PageHeader pageHeader = sequenceReader.nextPageHeader();
+ // memory data
+ if (pageHeader == null) {
+ batchDataList.set(idx, sequenceReader.nextBatch());
+ hasCachedSequenceDataList.set(idx, true);
+ if (skipPointInBatchData(idx)) {
+ return;
+ }
+ } else {
+ // page data
+
+ // timestamps of all points in the page are less than startTime
+ if (pageHeader.getMaxTimestamp() < startTime) {
+ sequenceReader.skipPageData();
+ continue;
+ } else if (pageHeader.getMinTimestamp() >= startTime) {
+ // timestamps of all points in the page are greater or equal to startTime, needn't to skip
+ return;
+ }
+ // the page has overlap with startTime
+ batchDataList.set(idx, sequenceReader.nextBatch());
+ hasCachedSequenceDataList.set(idx, true);
+ if (skipPointInBatchData(idx)) {
+ return;
+ }
+ }
+ }
+ }
+
+ /**
+ * skip points in unsequence reader whose timestamp is less than startTime.
+ *
+ * @param unsequenceReader unsequence reader
+ */
+ private void skipPointInUnsequenceData(IPointReader unsequenceReader) throws IOException {
+ while (unsequenceReader.hasNext() && unsequenceReader.current().getTimestamp() < startTime) {
+ unsequenceReader.next();
+ }
+ }
+
+ /**
+ * skip points in batch data whose timestamp is less than startTime.
+ *
+ * @param idx series index
+ * @return whether has next in batch data
+ */
+ private boolean skipPointInBatchData(int idx) {
+ BatchData batchData = batchDataList.get(idx);
+ boolean hasCachedSequenceData = hasCachedSequenceDataList.get(idx);
+ if (!hasCachedSequenceData) {
+ return false;
+ }
+
+ // skip the cached batch data points with timestamp less than startTime
+ while (batchData.hasNext() && batchData.currentTime() < startTime) {
+ batchData.next();
+ }
+ batchDataList.set(idx, batchData);
+ if (batchData.hasNext()) {
+ return true;
+ } else {
+ hasCachedSequenceDataList.set(idx, false);
+ return false;
+ }
+ }
+
+ private boolean canUseHeader(long minTime, long maxTime, IPointReader unSequenceReader,
+ AggregateFunction function)
+ throws IOException, ProcessorException {
+ if (timeFilter != null && !timeFilter.containStartEndTime(minTime, maxTime)) {
+ return false;
+ }
+
+ // cal unsequence data with timestamps between pages.
+ function.calculateValueFromUnsequenceReader(unSequenceReader, minTime);
+
+ return !(unSequenceReader.hasNext() && unSequenceReader.current().getTimestamp() <= maxTime);
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByWithValueFilterDataSetDataSet.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByWithValueFilterDataSetDataSet.java
new file mode 100644
index 0000000..7a6b3d2
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/groupby/GroupByWithValueFilterDataSetDataSet.java
@@ -0,0 +1,160 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.executor.groupby;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+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.aggregation.AggregateFunction;
+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.timegenerator.EngineTimeGenerator;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.common.RowRecord;
+import org.apache.iotdb.tsfile.read.expression.IExpression;
+import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGenerator;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+public class GroupByWithValueFilterDataSetDataSet extends GroupByEngineDataSet {
+
+ private List<EngineReaderByTimeStamp> allDataReaderList;
+ private TimeGenerator timestampGenerator;
+ /**
+ * cached timestamp for next group by partition.
+ */
+ private long timestamp;
+ /**
+ * if this object has cached timestamp for next group by partition.
+ */
+ private boolean hasCachedTimestamp;
+
+ /**
+ * group by batch calculation size.
+ */
+ private int timeStampFetchSize;
+
+ /**
+ * constructor.
+ */
+ public GroupByWithValueFilterDataSetDataSet(long jobId, List<Path> paths, long unit, long origin,
+ List<Pair<Long, Long>> mergedIntervals) {
+ super(jobId, paths, unit, origin, mergedIntervals);
+ this.allDataReaderList = new ArrayList<>();
+ this.timeStampFetchSize = 10 * IoTDBDescriptor.getInstance().getConfig().getFetchSize();
+ }
+
+ /**
+ * init reader and aggregate function.
+ */
+ public void initGroupBy(QueryContext context, List<String> aggres, IExpression expression)
+ throws FileNodeManagerException, PathErrorException, ProcessorException, IOException {
+ initAggreFuction(aggres);
+
+ QueryResourceManager.getInstance().beginQueryOfGivenExpression(context.getJobId(), expression);
+ QueryResourceManager
+ .getInstance().beginQueryOfGivenQueryPaths(context.getJobId(), selectedSeries);
+ this.timestampGenerator = new EngineTimeGenerator(expression, context);
+ this.allDataReaderList = SeriesReaderFactory
+ .getByTimestampReadersOfSelectedPaths(selectedSeries, context);
+ }
+
+ @Override
+ public RowRecord next() throws IOException {
+ if (!hasCachedTimeInterval) {
+ throw new IOException("need to call hasNext() before calling next()"
+ + " in GroupByWithOnlyTimeFilterDataSetDataSet.");
+ }
+ hasCachedTimeInterval = false;
+ for (AggregateFunction function : functions) {
+ function.init();
+ }
+
+ long[] timestampArray = new long[timeStampFetchSize];
+ int timeArrayLength = 0;
+ if (hasCachedTimestamp) {
+ if (timestamp < endTime) {
+ hasCachedTimestamp = false;
+ timestampArray[timeArrayLength++] = timestamp;
+ } else {
+ return constructRowRecord();
+ }
+ }
+
+ while (timestampGenerator.hasNext()) {
+ // construct timestamp array
+ timeArrayLength = constructTimeArrayForOneCal(timestampArray, timeArrayLength);
+
+ // cal result using timestamp array
+ for (int i = 0; i < selectedSeries.size(); i++) {
+ functions.get(i).calcAggregationUsingTimestamps(
+ timestampArray, timeArrayLength, allDataReaderList.get(i));
+ }
+
+ timeArrayLength = 0;
+ // judge if it's end
+ if (timestamp >= endTime) {
+ hasCachedTimestamp = true;
+ break;
+ }
+ }
+
+ if (timeArrayLength > 0) {
+ // cal result using timestamp array
+ for (int i = 0; i < selectedSeries.size(); i++) {
+ functions.get(i).calcAggregationUsingTimestamps(
+ timestampArray, timeArrayLength, allDataReaderList.get(i));
+ }
+ }
+ return constructRowRecord();
+ }
+
+ /**
+ * construct an array of timestamps for one batch of a group by partition calculating.
+ *
+ * @param timestampArray timestamp array
+ * @param timeArrayLength the current length of timestamp array
+ * @return time array length
+ */
+ private int constructTimeArrayForOneCal(long[] timestampArray, int timeArrayLength)
+ throws IOException {
+ for (int cnt = 1; cnt < timeStampFetchSize && timestampGenerator.hasNext(); cnt++) {
+ timestamp = timestampGenerator.next();
+ if (timestamp < endTime) {
+ timestampArray[timeArrayLength++] = timestamp;
+ } else {
+ hasCachedTimestamp = true;
+ break;
+ }
+ }
+ return timeArrayLength;
+ }
+
+ private RowRecord constructRowRecord() {
+ RowRecord record = new RowRecord(startTime);
+ functions.forEach(function -> record.addField(getField(function.getResult())));
+ return record;
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java
index 42e81f2..b731fc5 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java
@@ -20,24 +20,36 @@
package org.apache.iotdb.db.query.factory;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.List;
import org.apache.iotdb.db.engine.filenode.TsFileResource;
import org.apache.iotdb.db.engine.modification.Modification;
import org.apache.iotdb.db.engine.querycontext.OverflowInsertFile;
import org.apache.iotdb.db.engine.querycontext.OverflowSeriesDataSource;
+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.FileReaderManager;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.reader.AllDataReader;
+import org.apache.iotdb.db.query.reader.IBatchReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
import org.apache.iotdb.db.query.reader.IReader;
-import org.apache.iotdb.db.query.reader.mem.MemChunkReaderWithFilter;
-import org.apache.iotdb.db.query.reader.mem.MemChunkReaderWithoutFilter;
+import org.apache.iotdb.db.query.reader.mem.MemChunkReader;
+import org.apache.iotdb.db.query.reader.mem.MemChunkReaderByTimestamp;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader;
+import org.apache.iotdb.db.query.reader.merge.PriorityMergeReaderByTimestamp;
import org.apache.iotdb.db.query.reader.sequence.SealedTsFilesReader;
+import org.apache.iotdb.db.query.reader.sequence.SequenceDataReaderByTimestamp;
import org.apache.iotdb.db.query.reader.unsequence.EngineChunkReader;
+import org.apache.iotdb.db.query.reader.unsequence.EngineChunkReaderByTimestamp;
import org.apache.iotdb.db.utils.QueryUtils;
import org.apache.iotdb.tsfile.common.constant.StatisticConstant;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import org.apache.iotdb.tsfile.read.common.Chunk;
+import org.apache.iotdb.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl;
import org.apache.iotdb.tsfile.read.controller.MetadataQuerier;
import org.apache.iotdb.tsfile.read.controller.MetadataQuerierByFileImpl;
@@ -45,6 +57,7 @@ import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
import org.apache.iotdb.tsfile.read.filter.DigestForFilter;
import org.apache.iotdb.tsfile.read.filter.basic.Filter;
import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderByTimestamp;
import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderWithFilter;
import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderWithoutFilter;
import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReader;
@@ -110,15 +123,9 @@ public class SeriesReaderFactory {
// add reader for MemTable
if (overflowSeriesDataSource.hasRawChunk()) {
- if (filter != null) {
- unSeqMergeReader.addReaderWithPriority(
- new MemChunkReaderWithFilter(overflowSeriesDataSource.getReadableMemChunk(), filter),
- priorityValue);
- } else {
- unSeqMergeReader.addReaderWithPriority(
- new MemChunkReaderWithoutFilter(overflowSeriesDataSource.getReadableMemChunk()),
- priorityValue);
- }
+ unSeqMergeReader.addReaderWithPriority(
+ new MemChunkReader(overflowSeriesDataSource.getReadableMemChunk(), filter),
+ priorityValue);
}
// TODO add external sort when needed
@@ -141,22 +148,24 @@ public class SeriesReaderFactory {
singleSeriesExpression,
tsFileResource.getFilePath());
- PriorityMergeReader priorityMergeReader = new PriorityMergeReader();
-
- // Sequence reader
- IReader seriesInTsFileReader = createSealedTsFileReaderForMerge(tsFileResource,
+ // sequence reader
+ IBatchReader seriesInTsFileReader = createSealedTsFileReaderForMerge(tsFileResource,
singleSeriesExpression, context);
- priorityMergeReader.addReaderWithPriority(seriesInTsFileReader, 1);
- // UnSequence merge reader
- IReader unSeqMergeReader = createUnSeqMergeReader(overflowSeriesDataSource,
+ // unSequence merge reader
+ IPointReader unSeqMergeReader = createUnSeqMergeReader(overflowSeriesDataSource,
singleSeriesExpression.getFilter());
- priorityMergeReader.addReaderWithPriority(unSeqMergeReader, 2);
+ if (!seriesInTsFileReader.hasNext()) {
+ // only have unsequence data.
+ return unSeqMergeReader;
+ } else {
+ // merge sequence data with unsequence data.
+ return new AllDataReader(seriesInTsFileReader, unSeqMergeReader);
+ }
- return priorityMergeReader;
}
- private IReader createSealedTsFileReaderForMerge(TsFileResource fileNode,
+ private IBatchReader createSealedTsFileReaderForMerge(TsFileResource fileNode,
SingleSeriesExpression singleSeriesExpression,
QueryContext context)
throws IOException {
@@ -178,6 +187,84 @@ public class SeriesReaderFactory {
return new SealedTsFilesReader(seriesInTsFileReader, context);
}
+ /**
+ * construct ByTimestampReader, include sequential data and unsequential data.
+ *
+ * @param paths selected series path
+ * @param context query context
+ * @return the list of EngineReaderByTimeStamp
+ */
+ public static List<EngineReaderByTimeStamp> getByTimestampReadersOfSelectedPaths(
+ List<Path> paths, QueryContext context) throws IOException, FileNodeManagerException {
+
+ List<EngineReaderByTimeStamp> readersOfSelectedSeries = new ArrayList<>();
+
+ for (Path path : paths) {
+
+ 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);
+
+ readersOfSelectedSeries.add(mergeReaderByTimestamp);
+ }
+
+ return readersOfSelectedSeries;
+ }
+
+ /**
+ * This method is used to create unsequence insert reader by timestamp for IoTDB request, such as
+ * query, aggregation and groupby request.
+ */
+ public PriorityMergeReaderByTimestamp createUnSeqMergeReaderByTimestamp(
+ OverflowSeriesDataSource overflowSeriesDataSource)
+ throws IOException {
+
+ PriorityMergeReaderByTimestamp unSeqMergeReader = new PriorityMergeReaderByTimestamp();
+
+ int priorityValue = 1;
+
+ for (OverflowInsertFile overflowInsertFile : overflowSeriesDataSource
+ .getOverflowInsertFileList()) {
+
+ // store only one opened file stream into manager, to avoid too many opened files
+ TsFileSequenceReader unClosedTsFileReader = FileReaderManager.getInstance()
+ .get(overflowInsertFile.getFilePath(), false);
+
+ ChunkLoaderImpl chunkLoader = new ChunkLoaderImpl(unClosedTsFileReader);
+
+ for (ChunkMetaData chunkMetaData : overflowInsertFile.getChunkMetaDataList()) {
+
+ Chunk chunk = chunkLoader.getChunk(chunkMetaData);
+ ChunkReaderByTimestamp chunkReader = new ChunkReaderByTimestamp(chunk);
+
+ unSeqMergeReader
+ .addReaderWithPriority(new EngineChunkReaderByTimestamp(chunkReader), priorityValue);
+ priorityValue++;
+ }
+ }
+
+ // add reader for MemTable
+ if (overflowSeriesDataSource.hasRawChunk()) {
+ unSeqMergeReader.addReaderWithPriority(
+ new MemChunkReaderByTimestamp(overflowSeriesDataSource.getReadableMemChunk()),
+ priorityValue);
+ }
+
+ // TODO add external sort when needed
+ return unSeqMergeReader;
+ }
+
private static class SeriesReaderFactoryHelper {
private static final SeriesReaderFactory INSTANCE = new SeriesReaderFactory();
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/fill/IFill.java b/iotdb/src/main/java/org/apache/iotdb/db/query/fill/IFill.java
index 7ddce7a..d64b49a 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/fill/IFill.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/fill/IFill.java
@@ -16,18 +16,30 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.query.fill;
-import org.apache.iotdb.db.exception.ProcessorException;
+import java.io.IOException;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.factory.SeriesReaderFactory;
+import org.apache.iotdb.db.query.reader.AllDataReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader;
+import org.apache.iotdb.db.query.reader.sequence.SequenceDataReader;
+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;
+import org.apache.iotdb.tsfile.read.filter.TimeFilter;
+import org.apache.iotdb.tsfile.read.filter.basic.Filter;
public abstract class IFill {
long queryTime;
TSDataType dataType;
+ IPointReader allDataReader;
+
public IFill(TSDataType dataType, long queryTime) {
this.dataType = dataType;
this.queryTime = queryTime;
@@ -38,7 +50,24 @@ public abstract class IFill {
public abstract IFill copy(Path path);
- public abstract BatchData getFillResult() throws ProcessorException;
+ public abstract void constructReaders(QueryDataSource queryDataSource, QueryContext context)
+ throws IOException;
+
+ void constructReaders(QueryDataSource queryDataSource, QueryContext context, long beforeRange)
+ throws IOException {
+ Filter timeFilter = constructFilter(beforeRange);
+ // sequence reader for sealed tsfile, unsealed tsfile, memory
+ SequenceDataReader sequenceReader = new SequenceDataReader(queryDataSource.getSeqDataSource(),
+ timeFilter, context, false);
+
+ // unseq reader for all chunk groups in unSeqFile, memory
+ PriorityMergeReader unSeqMergeReader = SeriesReaderFactory.getInstance()
+ .createUnSeqMergeReader(queryDataSource.getOverflowSeriesDataSource(), timeFilter);
+
+ allDataReader = new AllDataReader(sequenceReader, unSeqMergeReader);
+ }
+
+ public abstract IPointReader getFillResult() throws IOException;
public TSDataType getDataType() {
return this.dataType;
@@ -48,11 +77,47 @@ public abstract class IFill {
this.dataType = dataType;
}
- public long getQueryTime() {
- return this.queryTime;
- }
-
public void setQueryTime(long queryTime) {
this.queryTime = queryTime;
}
+
+ private Filter constructFilter(long beforeRange) {
+ // if the fill time range is not set, beforeRange will be set to -1.
+ if (beforeRange == -1) {
+ return null;
+ }
+ return TimeFilter.gtEq(queryTime - beforeRange);
+ }
+
+ class TimeValuePairPointReader implements IPointReader {
+
+ private boolean isUsed;
+ private TimeValuePair pair;
+
+ public TimeValuePairPointReader(TimeValuePair pair) {
+ this.pair = pair;
+ this.isUsed = (pair == null);
+ }
+
+ @Override
+ public boolean hasNext() {
+ return !isUsed;
+ }
+
+ @Override
+ public TimeValuePair next() {
+ isUsed = true;
+ return pair;
+ }
+
+ @Override
+ public TimeValuePair current() {
+ return pair;
+ }
+
+ @Override
+ public void close() {
+ // no need to close
+ }
+ }
}
\ No newline at end of file
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/fill/LinearFill.java b/iotdb/src/main/java/org/apache/iotdb/db/query/fill/LinearFill.java
index 764f7c7..dc46082 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/fill/LinearFill.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/fill/LinearFill.java
@@ -16,18 +16,23 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.query.fill;
-import org.apache.iotdb.db.exception.ProcessorException;
+import java.io.IOException;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.exception.UnSupportedFillTypeException;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.IPointReader;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.db.utils.TsPrimitiveType;
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;
public class LinearFill extends IFill {
private long beforeRange;
private long afterRange;
- private BatchData result;
public LinearFill(long beforeRange, long afterRange) {
this.beforeRange = beforeRange;
@@ -42,7 +47,6 @@ public class LinearFill extends IFill {
super(dataType, queryTime);
this.beforeRange = beforeRange;
this.afterRange = afterRange;
- result = new BatchData(dataType, true, true);
}
public long getBeforeRange() {
@@ -67,7 +71,81 @@ public class LinearFill extends IFill {
}
@Override
- public BatchData getFillResult() throws ProcessorException {
- return result;
+ public void constructReaders(QueryDataSource queryDataSource, QueryContext context)
+ throws IOException {
+ super.constructReaders(queryDataSource, context, beforeRange);
+ }
+
+ @Override
+ public IPointReader getFillResult() throws IOException {
+ TimeValuePair beforePair = null;
+ TimeValuePair afterPair = null;
+ while (allDataReader.hasNext()) {
+ afterPair = allDataReader.next();
+ if (afterPair.getTimestamp() <= queryTime) {
+ beforePair = afterPair;
+ } else {
+ break;
+ }
+ }
+
+ if (beforePair == null || beforePair.getTimestamp() == queryTime) {
+ return new TimeValuePairPointReader(beforePair);
+ }
+
+ // if afterRange equals -1, this means that there is no time-bound filling.
+ if (afterRange == -1) {
+ return new TimeValuePairPointReader(average(beforePair, afterPair));
+ }
+
+ if (afterPair.getTimestamp() > queryTime + afterRange || afterPair.getTimestamp() < queryTime) {
+ return new TimeValuePairPointReader(new TimeValuePair(queryTime, null));
+ }
+ return new TimeValuePairPointReader(average(beforePair, afterPair));
+ }
+
+ // returns the average of two points
+ private TimeValuePair average(TimeValuePair beforePair, TimeValuePair afterPair) {
+ double totalTimeLength = (double) afterPair.getTimestamp() - beforePair.getTimestamp();
+ double beforeTimeLength = (double) (queryTime - beforePair.getTimestamp());
+ switch (dataType) {
+ case INT32:
+ int startIntValue = beforePair.getValue().getInt();
+ int endIntValue = afterPair.getValue().getInt();
+ int fillIntValue =
+ startIntValue + (int) ((double) (endIntValue - startIntValue) / totalTimeLength
+ * beforeTimeLength);
+ beforePair.setValue(TsPrimitiveType.getByType(TSDataType.INT32, fillIntValue));
+ break;
+ case INT64:
+ long startLongValue = beforePair.getValue().getLong();
+ long endLongValue = afterPair.getValue().getLong();
+ long fillLongValue =
+ startLongValue + (long) ((double) (endLongValue - startLongValue) / totalTimeLength
+ * beforeTimeLength);
+ beforePair.setValue(TsPrimitiveType.getByType(TSDataType.INT64, fillLongValue));
+ break;
+ case FLOAT:
+ float startFloatValue = beforePair.getValue().getFloat();
+ float endFloatValue = afterPair.getValue().getFloat();
+ float fillFloatValue =
+ startFloatValue + (float) ((endFloatValue - startFloatValue) / totalTimeLength
+ * beforeTimeLength);
+ beforePair.setValue(TsPrimitiveType.getByType(TSDataType.FLOAT, fillFloatValue));
+ break;
+ case DOUBLE:
+ double startDoubleValue = beforePair.getValue().getDouble();
+ double endDoubleValue = afterPair.getValue().getDouble();
+ double fillDoubleValue =
+ startDoubleValue + ((endDoubleValue - startDoubleValue) / totalTimeLength
+ * beforeTimeLength);
+ beforePair.setValue(TsPrimitiveType.getByType(TSDataType.DOUBLE, fillDoubleValue));
+ break;
+ default:
+ throw new UnSupportedFillTypeException("Unsupported linear fill data type : " + dataType);
+
+ }
+ beforePair.setTimestamp(queryTime);
+ return beforePair;
}
}
\ No newline at end of file
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/fill/PreviousFill.java b/iotdb/src/main/java/org/apache/iotdb/db/query/fill/PreviousFill.java
index 1c130b9..b75fb4f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/fill/PreviousFill.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/fill/PreviousFill.java
@@ -18,21 +18,21 @@
*/
package org.apache.iotdb.db.query.fill;
-import org.apache.iotdb.db.exception.ProcessorException;
+import java.io.IOException;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.query.context.QueryContext;
+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;
import org.apache.iotdb.tsfile.read.common.Path;
public class PreviousFill extends IFill {
private long beforeRange;
- private BatchData result;
-
public PreviousFill(TSDataType dataType, long queryTime, long beforeRange) {
super(dataType, queryTime);
this.beforeRange = beforeRange;
- result = new BatchData(dataType, true, true);
}
public PreviousFill(long beforeRange) {
@@ -44,12 +44,34 @@ public class PreviousFill extends IFill {
return new PreviousFill(dataType, queryTime, beforeRange);
}
+ @Override
+ public void constructReaders(QueryDataSource queryDataSource, QueryContext context)
+ throws IOException {
+ super.constructReaders(queryDataSource, context, beforeRange);
+ }
+
public long getBeforeRange() {
return beforeRange;
}
@Override
- public BatchData getFillResult() throws ProcessorException {
- return result;
+ public IPointReader getFillResult() throws IOException {
+ TimeValuePair beforePair = null;
+ TimeValuePair cachedPair = null;
+ while (allDataReader.hasNext()) {
+ cachedPair = allDataReader.next();
+ if (cachedPair.getTimestamp() <= queryTime) {
+ beforePair = cachedPair;
+ } else {
+ break;
+ }
+ }
+
+ if (beforePair != null) {
+ beforePair.setTimestamp(queryTime);
+ } else {
+ beforePair = new TimeValuePair(queryTime, null);
+ }
+ return new TimeValuePairPointReader(beforePair);
}
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/AllDataReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/AllDataReader.java
new file mode 100644
index 0000000..6eaa4bb
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/AllDataReader.java
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.reader;
+
+import java.io.IOException;
+import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.db.utils.TimeValuePairUtils;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public class AllDataReader implements IPointReader {
+
+ private IBatchReader batchReader;
+ private IPointReader pointReader;
+
+ private boolean hasCachedBatchData;
+ private BatchData batchData;
+
+ /**
+ * merge sequence reader, unsequence reader.
+ */
+ public AllDataReader(IBatchReader batchReader, IPointReader pointReader) {
+ this.batchReader = batchReader;
+ this.pointReader = pointReader;
+
+ this.hasCachedBatchData = false;
+ }
+
+ @Override
+ public boolean hasNext() throws IOException {
+ if (hasNextInBatchDataOrBatchReader()) {
+ return true;
+ }
+ // has value in pointReader
+ return pointReader != null && pointReader.hasNext();
+ }
+
+ @Override
+ public TimeValuePair next() throws IOException {
+
+ // has next in both batch reader and point reader
+ if (hasNextInBothReader()) {
+ long timeInPointReader = pointReader.current().getTimestamp();
+ long timeInBatchData = batchData.currentTime();
+ if (timeInPointReader > timeInBatchData) {
+ TimeValuePair timeValuePair = TimeValuePairUtils.getCurrentTimeValuePair(batchData);
+ batchData.next();
+ return timeValuePair;
+ } else if (timeInPointReader == timeInBatchData) {
+ batchData.next();
+ return pointReader.next();
+ } else {
+ return pointReader.next();
+ }
+ }
+
+ // only has next in batch reader
+ if (hasNextInBatchDataOrBatchReader()) {
+ TimeValuePair timeValuePair = TimeValuePairUtils.getCurrentTimeValuePair(batchData);
+ batchData.next();
+ return timeValuePair;
+ }
+
+ // only has next in point reader
+ if (pointReader != null && pointReader.hasNext()) {
+ return pointReader.next();
+ }
+ return null;
+ }
+
+ /**
+ * judge if has next in both batch record and pointReader.
+ */
+ private boolean hasNextInBothReader() throws IOException {
+ if (!hasNextInBatchDataOrBatchReader()) {
+ return false;
+ }
+ return pointReader != null && pointReader.hasNext();
+ }
+
+ /**
+ * judge if has next in batch record, either in batch data or in batch reader.
+ */
+ private boolean hasNextInBatchDataOrBatchReader() throws IOException {
+ // has value in batchData
+ if (hasCachedBatchData && batchData.hasNext()) {
+ return true;
+ } else {
+ hasCachedBatchData = false;
+ }
+
+ // has value in batchReader
+ while (batchReader.hasNext()) {
+ batchData = batchReader.nextBatch();
+ if (batchData.hasNext()) {
+ hasCachedBatchData = true;
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public TimeValuePair current() throws IOException {
+ throw new IOException("current() in AllDataReader is an empty method.");
+ }
+
+ @Override
+ public void close() throws IOException {
+ batchReader.close();
+ pointReader.close();
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IAggregateReader.java
similarity index 66%
copy from iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java
copy to iotdb/src/main/java/org/apache/iotdb/db/query/reader/IAggregateReader.java
index 8ddf0ad..edc985f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IAggregateReader.java
@@ -16,17 +16,19 @@
* specific language governing permissions and limitations
* under the License.
*/
-package org.apache.iotdb.db.query.reader.merge;
+
+package org.apache.iotdb.db.query.reader;
import java.io.IOException;
-import org.apache.iotdb.db.query.reader.IReader;
-import org.apache.iotdb.db.utils.TsPrimitiveType;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
-public interface EngineReaderByTimeStamp extends IReader {
+public interface IAggregateReader extends IBatchReader {
/**
- * Given a timestamp, the reader is supposed to return the corresponding value in the timestamp.
- * If no value in this timestamp, null will be returned.
+ * Returns meta-information of batch data. If batch data comes from memory, return null. If batch
+ * data comes from page data, return pageHeader.
*/
- TsPrimitiveType getValueInTimestamp(long timestamp) throws IOException;
+ PageHeader nextPageHeader() throws IOException;
+
+ void skipPageData() throws IOException;
}
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IBatchReader.java
similarity index 74%
copy from iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
copy to iotdb/src/main/java/org/apache/iotdb/db/query/reader/IBatchReader.java
index 3ac1540..6ca07c2 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IBatchReader.java
@@ -1,29 +1,32 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.query.control;
-
-import org.junit.Test;
-
-public class QueryTokenManagerTest {
-
- @Test
- public void test() {
- //TODO
- }
-}
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.reader;
+
+import java.io.IOException;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+
+public interface IBatchReader {
+
+ boolean hasNext() throws IOException;
+
+ BatchData nextBatch() throws IOException;
+
+ void close() throws IOException;
+}
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IPointReader.java
similarity index 79%
copy from iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
copy to iotdb/src/main/java/org/apache/iotdb/db/query/reader/IPointReader.java
index 3ac1540..2970e1c 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IPointReader.java
@@ -1,29 +1,28 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.query.control;
-
-import org.junit.Test;
-
-public class QueryTokenManagerTest {
-
- @Test
- public void test() {
- //TODO
- }
-}
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.reader;
+
+import java.io.IOException;
+import org.apache.iotdb.db.utils.TimeValuePair;
+
+public interface IPointReader extends IReader {
+
+ TimeValuePair current() throws IOException;
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IReader.java
index 6a574d6..2982a97 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/IReader.java
@@ -16,15 +16,15 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.query.reader;
import java.io.IOException;
import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.tsfile.read.common.BatchData;
/**
- * Vital read interface. Batch method is used to increase query speed. Getting a batch of data
- * a time is faster than getting one point a time.
+ * Vital read interface. Batch method is used to increase query speed. Getting a batch of data a
+ * time is faster than getting one point a time.
*/
public interface IReader {
@@ -32,13 +32,5 @@ public interface IReader {
TimeValuePair next() throws IOException;
- void skipCurrentTimeValuePair() throws IOException;
-
void close() throws IOException;
-
- boolean hasNextBatch();
-
- BatchData nextBatch();
-
- BatchData currentBatch();
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithFilter.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReader.java
similarity index 56%
rename from iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithFilter.java
rename to iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReader.java
index 1f21a84..8c3e9cf 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithFilter.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReader.java
@@ -16,26 +16,36 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.query.reader.mem;
import java.util.Iterator;
-import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
-import org.apache.iotdb.db.query.reader.IReader;
+import org.apache.iotdb.db.query.reader.IAggregateReader;
+import org.apache.iotdb.db.query.reader.IBatchReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.common.BatchData;
import org.apache.iotdb.tsfile.read.filter.basic.Filter;
-public class MemChunkReaderWithFilter implements IReader {
+public class MemChunkReader implements IPointReader, IBatchReader, IAggregateReader {
private Iterator<TimeValuePair> timeValuePairIterator;
private Filter filter;
private boolean hasCachedTimeValuePair;
private TimeValuePair cachedTimeValuePair;
- public MemChunkReaderWithFilter(ReadOnlyMemChunk readableChunk, Filter filter) {
+ private TSDataType dataType;
+
+ /**
+ * memory data reader.
+ */
+ public MemChunkReader(ReadOnlyMemChunk readableChunk, Filter filter) {
timeValuePairIterator = readableChunk.getIterator();
this.filter = filter;
+ this.dataType = readableChunk.getDataType();
}
@Override
@@ -45,7 +55,8 @@ public class MemChunkReaderWithFilter implements IReader {
}
while (timeValuePairIterator.hasNext()) {
TimeValuePair timeValuePair = timeValuePairIterator.next();
- if (filter.satisfy(timeValuePair.getTimestamp(), timeValuePair.getValue().getValue())) {
+ if (filter == null || filter
+ .satisfy(timeValuePair.getTimestamp(), timeValuePair.getValue().getValue())) {
hasCachedTimeValuePair = true;
cachedTimeValuePair = timeValuePair;
break;
@@ -65,27 +76,45 @@ public class MemChunkReaderWithFilter implements IReader {
}
@Override
- public void skipCurrentTimeValuePair() {
- next();
+ public TimeValuePair current() {
+ if (!hasCachedTimeValuePair) {
+ cachedTimeValuePair = timeValuePairIterator.next();
+ hasCachedTimeValuePair = true;
+ }
+ return cachedTimeValuePair;
}
@Override
- public void close() {
- // Do nothing because mem chunk reader will not open files
+ public BatchData nextBatch() {
+ BatchData batchData = new BatchData(dataType, true);
+ if (hasCachedTimeValuePair) {
+ hasCachedTimeValuePair = false;
+ batchData.putTime(cachedTimeValuePair.getTimestamp());
+ batchData.putAnObject(cachedTimeValuePair.getValue().getValue());
+ }
+ while (timeValuePairIterator.hasNext()) {
+ TimeValuePair timeValuePair = timeValuePairIterator.next();
+ if (filter == null || filter
+ .satisfy(timeValuePair.getTimestamp(), timeValuePair.getValue().getValue())) {
+ batchData.putTime(timeValuePair.getTimestamp());
+ batchData.putAnObject(timeValuePair.getValue().getValue());
+ }
+ }
+ return batchData;
}
@Override
- public boolean hasNextBatch() {
- return false;
+ public void close() {
+ // Do nothing because mem chunk reader will not open files
}
@Override
- public BatchData nextBatch() {
+ public PageHeader nextPageHeader() {
return null;
}
@Override
- public BatchData currentBatch() {
- return null;
+ public void skipPageData() {
+ nextBatch();
}
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java
index dbd6beb..9827f5e 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java
@@ -16,15 +16,13 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.query.reader.mem;
-import java.io.IOException;
import java.util.Iterator;
import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.db.utils.TsPrimitiveType;
-import org.apache.iotdb.tsfile.read.common.BatchData;
public class MemChunkReaderByTimestamp implements EngineReaderByTimeStamp {
@@ -44,8 +42,7 @@ public class MemChunkReaderByTimestamp implements EngineReaderByTimeStamp {
return timeValuePairIterator.hasNext();
}
- @Override
- public TimeValuePair next() throws IOException {
+ private TimeValuePair next() {
if (hasCachedTimeValuePair) {
hasCachedTimeValuePair = false;
return cachedTimeValuePair;
@@ -54,25 +51,15 @@ public class MemChunkReaderByTimestamp implements EngineReaderByTimeStamp {
}
}
- @Override
- public void skipCurrentTimeValuePair() throws IOException {
- next();
- }
-
- @Override
- public void close() {
- // Do nothing because mem chunk reader will not open files
- }
-
// TODO consider change timeValuePairIterator to List structure, and use binary search instead of
// sequential search
@Override
- public TsPrimitiveType getValueInTimestamp(long timestamp) throws IOException {
+ public Object getValueInTimestamp(long timestamp) {
while (hasNext()) {
TimeValuePair timeValuePair = next();
long time = timeValuePair.getTimestamp();
if (time == timestamp) {
- return timeValuePair.getValue();
+ return timeValuePair.getValue().getValue();
} else if (time > timestamp) {
hasCachedTimeValuePair = true;
cachedTimeValuePair = timeValuePair;
@@ -81,20 +68,4 @@ public class MemChunkReaderByTimestamp implements EngineReaderByTimeStamp {
}
return null;
}
-
- @Override
- public boolean hasNextBatch() {
- return false;
- }
-
- @Override
- public BatchData nextBatch() {
- return null;
- }
-
- @Override
- public BatchData currentBatch() {
- return null;
- }
-
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithoutFilter.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithoutFilter.java
deleted file mode 100644
index bae7496..0000000
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithoutFilter.java
+++ /dev/null
@@ -1,70 +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.db.query.reader.mem;
-
-import java.util.Iterator;
-import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
-import org.apache.iotdb.db.query.reader.IReader;
-import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.tsfile.read.common.BatchData;
-
-// TODO merge MemChunkReaderWithoutFilter and MemChunkReaderWithFilter to one class
-public class MemChunkReaderWithoutFilter implements IReader {
-
- private Iterator<TimeValuePair> timeValuePairIterator;
-
- public MemChunkReaderWithoutFilter(TimeValuePairSorter readableChunk) {
- timeValuePairIterator = readableChunk.getIterator();
- }
-
- @Override
- public boolean hasNext() {
- return timeValuePairIterator.hasNext();
- }
-
- @Override
- public TimeValuePair next() {
- return timeValuePairIterator.next();
- }
-
- @Override
- public void skipCurrentTimeValuePair() {
- next();
- }
-
- @Override
- public void close() {
- // Do nothing because mem chunk reader will not open files
- }
-
- @Override
- public boolean hasNextBatch() {
- return false;
- }
-
- @Override
- public BatchData nextBatch() {
- return null;
- }
-
- @Override
- public BatchData currentBatch() {
- return null;
- }
-}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java
index 8ddf0ad..65a8849 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java
@@ -19,14 +19,14 @@
package org.apache.iotdb.db.query.reader.merge;
import java.io.IOException;
-import org.apache.iotdb.db.query.reader.IReader;
-import org.apache.iotdb.db.utils.TsPrimitiveType;
-public interface EngineReaderByTimeStamp extends IReader {
+public interface EngineReaderByTimeStamp {
/**
* Given a timestamp, the reader is supposed to return the corresponding value in the timestamp.
* If no value in this timestamp, null will be returned.
*/
- TsPrimitiveType getValueInTimestamp(long timestamp) throws IOException;
+ Object getValueInTimestamp(long timestamp) throws IOException;
+
+ boolean hasNext() throws IOException;
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReader.java
index e100e50..e76d142 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReader.java
@@ -16,36 +16,34 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.query.reader.merge;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.PriorityQueue;
-import org.apache.iotdb.db.query.reader.IReader;
+import org.apache.iotdb.db.query.reader.IPointReader;
import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.tsfile.read.common.BatchData;
/**
* <p>
- * Usage:
- * (1) merge multiple chunk group readers in the unsequence file
- * (2)merge sequence reader, unsequence reader and mem reader
+ * Usage: (1) merge multiple chunk group readers in the unsequence file.
* </p>
*/
-public class PriorityMergeReader implements IReader {
+public class PriorityMergeReader implements IPointReader {
public static final int LOW_PRIORITY = 1;
public static final int HIGH_PRIORITY = 2;
- private List<IReader> readerList = new ArrayList<>();
+ private List<IPointReader> readerList = new ArrayList<>();
private List<Integer> priorityList = new ArrayList<>();
private PriorityQueue<Element> heap = new PriorityQueue<>();
/**
- * The bigger the priority value is, the higher the priority of this reader is
+ * The bigger the priority value is, the higher the priority of this reader is.
*/
- public void addReaderWithPriority(IReader reader, int priority) throws IOException {
+ public void addReaderWithPriority(IPointReader reader, int priority) throws IOException {
if (reader.hasNext()) {
heap.add(new Element(readerList.size(), reader.next(), priority));
}
@@ -65,11 +63,16 @@ public class PriorityMergeReader implements IReader {
return top.timeValuePair;
}
+ @Override
+ public TimeValuePair current() {
+ return heap.peek().timeValuePair;
+ }
+
private void updateHeap(Element top) throws IOException {
while (!heap.isEmpty() && heap.peek().timeValuePair.getTimestamp() == top.timeValuePair
.getTimestamp()) {
Element e = heap.poll();
- IReader reader = readerList.get(e.index);
+ IPointReader reader = readerList.get(e.index);
if (reader.hasNext()) {
heap.add(new Element(e.index, reader.next(), priorityList.get(e.index)));
}
@@ -77,35 +80,12 @@ public class PriorityMergeReader implements IReader {
}
@Override
- public void skipCurrentTimeValuePair() throws IOException {
- if (hasNext()) {
- next();
- }
- }
-
- @Override
public void close() throws IOException {
- for (IReader reader : readerList) {
+ for (IPointReader reader : readerList) {
reader.close();
}
}
- @Override
- public boolean hasNextBatch() {
- // TODO
- return false;
- }
-
- @Override
- public BatchData nextBatch() {
- return null;
- }
-
- @Override
- public BatchData currentBatch() {
- return null;
- }
-
protected class Element implements Comparable<Element> {
int index;
@@ -133,11 +113,11 @@ public class PriorityMergeReader implements IReader {
}
@Override
- public boolean equals(Object o){
- if (o instanceof Element){
+ public boolean equals(Object o) {
+ if (o instanceof Element) {
Element element = (Element) o;
if (this.timeValuePair.getTimestamp() == element.timeValuePair.getTimestamp()
- && this.priority.equals(element.priority)){
+ && this.priority.equals(element.priority)) {
return true;
}
}
@@ -145,7 +125,7 @@ public class PriorityMergeReader implements IReader {
}
@Override
- public int hashCode(){
+ public int hashCode() {
return (int) (timeValuePair.getTimestamp() * 31 + priority.hashCode());
}
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java
index 244291a..0f42e87 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java
@@ -16,44 +16,54 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.query.reader.merge;
import java.io.IOException;
-import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.db.utils.TsPrimitiveType;
+import java.util.ArrayList;
+import java.util.List;
/**
- * TODO the process of PriorityMergeReaderByTimestamp can be optimized.
+ * <p>
+ * Usage: Get value in timestamp by sorting time-value pair in multiple readers with time and
+ * priority. (1) merge multiple chunk group readers in the unsequence file (2)merge sequence reader,
+ * unsequence reader and mem reader
+ * </p>
*/
-public class PriorityMergeReaderByTimestamp extends PriorityMergeReader implements
- EngineReaderByTimeStamp {
+public class PriorityMergeReaderByTimestamp implements EngineReaderByTimeStamp {
+
+ private List<EngineReaderByTimeStamp> readerList = new ArrayList<>();
+ private List<Integer> priorityList = new ArrayList<>();
- private boolean hasCachedTimeValuePair;
- private TimeValuePair cachedTimeValuePair;
+ /**
+ * This function doesn't sort reader by priority. So you have to call this function in order of
+ * reader priority from small to large.
+ */
+ public void addReaderWithPriority(EngineReaderByTimeStamp reader, int priority) {
+ readerList.add(reader);
+ priorityList.add(priority);
+ }
@Override
- public TsPrimitiveType getValueInTimestamp(long timestamp) throws IOException {
-
- if (hasCachedTimeValuePair) {
- if (cachedTimeValuePair.getTimestamp() == timestamp) {
- hasCachedTimeValuePair = false;
- return cachedTimeValuePair.getValue();
- } else if (cachedTimeValuePair.getTimestamp() > timestamp) {
- return null;
+ public Object getValueInTimestamp(long timestamp) throws IOException {
+ Object value = null;
+ for (int i = readerList.size() - 1; i >= 0; i--) {
+ value = readerList.get(i).getValueInTimestamp(timestamp);
+ if (value != null) {
+ return value;
}
}
+ return value;
+ }
- while (hasNext()) {
- cachedTimeValuePair = next();
- if (cachedTimeValuePair.getTimestamp() == timestamp) {
- hasCachedTimeValuePair = false;
- return cachedTimeValuePair.getValue();
- } else if (cachedTimeValuePair.getTimestamp() > timestamp) {
- hasCachedTimeValuePair = true;
- return null;
+ @Override
+ public boolean hasNext() throws IOException {
+ for (int i = readerList.size() - 1; i >= 0; i--) {
+ if (readerList.get(i).hasNext()) {
+ return true;
}
}
-
- return null;
+ return false;
}
+
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReader.java
index d83af0f..075fa08 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReader.java
@@ -21,15 +21,16 @@ package org.apache.iotdb.db.query.reader.sequence;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
import org.apache.iotdb.db.engine.filenode.TsFileResource;
import org.apache.iotdb.db.engine.modification.Modification;
import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.db.query.control.FileReaderManager;
-import org.apache.iotdb.db.query.reader.IReader;
+import org.apache.iotdb.db.query.reader.IAggregateReader;
+import org.apache.iotdb.db.query.reader.IBatchReader;
import org.apache.iotdb.db.utils.QueryUtils;
-import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.db.utils.TimeValuePairUtils;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import org.apache.iotdb.tsfile.read.common.BatchData;
@@ -42,37 +43,50 @@ import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReader;
import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderWithFilter;
import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderWithoutFilter;
-public class SealedTsFilesReader implements IReader {
+public class SealedTsFilesReader implements IBatchReader, IAggregateReader {
private Path seriesPath;
private List<TsFileResource> sealedTsFiles;
- private int usedIntervalFileIndex;
+ private int indexOfNextTsFileResource;
private FileSeriesReader seriesReader;
private Filter filter;
- private BatchData data;
- private boolean hasCachedData;
private QueryContext context;
+ private boolean isReverse;
+ /**
+ * init with seriesPath, sealedTsFiles, filter, context.
+ *
+ * @param seriesPath path
+ * @param sealedTsFiles sealed file list
+ * @param filter null if no filter
+ * @param isReverse true-traverse chunks from behind forward, false-traverse chunks from front to
+ * back.
+ */
public SealedTsFilesReader(Path seriesPath, List<TsFileResource> sealedTsFiles, Filter filter,
- QueryContext context) {
- this(seriesPath, sealedTsFiles, context);
+ QueryContext context, boolean isReverse) {
+ this(seriesPath, sealedTsFiles, context, isReverse);
this.filter = filter;
-
}
/**
* init with seriesPath and sealedTsFiles.
*/
public SealedTsFilesReader(Path seriesPath, List<TsFileResource> sealedTsFiles,
- QueryContext context) {
+ QueryContext context, boolean isReverse) {
+ if (isReverse) {
+ Collections.reverse(sealedTsFiles);
+ }
this.seriesPath = seriesPath;
this.sealedTsFiles = sealedTsFiles;
- this.usedIntervalFileIndex = 0;
+ this.indexOfNextTsFileResource = 0;
this.seriesReader = null;
- this.hasCachedData = false;
this.context = context;
+ this.isReverse = isReverse;
}
+ /**
+ * init with seriesReader and queryContext.
+ */
public SealedTsFilesReader(FileSeriesReader seriesReader, QueryContext context) {
this.seriesReader = seriesReader;
sealedTsFiles = new ArrayList<>();
@@ -81,109 +95,63 @@ public class SealedTsFilesReader implements IReader {
@Override
public boolean hasNext() throws IOException {
- if (hasCachedData) {
+
+ // try to get next batch data from current reader
+ if (seriesReader != null && seriesReader.hasNextBatch()) {
return true;
}
- while (!hasCachedData) {
- boolean flag = false;
-
- // try to get next time value pair from current batch data
- if (data != null && data.hasNext()) {
- hasCachedData = true;
- return true;
- }
-
- // try to get next batch data from current reader
- if (seriesReader != null && seriesReader.hasNextBatch()) {
- data = seriesReader.nextBatch();
- if (data.hasNext()) {
- hasCachedData = true;
- return true;
- } else {
- flag = true;
- }
- }
-
+ // init until reach a satisfied reader
+ while (indexOfNextTsFileResource < sealedTsFiles.size()) {
// try to get next batch data from next reader
- while (!flag && usedIntervalFileIndex < sealedTsFiles.size()) {
- // init until reach a satisfied reader
- if (seriesReader == null || !seriesReader.hasNextBatch()) {
- TsFileResource fileNode = sealedTsFiles.get(usedIntervalFileIndex++);
- if (singleTsFileSatisfied(fileNode)) {
- initSingleTsFileReader(fileNode, context);
- } else {
- flag = true;
- }
- }
- if (!flag && seriesReader.hasNextBatch()) {
- data = seriesReader.nextBatch();
-
- // notice that, data maybe an empty batch data, so an examination must exist
- if (data.hasNext()) {
- hasCachedData = true;
- return true;
- }
- }
+ TsFileResource tsfile = sealedTsFiles.get(indexOfNextTsFileResource++);
+ if (singleTsFileSatisfied(tsfile)) {
+ initSingleTsFileReader(tsfile, context);
+ } else {
+ continue;
}
- if (!flag || data == null || !data.hasNext()) {
- break;
+ if (seriesReader.hasNextBatch()) {
+ return true;
}
}
return false;
}
- @Override
- public TimeValuePair next() {
- TimeValuePair timeValuePair = TimeValuePairUtils.getCurrentTimeValuePair(data);
- data.next();
- hasCachedData = false;
- return timeValuePair;
- }
-
- @Override
- public void skipCurrentTimeValuePair() {
- next();
- }
-
- @Override
- public void close() throws IOException {
- if (seriesReader != null) {
- seriesReader.close();
- }
- }
-
- private boolean singleTsFileSatisfied(TsFileResource fileNode) {
+ private boolean singleTsFileSatisfied(TsFileResource tsfile) {
if (filter == null) {
return true;
}
- long startTime = fileNode.getStartTime(seriesPath.getDevice());
- long endTime = fileNode.getEndTime(seriesPath.getDevice());
+ long startTime = tsfile.getStartTime(seriesPath.getDevice());
+ long endTime = tsfile.getEndTime(seriesPath.getDevice());
return filter.satisfyStartEndTime(startTime, endTime);
}
- private void initSingleTsFileReader(TsFileResource fileNode, QueryContext context)
+ private void initSingleTsFileReader(TsFileResource tsfile, QueryContext context)
throws IOException {
// to avoid too many opened files
TsFileSequenceReader tsFileReader = FileReaderManager.getInstance()
- .get(fileNode.getFilePath(), true);
+ .get(tsfile.getFilePath(), true);
MetadataQuerierByFileImpl metadataQuerier = new MetadataQuerierByFileImpl(tsFileReader);
List<ChunkMetaData> metaDataList = metadataQuerier.getChunkMetaDataList(seriesPath);
- List<Modification> pathModifications = context.getPathModifications(fileNode.getModFile(),
+ List<Modification> pathModifications = context.getPathModifications(tsfile.getModFile(),
seriesPath.getFullPath());
- if (pathModifications.size() > 0) {
+ if (!pathModifications.isEmpty()) {
QueryUtils.modifyChunkMetaData(metaDataList, pathModifications);
}
ChunkLoader chunkLoader = new ChunkLoaderImpl(tsFileReader);
+ if (isReverse) {
+ Collections.reverse(metaDataList);
+ }
+
if (filter == null) {
seriesReader = new FileSeriesReaderWithoutFilter(chunkLoader, metaDataList);
} else {
@@ -192,17 +160,24 @@ public class SealedTsFilesReader implements IReader {
}
@Override
- public boolean hasNextBatch() {
- return false;
+ public BatchData nextBatch() throws IOException {
+ return seriesReader.nextBatch();
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (seriesReader != null) {
+ seriesReader.close();
+ }
}
@Override
- public BatchData nextBatch() {
- return null;
+ public PageHeader nextPageHeader() throws IOException {
+ return seriesReader.nextPageHeader();
}
@Override
- public BatchData currentBatch() {
- return null;
+ public void skipPageData() {
+ seriesReader.skipPageData();
}
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java
new file mode 100644
index 0000000..e58583c
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java
@@ -0,0 +1,136 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.reader.sequence;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.iotdb.db.engine.filenode.TsFileResource;
+import org.apache.iotdb.db.engine.modification.Modification;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.FileReaderManager;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.db.utils.QueryUtils;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.controller.ChunkLoader;
+import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl;
+import org.apache.iotdb.tsfile.read.controller.MetadataQuerierByFileImpl;
+import org.apache.iotdb.tsfile.read.reader.series.SeriesReaderByTimestamp;
+
+public class SealedTsFilesReaderByTimestamp implements EngineReaderByTimeStamp {
+
+ private Path seriesPath;
+ private List<TsFileResource> sealedTsFiles;
+ private int nextIntervalFileIndex;
+ private SeriesReaderByTimestamp seriesReader;
+ private QueryContext context;
+
+ /**
+ * init with seriesPath and sealedTsFiles.
+ */
+ public SealedTsFilesReaderByTimestamp(Path seriesPath, List<TsFileResource> sealedTsFiles,
+ QueryContext context) {
+ this.seriesPath = seriesPath;
+ this.sealedTsFiles = sealedTsFiles;
+ this.nextIntervalFileIndex = 0;
+ this.seriesReader = null;
+ this.context = context;
+ }
+
+ @Override
+ public Object getValueInTimestamp(long timestamp) throws IOException {
+ Object value = null;
+ if (seriesReader != null) {
+ value = seriesReader.getValueInTimestamp(timestamp);
+ if (value != null || seriesReader.hasNext()) {
+ return value;
+ }
+ }
+ constructReader(timestamp);
+ if (seriesReader != null) {
+ value = seriesReader.getValueInTimestamp(timestamp);
+ if (value != null || seriesReader.hasNext()) {
+ return value;
+ }
+ }
+
+ return value;
+ }
+
+ @Override
+ public boolean hasNext() throws IOException {
+ if (seriesReader != null && seriesReader.hasNext()) {
+ return true;
+ }
+ while (nextIntervalFileIndex < sealedTsFiles.size()) {
+ initSingleTsFileReader(sealedTsFiles.get(nextIntervalFileIndex), context);
+ nextIntervalFileIndex++;
+ if (seriesReader.hasNext()) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ /**
+ * construct reader with the file that might overlap this timestamp.
+ */
+ private void constructReader(long timestamp) throws IOException {
+ while (nextIntervalFileIndex < sealedTsFiles.size()) {
+ if (singleTsFileSatisfied(sealedTsFiles.get(nextIntervalFileIndex), timestamp)) {
+ initSingleTsFileReader(sealedTsFiles.get(nextIntervalFileIndex), context);
+ nextIntervalFileIndex++;
+ return;
+ }
+ nextIntervalFileIndex++;
+ }
+ }
+
+ /**
+ * Judge whether the file should be skipped.
+ */
+ private boolean singleTsFileSatisfied(TsFileResource fileNode, long timestamp) {
+ long endTime = fileNode.getEndTime(seriesPath.getDevice());
+ return endTime >= timestamp;
+ }
+
+ private void initSingleTsFileReader(TsFileResource fileNode, QueryContext context)
+ throws IOException {
+
+ // to avoid too many opened files
+ TsFileSequenceReader tsFileReader = FileReaderManager.getInstance()
+ .get(fileNode.getFilePath(), true);
+
+ MetadataQuerierByFileImpl metadataQuerier = new MetadataQuerierByFileImpl(tsFileReader);
+ List<ChunkMetaData> metaDataList = metadataQuerier.getChunkMetaDataList(seriesPath);
+
+ List<Modification> pathModifications = context.getPathModifications(fileNode.getModFile(),
+ seriesPath.getFullPath());
+ if (!pathModifications.isEmpty()) {
+ QueryUtils.modifyChunkMetaData(metaDataList, pathModifications);
+ }
+ ChunkLoader chunkLoader = new ChunkLoaderImpl(tsFileReader);
+
+ seriesReader = new SeriesReaderByTimestamp(chunkLoader, metaDataList);
+
+ }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReader.java
index e4ca98f..b691d7f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReader.java
@@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.query.reader.sequence;
import java.io.IOException;
@@ -23,10 +24,10 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.iotdb.db.engine.querycontext.GlobalSortedSeriesDataSource;
import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.query.reader.IReader;
-import org.apache.iotdb.db.query.reader.mem.MemChunkReaderWithFilter;
-import org.apache.iotdb.db.query.reader.mem.MemChunkReaderWithoutFilter;
-import org.apache.iotdb.db.utils.TimeValuePair;
+import org.apache.iotdb.db.query.reader.IAggregateReader;
+import org.apache.iotdb.db.query.reader.IBatchReader;
+import org.apache.iotdb.db.query.reader.mem.MemChunkReader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
import org.apache.iotdb.tsfile.read.common.BatchData;
import org.apache.iotdb.tsfile.read.filter.basic.Filter;
@@ -36,49 +37,77 @@ import org.apache.iotdb.tsfile.read.filter.basic.Filter;
* in MemTable.
* </p>
*/
-public class SequenceDataReader implements IReader {
+public class SequenceDataReader implements IBatchReader, IAggregateReader {
- private List<IReader> seriesReaders;
+ private List<IAggregateReader> seriesReaders;
private boolean curReaderInitialized;
private int nextSeriesReaderIndex;
- private IReader currentSeriesReader;
+ private IAggregateReader currentSeriesReader;
/**
- * init with globalSortedSeriesDataSource and filter.
+ * init with globalSortedSeriesDataSource, filter, context and isReverse.
+ *
+ * @param sources data source
+ * @param filter null if no filter
+ * @param context query context
+ * @param isReverse true-traverse chunks from behind forward, false-traverse chunks from front to
+ * back.
*/
public SequenceDataReader(GlobalSortedSeriesDataSource sources, Filter filter,
- QueryContext context)
- throws IOException {
+ QueryContext context, boolean isReverse) throws IOException {
seriesReaders = new ArrayList<>();
curReaderInitialized = false;
nextSeriesReaderIndex = 0;
- // add reader for sealed TsFiles
- if (sources.hasSealedTsFiles()) {
- seriesReaders.add(
- new SealedTsFilesReader(sources.getSeriesPath(), sources.getSealedTsFiles(), filter,
- context));
- }
+ if (isReverse) {
+ // add data in memTable
+ if (sources.hasRawSeriesChunk()) {
+ seriesReaders.add(new MemChunkReader(sources.getReadableChunk(), filter));
+ }
- // add reader for unSealed TsFile
- if (sources.hasUnsealedTsFile()) {
- seriesReaders.add(new UnSealedTsFileReader(sources.getUnsealedTsFile(), filter));
- }
+ // add reader for unSealed TsFile
+ if (sources.hasUnsealedTsFile()) {
+ seriesReaders.add(new UnSealedTsFileReader(sources.getUnsealedTsFile(), filter, isReverse));
+ }
+
+ // add reader for sealed TsFiles
+ if (sources.hasSealedTsFiles()) {
+ seriesReaders.add(
+ new SealedTsFilesReader(sources.getSeriesPath(), sources.getSealedTsFiles(), filter,
+ context, isReverse));
+ }
+ } else {
+ // add reader for sealed TsFiles
+ if (sources.hasSealedTsFiles()) {
+ seriesReaders.add(
+ new SealedTsFilesReader(sources.getSeriesPath(), sources.getSealedTsFiles(), filter,
+ context, isReverse));
+ }
- // add data in memTable
- if (sources.hasRawSeriesChunk()) {
- if (filter == null) {
- seriesReaders.add(new MemChunkReaderWithoutFilter(sources.getReadableChunk()));
- } else {
- seriesReaders.add(new MemChunkReaderWithFilter(sources.getReadableChunk(), filter));
+ // add reader for unSealed TsFile
+ if (sources.hasUnsealedTsFile()) {
+ seriesReaders.add(new UnSealedTsFileReader(sources.getUnsealedTsFile(), filter, isReverse));
+ }
+
+ // add data in memTable
+ if (sources.hasRawSeriesChunk()) {
+ seriesReaders.add(new MemChunkReader(sources.getReadableChunk(), filter));
}
}
+ }
+ /**
+ * init with globalSortedSeriesDataSource, filter, context and isReverse.
+ */
+ public SequenceDataReader(GlobalSortedSeriesDataSource sources, Filter filter,
+ QueryContext context) throws IOException {
+ this(sources, filter, context, false);
}
@Override
public boolean hasNext() throws IOException {
+
if (curReaderInitialized && currentSeriesReader.hasNext()) {
return true;
} else {
@@ -96,35 +125,24 @@ public class SequenceDataReader implements IReader {
}
@Override
- public TimeValuePair next() throws IOException {
- return currentSeriesReader.next();
- }
-
- @Override
- public void skipCurrentTimeValuePair() throws IOException {
- next();
- }
-
- @Override
public void close() throws IOException {
- for (IReader seriesReader : seriesReaders) {
+ for (IBatchReader seriesReader : seriesReaders) {
seriesReader.close();
}
}
@Override
- public boolean hasNextBatch() {
- return false;
+ public BatchData nextBatch() throws IOException {
+ return currentSeriesReader.nextBatch();
}
@Override
- public BatchData nextBatch() {
- return null;
+ public PageHeader nextPageHeader() throws IOException {
+ return currentSeriesReader.nextPageHeader();
}
@Override
- public BatchData currentBatch() {
- return null;
+ public void skipPageData() throws IOException {
+ currentSeriesReader.skipPageData();
}
-
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java
new file mode 100644
index 0000000..a5fdc44
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.reader.sequence;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iotdb.db.engine.querycontext.GlobalSortedSeriesDataSource;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.mem.MemChunkReaderByTimestamp;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+
+public class SequenceDataReaderByTimestamp implements EngineReaderByTimeStamp {
+
+ private List<EngineReaderByTimeStamp> seriesReaders;
+ private int nextSeriesReaderIndex;
+ private EngineReaderByTimeStamp currentSeriesReader;
+
+ /**
+ * init with globalSortedSeriesDataSource and filter.
+ */
+ public SequenceDataReaderByTimestamp(GlobalSortedSeriesDataSource sources, QueryContext context)
+ throws IOException {
+ seriesReaders = new ArrayList<>();
+
+ nextSeriesReaderIndex = 0;
+
+ // add reader for sealed TsFiles
+ if (sources.hasSealedTsFiles()) {
+ seriesReaders.add(
+ new SealedTsFilesReaderByTimestamp(sources.getSeriesPath(), sources.getSealedTsFiles(),
+ context));
+ }
+
+ // add reader for unSealed TsFile
+ if (sources.hasUnsealedTsFile()) {
+ seriesReaders.add(new UnSealedTsFilesReaderByTimestamp(sources.getUnsealedTsFile()));
+ }
+
+ // add data in memTable
+ if (sources.hasRawSeriesChunk()) {
+ seriesReaders.add(new MemChunkReaderByTimestamp(sources.getReadableChunk()));
+ }
+
+ }
+
+ /**
+ * This method is used only in unit test.
+ */
+ public SequenceDataReaderByTimestamp(List<EngineReaderByTimeStamp> seriesReaders) {
+ this.seriesReaders = seriesReaders;
+ nextSeriesReaderIndex = 0;
+ }
+
+
+ @Override
+ public Object getValueInTimestamp(long timestamp) throws IOException {
+ Object value = null;
+ if (currentSeriesReader != null) {
+ value = currentSeriesReader.getValueInTimestamp(timestamp);
+ if (value != null || currentSeriesReader.hasNext()) {
+ return value;
+ }
+ }
+
+ while (nextSeriesReaderIndex < seriesReaders.size()) {
+ currentSeriesReader = seriesReaders.get(nextSeriesReaderIndex++);
+ if (currentSeriesReader != null) {
+ value = currentSeriesReader.getValueInTimestamp(timestamp);
+ if (value != null || currentSeriesReader.hasNext()) {
+ return value;
+ }
+ }
+ }
+ return value;
+ }
+
+ @Override
+ public boolean hasNext() throws IOException {
+ if (currentSeriesReader != null && currentSeriesReader.hasNext()) {
+ return true;
+ }
+ while (nextSeriesReaderIndex < seriesReaders.size()) {
+ currentSeriesReader = seriesReaders.get(nextSeriesReaderIndex++);
+ if (currentSeriesReader != null && currentSeriesReader.hasNext()) {
+ return true;
+ }
+ }
+ return false;
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReader.java
index 3c02930..d4e75e6 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReader.java
@@ -16,14 +16,18 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.query.reader.sequence;
import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
import org.apache.iotdb.db.engine.querycontext.UnsealedTsFile;
import org.apache.iotdb.db.query.control.FileReaderManager;
-import org.apache.iotdb.db.query.reader.IReader;
-import org.apache.iotdb.db.utils.TimeValuePair;
-import org.apache.iotdb.db.utils.TimeValuePairUtils;
+import org.apache.iotdb.db.query.reader.IAggregateReader;
+import org.apache.iotdb.db.query.reader.IBatchReader;
+import org.apache.iotdb.tsfile.file.header.PageHeader;
+import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import org.apache.iotdb.tsfile.read.common.BatchData;
import org.apache.iotdb.tsfile.read.common.Path;
@@ -34,58 +38,43 @@ import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReader;
import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderWithFilter;
import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderWithoutFilter;
-public class UnSealedTsFileReader implements IReader {
+public class UnSealedTsFileReader implements IBatchReader, IAggregateReader {
protected Path seriesPath;
private FileSeriesReader unSealedReader;
- private BatchData data;
/**
* Construct funtion for UnSealedTsFileReader.
*
* @param unsealedTsFile -param to initial
* @param filter -filter
+ * @param isReverse true-traverse chunks from behind forward; false-traverse chunks from front to
+ * back;
*/
- public UnSealedTsFileReader(UnsealedTsFile unsealedTsFile, Filter filter) throws IOException {
+ public UnSealedTsFileReader(UnsealedTsFile unsealedTsFile, Filter filter, boolean isReverse)
+ throws IOException {
TsFileSequenceReader unClosedTsFileReader = FileReaderManager.getInstance()
.get(unsealedTsFile.getFilePath(),
false);
ChunkLoader chunkLoader = new ChunkLoaderImpl(unClosedTsFileReader);
+ List<ChunkMetaData> metaDataList = unsealedTsFile.getChunkMetaDataList();
+ //reverse chunk metadata list if traversing chunks from behind forward
+ if (isReverse && metaDataList != null && !metaDataList.isEmpty()) {
+ Collections.reverse(metaDataList);
+ }
+
if (filter == null) {
- unSealedReader = new FileSeriesReaderWithoutFilter(chunkLoader,
- unsealedTsFile.getChunkMetaDataList());
+ unSealedReader = new FileSeriesReaderWithoutFilter(chunkLoader, metaDataList);
} else {
- unSealedReader = new FileSeriesReaderWithFilter(chunkLoader,
- unsealedTsFile.getChunkMetaDataList(),
- filter);
+ unSealedReader = new FileSeriesReaderWithFilter(chunkLoader, metaDataList, filter);
}
-
}
@Override
public boolean hasNext() throws IOException {
- if (data == null || !data.hasNext()) {
- if (!unSealedReader.hasNextBatch()) {
- return false;
- }
- data = unSealedReader.nextBatch();
- }
-
- return data.hasNext();
- }
-
- @Override
- public TimeValuePair next() {
- TimeValuePair timeValuePair = TimeValuePairUtils.getCurrentTimeValuePair(data);
- data.next();
- return timeValuePair;
- }
-
- @Override
- public void skipCurrentTimeValuePair() {
- data.next();
+ return unSealedReader.hasNextBatch();
}
@Override
@@ -96,17 +85,17 @@ public class UnSealedTsFileReader implements IReader {
}
@Override
- public boolean hasNextBatch() {
- return false;
+ public BatchData nextBatch() throws IOException {
+ return unSealedReader.nextBatch();
}
@Override
- public BatchData nextBatch() {
- return null;
+ public PageHeader nextPageHeader() throws IOException {
+ return unSealedReader.nextPageHeader();
}
@Override
- public BatchData currentBatch() {
- return null;
+ public void skipPageData() {
+ unSealedReader.skipPageData();
}
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java
new file mode 100644
index 0000000..4aebdb8
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.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.db.query.reader.sequence;
+
+import java.io.IOException;
+import org.apache.iotdb.db.engine.querycontext.UnsealedTsFile;
+import org.apache.iotdb.db.query.control.FileReaderManager;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
+import org.apache.iotdb.tsfile.read.common.Path;
+import org.apache.iotdb.tsfile.read.controller.ChunkLoader;
+import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl;
+import org.apache.iotdb.tsfile.read.reader.series.SeriesReaderByTimestamp;
+
+public class UnSealedTsFilesReaderByTimestamp implements EngineReaderByTimeStamp {
+
+ protected Path seriesPath;
+ private SeriesReaderByTimestamp unSealedReader;
+
+ /**
+ * Construct funtion for UnSealedTsFileReader.
+ *
+ * @param unsealedTsFile -param to initial
+ */
+ public UnSealedTsFilesReaderByTimestamp(UnsealedTsFile unsealedTsFile) throws IOException {
+
+ TsFileSequenceReader unClosedTsFileReader = FileReaderManager.getInstance()
+ .get(unsealedTsFile.getFilePath(),
+ false);
+ ChunkLoader chunkLoader = new ChunkLoaderImpl(unClosedTsFileReader);
+ unSealedReader = new SeriesReaderByTimestamp(chunkLoader,
+ unsealedTsFile.getChunkMetaDataList());
+
+
+ }
+
+ @Override
+ public Object getValueInTimestamp(long timestamp) throws IOException {
+ return unSealedReader.getValueInTimestamp(timestamp);
+ }
+
+ @Override
+ public boolean hasNext() throws IOException {
+ return unSealedReader.hasNext();
+ }
+
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReader.java
index e49e968..2465cbd 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReader.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReader.java
@@ -16,17 +16,18 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.query.reader.unsequence;
import java.io.IOException;
-import org.apache.iotdb.db.query.reader.IReader;
+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.TsFileSequenceReader;
import org.apache.iotdb.tsfile.read.common.BatchData;
import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader;
-public class EngineChunkReader implements IReader {
+public class EngineChunkReader implements IPointReader {
private ChunkReader chunkReader;
private BatchData data;
@@ -44,15 +45,16 @@ public class EngineChunkReader implements IReader {
@Override
public boolean hasNext() throws IOException {
- if (data == null || !data.hasNext()) {
- if (chunkReader.hasNextBatch()) {
- data = chunkReader.nextBatch();
- } else {
- return false;
+ if (data != null && data.hasNext()) {
+ return true;
+ }
+ while (chunkReader.hasNextBatch()) {
+ data = chunkReader.nextBatch();
+ if (data.hasNext()) {
+ return true;
}
}
-
- return data.hasNext();
+ return false;
}
@Override
@@ -63,8 +65,8 @@ public class EngineChunkReader implements IReader {
}
@Override
- public void skipCurrentTimeValuePair() {
- next();
+ public TimeValuePair current() {
+ return TimeValuePairUtils.getCurrentTimeValuePair(data);
}
@Override
@@ -72,19 +74,4 @@ public class EngineChunkReader implements IReader {
this.chunkReader.close();
this.unClosedTsFileReader.close();
}
-
- @Override
- public boolean hasNextBatch() {
- return false;
- }
-
- @Override
- public BatchData nextBatch() {
- return null;
- }
-
- @Override
- public BatchData currentBatch() {
- return null;
- }
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java
new file mode 100644
index 0000000..c6f22a4
--- /dev/null
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.query.reader.unsequence;
+
+import java.io.IOException;
+import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderByTimestamp;
+
+public class EngineChunkReaderByTimestamp implements EngineReaderByTimeStamp {
+
+ private ChunkReaderByTimestamp chunkReader;
+ private BatchData data;
+
+ public EngineChunkReaderByTimestamp(ChunkReaderByTimestamp chunkReader) {
+ this.chunkReader = chunkReader;
+ }
+
+ /**
+ * get value with time equals timestamp. If there is no such point, return null.
+ */
+ @Override
+ public Object getValueInTimestamp(long timestamp) throws IOException {
+
+ if (!hasNext()) {
+ return null;
+ }
+
+ while (data != null) {
+ Object value = data.getValueInTimestamp(timestamp);
+ if (value != null) {
+ return value;
+ }
+ if (data.hasNext()) {
+ return null;
+ } else {
+ chunkReader.setCurrentTimestamp(timestamp);
+ if (chunkReader.hasNextBatch()) {
+ data = chunkReader.nextBatch();
+ } else {
+ return null;
+ }
+ }
+ }
+
+ return null;
+ }
+
+ @Override
+ public boolean hasNext() throws IOException {
+ if (data != null && data.hasNext()) {
+ return true;
+ }
+ if (chunkReader != null && chunkReader.hasNextBatch()) {
+ data = chunkReader.nextBatch();
+ return true;
+ }
+ return false;
+ }
+}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineLeafNode.java b/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineLeafNode.java
index 292ca0e..3d4b11c 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineLeafNode.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineLeafNode.java
@@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.query.timegenerator;
import java.io.IOException;
@@ -45,16 +46,6 @@ public class EngineLeafNode implements Node {
}
/**
- * check if current time of current batch is equals to input time.
- */
- public boolean currentTimeIs(long time) {
- if (!reader.currentBatch().hasNext()) {
- return false;
- }
- return reader.currentBatch().currentTime() == time;
- }
-
- /**
* check if current value is equals to input value.
*/
public Object currentValue(long time) {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineNodeConstructor.java b/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineNodeConstructor.java
index 0cb02c8..4ffe62b 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineNodeConstructor.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineNodeConstructor.java
@@ -16,6 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
+
package org.apache.iotdb.db.query.timegenerator;
import static org.apache.iotdb.tsfile.read.expression.ExpressionType.AND;
@@ -26,8 +27,9 @@ import java.io.IOException;
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.QueryDataSourceManager;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
import org.apache.iotdb.db.query.factory.SeriesReaderFactory;
+import org.apache.iotdb.db.query.reader.AllDataReader;
import org.apache.iotdb.db.query.reader.IReader;
import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader;
import org.apache.iotdb.db.query.reader.sequence.SequenceDataReader;
@@ -42,10 +44,8 @@ import org.apache.iotdb.tsfile.read.query.timegenerator.node.OrNode;
public class EngineNodeConstructor {
- private long jobId;
- public EngineNodeConstructor(long jobId) {
- this.jobId = jobId;
+ public EngineNodeConstructor() {
}
/**
@@ -87,24 +87,26 @@ public class EngineNodeConstructor {
QueryContext context)
throws IOException, FileNodeManagerException {
- QueryDataSource queryDataSource = QueryDataSourceManager.getQueryDataSource(jobId,
+ QueryDataSource queryDataSource = QueryResourceManager.getInstance().getQueryDataSource(
singleSeriesExpression.getSeriesPath(), context);
Filter filter = singleSeriesExpression.getFilter();
- PriorityMergeReader priorityReader = new PriorityMergeReader();
-
// reader for all sequence data
SequenceDataReader tsFilesReader = new SequenceDataReader(queryDataSource.getSeqDataSource(),
filter, context);
- priorityReader.addReaderWithPriority(tsFilesReader, 1);
// reader for all unSequence data
PriorityMergeReader unSeqMergeReader = SeriesReaderFactory.getInstance()
.createUnSeqMergeReader(queryDataSource.getOverflowSeriesDataSource(), filter);
- priorityReader.addReaderWithPriority(unSeqMergeReader, 2);
- return priorityReader;
+ if (!tsFilesReader.hasNext()) {
+ //only have unsequence data.
+ return unSeqMergeReader;
+ } else {
+ //merge sequence data with unsequence data.
+ return new AllDataReader(tsFilesReader, unSeqMergeReader);
+ }
}
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineTimeGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineTimeGenerator.java
index a151677..350ea6f 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineTimeGenerator.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/query/timegenerator/EngineTimeGenerator.java
@@ -34,20 +34,18 @@ public class EngineTimeGenerator implements TimeGenerator {
private IExpression expression;
private Node operatorNode;
- private long jobId;
/**
* Constructor of EngineTimeGenerator.
*/
- public EngineTimeGenerator(long jobId, IExpression expression, QueryContext context)
- throws IOException, FileNodeManagerException {
- this.jobId = jobId;
+ public EngineTimeGenerator(IExpression expression, QueryContext context)
+ throws FileNodeManagerException {
this.expression = expression;
initNode(context);
}
- private void initNode(QueryContext context) throws IOException, FileNodeManagerException {
- EngineNodeConstructor engineNodeConstructor = new EngineNodeConstructor(jobId);
+ private void initNode(QueryContext context) throws FileNodeManagerException {
+ EngineNodeConstructor engineNodeConstructor = new EngineNodeConstructor();
this.operatorNode = engineNodeConstructor.construct(expression, context);
}
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java b/iotdb/src/main/java/org/apache/iotdb/db/rescon/package-info.java
similarity index 81%
copy from iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
copy to iotdb/src/main/java/org/apache/iotdb/db/rescon/package-info.java
index 3ac1540..96c83c9 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/query/control/QueryTokenManagerTest.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/rescon/package-info.java
@@ -1,29 +1,24 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iotdb.db.query.control;
-
-import org.junit.Test;
-
-public class QueryTokenManagerTest {
-
- @Test
- public void test() {
- //TODO
- }
-}
+/**
+ * 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.
+ */
+
+/**
+ * rescon means Resource Control, classes in this package provide global control over various
+ * resources shared in IoTDB.
+ */
+package org.apache.iotdb.db.rescon;
\ No newline at end of file
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/service/IoTDB.java b/iotdb/src/main/java/org/apache/iotdb/db/service/IoTDB.java
index c63ec74..d85f056 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/service/IoTDB.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/service/IoTDB.java
@@ -105,7 +105,6 @@ public class IoTDB implements IoTDBMBean {
registerManager.register(CloseMergeService.getInstance());
registerManager.register(StatMonitor.getInstance());
registerManager.register(BasicMemController.getInstance());
- registerManager.register(FileReaderManager.getInstance());
registerManager.register(SyncServerManager.getInstance());
JMXService.registerMBean(getInstance(), mbeanName);
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
index c5fd6f7..a68f586 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java
@@ -49,8 +49,8 @@ import org.apache.iotdb.db.qp.logical.Operator;
import org.apache.iotdb.db.qp.physical.PhysicalPlan;
import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
-import org.apache.iotdb.db.query.control.OpenedFilePathsManager;
-import org.apache.iotdb.db.query.control.QueryTokenManager;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
import org.apache.iotdb.service.rpc.thrift.ServerProperties;
import org.apache.iotdb.service.rpc.thrift.TSCancelOperationReq;
import org.apache.iotdb.service.rpc.thrift.TSCancelOperationResp;
@@ -79,6 +79,7 @@ import org.apache.iotdb.service.rpc.thrift.TSSetTimeZoneResp;
import org.apache.iotdb.service.rpc.thrift.TS_SessionHandle;
import org.apache.iotdb.service.rpc.thrift.TS_Status;
import org.apache.iotdb.service.rpc.thrift.TS_StatusCode;
+import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException;
import org.apache.iotdb.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
import org.apache.thrift.TException;
@@ -104,6 +105,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
private ThreadLocal<HashMap<String, QueryDataSet>> queryRet = new ThreadLocal<>();
private ThreadLocal<ZoneId> zoneIds = new ThreadLocal<>();
private IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+ private ThreadLocal<Map<Long, QueryContext>> contextMapLocal = new ThreadLocal<>();
public TSServiceImpl() throws IOException {
// do nothing because there is no need
@@ -184,11 +186,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
public TSCloseOperationResp closeOperation(TSCloseOperationReq req) throws TException {
LOGGER.info("{}: receive close operation", IoTDBConstant.GLOBAL_DB_NAME);
try {
- // end query for all the query tokens created by current thread
- QueryTokenManager.getInstance().endQueryForCurrentRequestThread();
- // remove usage of opened file paths of current thread
- OpenedFilePathsManager.getInstance().removeUsedFilesForCurrentRequestThread();
+ releaseQueryResource(req);
clearAllStatusForCurrentRequest();
} catch (FileNodeManagerException e) {
@@ -197,6 +196,22 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
return new TSCloseOperationResp(new TS_Status(TS_StatusCode.SUCCESS_STATUS));
}
+ private void releaseQueryResource(TSCloseOperationReq req) throws FileNodeManagerException {
+ 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());
+ }
+ } else {
+ QueryResourceManager.getInstance()
+ .endQueryForGivenJob(contextMap.remove(req.queryId).getJobId());
+ }
+ }
+
private void clearAllStatusForCurrentRequest() {
if (this.queryRet.get() != null) {
this.queryRet.get().clear();
@@ -206,7 +221,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
}
}
- private TS_Status getErrorStatus(String message){
+ private TS_Status getErrorStatus(String message) {
TS_Status status = new TS_Status(TS_StatusCode.ERROR_STATUS);
status.setErrorMessage(message);
return status;
@@ -230,7 +245,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
resp.setShowTimeseriesList(showTimeseriesList);
} catch (PathErrorException e) {
status = getErrorStatus(
- String.format("Failed to fetch timeseries %s's metadata because: %s",
+ String.format("Failed to fetch timeseries %s's metadata because: %s",
req.getColumnPath(), e));
resp.setStatus(status);
return resp;
@@ -239,8 +254,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
.error(String.format("Failed to fetch timeseries %s's metadata", req.getColumnPath()),
outOfMemoryError);
status = getErrorStatus(
- String.format("Failed to fetch timeseries %s's metadata because: %s",
- req.getColumnPath(), outOfMemoryError));
+ String.format("Failed to fetch timeseries %s's metadata because: %s",
+ req.getColumnPath(), outOfMemoryError));
resp.setStatus(status);
return resp;
}
@@ -251,14 +266,15 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
Set<String> storageGroups = MManager.getInstance().getAllStorageGroup();
resp.setShowStorageGroups(storageGroups);
} catch (PathErrorException e) {
- status = getErrorStatus(String.format("Failed to fetch storage groups' metadata because: %s", e));
+ status = getErrorStatus(
+ String.format("Failed to fetch storage groups' metadata because: %s", e));
resp.setStatus(status);
return resp;
} catch (OutOfMemoryError outOfMemoryError) { // TODO OOME
LOGGER.error("Failed to fetch storage groups' metadata", outOfMemoryError);
status = getErrorStatus(
- String.format("Failed to fetch storage groups' metadata because: %s",
- outOfMemoryError));
+ String.format("Failed to fetch storage groups' metadata because: %s",
+ outOfMemoryError));
resp.setStatus(status);
return resp;
}
@@ -270,7 +286,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
metadataInJson = MManager.getInstance().getMetadataInString();
} catch (OutOfMemoryError outOfMemoryError) { // TODO OOME
LOGGER.error("Failed to fetch all metadata in json", outOfMemoryError);
- status = getErrorStatus(String.format("Failed to fetch all metadata in json because: %s", outOfMemoryError));
+ status = getErrorStatus(
+ String.format("Failed to fetch all metadata in json because: %s", outOfMemoryError));
resp.setStatus(status);
return resp;
}
@@ -295,7 +312,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
return resp;
} catch (OutOfMemoryError outOfMemoryError) { // TODO OOME
LOGGER.error("Failed to get delta object map", outOfMemoryError);
- status = getErrorStatus(String.format("Failed to get delta object map because: %s", outOfMemoryError));
+ status = getErrorStatus(
+ String.format("Failed to get delta object map because: %s", outOfMemoryError));
break;
}
status = new TS_Status(TS_StatusCode.SUCCESS_STATUS);
@@ -317,7 +335,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
try {
resp.setColumnsList(MManager.getInstance().getPaths(req.getColumnPath()));
} catch (PathErrorException e) {
- status = getErrorStatus(String.format("Failed to fetch %s's all columns because: %s", req.getColumnPath(), e));
+ status = getErrorStatus(String
+ .format("Failed to fetch %s's all columns because: %s", req.getColumnPath(), e));
resp.setStatus(status);
return resp;
} catch (OutOfMemoryError outOfMemoryError) { // TODO OOME
@@ -587,10 +606,7 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
int fetchSize = req.getFetch_size();
QueryDataSet queryDataSet;
if (!queryRet.get().containsKey(statement)) {
- PhysicalPlan physicalPlan = queryStatus.get().get(statement);
- processor.getExecutor().setFetchSize(fetchSize);
- queryDataSet = processor.getExecutor().processQuery(physicalPlan);
- queryRet.get().put(statement, queryDataSet);
+ queryDataSet = createNewDataSet(statement, fetchSize, req);
} else {
queryDataSet = queryRet.get().get(statement);
}
@@ -610,6 +626,26 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext {
}
}
+ private QueryDataSet createNewDataSet(String statement, int fetchSize, TSFetchResultsReq req)
+ throws PathErrorException, QueryFilterOptimizationException, FileNodeManagerException,
+ ProcessorException, IOException {
+ PhysicalPlan physicalPlan = queryStatus.get().get(statement);
+ processor.getExecutor().setFetchSize(fetchSize);
+
+ QueryContext context = new QueryContext(QueryResourceManager.getInstance().assignJobId());
+ Map<Long, QueryContext> contextMap = contextMapLocal.get();
+ if (contextMap == null) {
+ contextMap = new HashMap<>();
+ contextMapLocal.set(contextMap);
+ }
+ contextMap.put(req.queryId, context);
+
+ QueryDataSet queryDataSet = processor.getExecutor().processQuery((QueryPlan) physicalPlan,
+ context);
+ queryRet.get().put(statement, queryDataSet);
+ return queryDataSet;
+ }
+
@Override
public TSExecuteStatementResp executeUpdateStatement(TSExecuteStatementReq req)
throws TException {
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
index 784abc0..8f6fe21 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java
@@ -33,6 +33,9 @@ public class QueryUtils {
/**
* modifyChunkMetaData iterates the chunkMetaData and applies all available modifications on it to
* generate a ModifiedChunkMetadata.
+ * <br/>
+ * the caller should guarantee that chunkMetaData and modifications refer to the same time series
+ * paths.
* @param chunkMetaData the original chunkMetaData.
* @param modifications all possible modifications.
*/
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/TimeValuePairUtils.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/TimeValuePairUtils.java
index 241222a..72fb735 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/utils/TimeValuePairUtils.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/TimeValuePairUtils.java
@@ -18,6 +18,7 @@
*/
package org.apache.iotdb.db.utils;
+import org.apache.iotdb.db.query.aggregation.AggreResultData;
import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException;
import org.apache.iotdb.tsfile.read.common.BatchData;
@@ -51,4 +52,29 @@ public class TimeValuePairUtils {
throw new UnSupportedDataTypeException(String.valueOf(data.getDataType()));
}
}
+
+ /**
+ * get given data's current (time,value) pair.
+ *
+ * @param data -AggreResultData
+ * @return -given data's (time,value) pair
+ */
+ public static TimeValuePair getCurrentTimeValuePair(AggreResultData data) {
+ switch (data.getDataType()) {
+ case INT32:
+ return new TimeValuePair(data.getTimestamp(), new TsPrimitiveType.TsInt(data.getIntRet()));
+ case INT64:
+ return new TimeValuePair(data.getTimestamp(), new TsPrimitiveType.TsLong(data.getLongRet()));
+ case FLOAT:
+ return new TimeValuePair(data.getTimestamp(), new TsPrimitiveType.TsFloat(data.getFloatRet()));
+ case DOUBLE:
+ return new TimeValuePair(data.getTimestamp(), new TsPrimitiveType.TsDouble(data.getDoubleRet()));
+ case TEXT:
+ return new TimeValuePair(data.getTimestamp(), new TsPrimitiveType.TsBinary(data.getBinaryRet()));
+ case BOOLEAN:
+ return new TimeValuePair(data.getTimestamp(), new TsPrimitiveType.TsBoolean(data.isBooleanRet()));
+ default:
+ throw new UnSupportedDataTypeException(String.valueOf(data.getDataType()));
+ }
+ }
}
diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/TsPrimitiveType.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/TsPrimitiveType.java
index ea43c28..d77704e 100644
--- a/iotdb/src/main/java/org/apache/iotdb/db/utils/TsPrimitiveType.java
+++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/TsPrimitiveType.java
@@ -26,7 +26,7 @@ import org.apache.iotdb.tsfile.utils.Binary;
public abstract class TsPrimitiveType implements Serializable {
/**
- * get tsPrimitiveType by dataType.
+ * get tsPrimitiveType by resultDataType.
*
* @param dataType -given TsDataType
* @param v -
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java
index 35bfc84..03d3e57 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java
@@ -20,6 +20,8 @@
package org.apache.iotdb.db.engine.modification;
import static junit.framework.TestCase.assertTrue;
+import static org.apache.iotdb.db.utils.EnvironmentUtils.TEST_QUERY_CONTEXT;
+import static org.apache.iotdb.db.utils.EnvironmentUtils.TEST_QUERY_JOB_ID;
import static org.junit.Assert.assertEquals;
import java.io.File;
@@ -35,8 +37,9 @@ import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
import org.apache.iotdb.db.exception.FileNodeManagerException;
import org.apache.iotdb.db.exception.MetadataArgsErrorException;
import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.StartupException;
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.utils.EnvironmentUtils;
import org.apache.iotdb.db.utils.TimeValuePair;
import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
@@ -67,7 +70,9 @@ public class DeletionFileNodeTest {
@Before
public void setup() throws MetadataArgsErrorException,
- PathErrorException, IOException, FileNodeManagerException {
+ PathErrorException, IOException, FileNodeManagerException, StartupException {
+ EnvironmentUtils.envSetUp();
+
MManager.getInstance().setStorageLevelToMTree(processorName);
for (int i = 0; i < 10; i++) {
MManager.getInstance().addPathToMTree(processorName + "." + measurements[i], dataType,
@@ -103,8 +108,10 @@ public class DeletionFileNodeTest {
SingleSeriesExpression expression = new SingleSeriesExpression(new Path(processorName,
measurements[5]), null);
- QueryContext context = new QueryContext();
- QueryDataSource dataSource = FileNodeManager.getInstance().query(expression, context);
+ QueryResourceManager.getInstance().beginQueryOfGivenExpression(TEST_QUERY_JOB_ID, expression);
+ QueryDataSource dataSource = QueryResourceManager.getInstance()
+ .getQueryDataSource(expression.getSeriesPath(), TEST_QUERY_CONTEXT);
+
Iterator<TimeValuePair> timeValuePairs =
dataSource.getSeqDataSource().getReadableChunk().getIterator();
int count = 0;
@@ -113,6 +120,7 @@ public class DeletionFileNodeTest {
count++;
}
assertEquals(50, count);
+ QueryResourceManager.getInstance().endQueryForGivenJob(TEST_QUERY_JOB_ID);
}
@Test
@@ -185,8 +193,11 @@ public class DeletionFileNodeTest {
SingleSeriesExpression expression = new SingleSeriesExpression(new Path(processorName,
measurements[5]), null);
- QueryContext context = new QueryContext();
- QueryDataSource dataSource = FileNodeManager.getInstance().query(expression, context);
+
+ QueryResourceManager.getInstance().beginQueryOfGivenExpression(TEST_QUERY_JOB_ID, expression);
+ QueryDataSource dataSource = QueryResourceManager.getInstance()
+ .getQueryDataSource(expression.getSeriesPath(), TEST_QUERY_CONTEXT);
+
Iterator<TimeValuePair> timeValuePairs =
dataSource.getOverflowSeriesDataSource().getReadableMemChunk().getIterator();
int count = 0;
@@ -195,6 +206,8 @@ public class DeletionFileNodeTest {
count++;
}
assertEquals(50, count);
+
+ QueryResourceManager.getInstance().endQueryForGivenJob(TEST_QUERY_JOB_ID);
}
@Test
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java
index c3e7dc3..3e9bb99 100644
--- a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java
+++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java
@@ -19,6 +19,7 @@
package org.apache.iotdb.db.engine.modification;
+import static org.apache.iotdb.db.utils.EnvironmentUtils.TEST_QUERY_CONTEXT;
import static org.junit.Assert.assertEquals;
import java.io.IOException;
@@ -30,6 +31,7 @@ import org.apache.iotdb.db.engine.memcontrol.BasicMemController.UsageLevel;
import org.apache.iotdb.db.exception.FileNodeManagerException;
import org.apache.iotdb.db.exception.MetadataArgsErrorException;
import org.apache.iotdb.db.exception.PathErrorException;
+import org.apache.iotdb.db.exception.StartupException;
import org.apache.iotdb.db.metadata.MManager;
import org.apache.iotdb.db.query.executor.EngineQueryRouter;
import org.apache.iotdb.db.utils.EnvironmentUtils;
@@ -63,7 +65,9 @@ public class DeletionQueryTest {
@Before
public void setup() throws MetadataArgsErrorException,
- PathErrorException, IOException, FileNodeManagerException {
+ PathErrorException, IOException, FileNodeManagerException, StartupException {
+ EnvironmentUtils.envSetUp();
+
MManager.getInstance().setStorageLevelToMTree(processorName);
for (int i = 0; i < 10; i++) {
MManager.getInstance().addPathToMTree(processorName + "." + measurements[i], dataType,
@@ -103,7 +107,7 @@ public class DeletionQueryTest {
pathList.add(new Path(processorName, measurements[5]));
QueryExpression queryExpression = QueryExpression.create(pathList, null);
- QueryDataSet dataSet = router.query(queryExpression);
+ QueryDataSet dataSet = router.query(queryExpression, TEST_QUERY_CONTEXT);
int count = 0;
while (dataSet.hasNext()) {
@@ -134,7 +138,7 @@ public class DeletionQueryTest {
pathList.add(new Path(processorName, measurements[5]));
QueryExpression queryExpression = QueryExpression.create(pathList, null);
- QueryDataSet dataSet = router.query(queryExpression);
+ QueryDataSet dataSet = router.query(queryExpression, TEST_QUERY_CONTEXT);
int count = 0;
while (dataSet.hasNext()) {
@@ -176,7 +180,7 @@ public class DeletionQueryTest {
pathList.add(new Path(processorName, measurements[5]));
QueryExpression queryExpression = QueryExpression.create(pathList, null);
- QueryDataSet dataSet = router.query(queryExpression);
+ QueryDataSet dataSet = router.query(queryExpression, TEST_QUERY_CONTEXT);
int count = 0;
while (dataSet.hasNext()) {
@@ -218,7 +222,7 @@ public class DeletionQueryTest {
pathList.add(new Path(processorName, measurements[5]));
QueryExpression queryExpression = QueryExpression.create(pathList, null);
- QueryDataSet dataSet = router.query(queryExpression);
+ QueryDataSet dataSet = router.query(queryExpression, TEST_QUERY_CONTEXT);
int count = 0;
while (dataSet.hasNext()) {
@@ -282,7 +286,7 @@ public class DeletionQueryTest {
pathList.add(new Path(processorName, measurements[5]));
QueryExpression queryExpression = QueryExpression.create(pathList, null);
- QueryDataSet dataSet = router.query(queryExpression);
+ QueryDataSet dataSet = router.query(queryExpression, TEST_QUERY_CONTEXT);
int count = 0;
while (dataSet.hasNext()) {
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IOTDBFillIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IOTDBFillIT.java
new file mode 100644
index 0000000..261cc17
--- /dev/null
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IOTDBFillIT.java
@@ -0,0 +1,351 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iotdb.db.integration;
+
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class IOTDBFillIT {
+
+ private static IoTDB daemon;
+
+ private static String[] dataSet1 = new String[]{
+ "SET STORAGE GROUP TO root.ln.wf01.wt01",
+ "CREATE TIMESERIES root.ln.wf01.wt01.status WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.ln.wf01.wt01.temperature WITH DATATYPE=DOUBLE, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.ln.wf01.wt01.hardware WITH DATATYPE=INT32, ENCODING=PLAIN",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(1, 1.1, false, 11)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(2, 2.2, true, 22)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(3, 3.3, false, 33 )",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(4, 4.4, false, 44)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(5, 5.5, false, 55)",
+ "flush",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(100, 100.1, false, 110)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(150, 200.2, true, 220)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(200, 300.3, false, 330 )",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(250, 400.4, false, 440)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(300, 500.5, false, 550)",
+ "flush",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(10, 10.1, false, 110)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(20, 20.2, true, 220)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(30, 30.3, false, 330 )",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(40, 40.4, false, 440)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(50, 50.5, false, 550)",
+ "flush",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(500, 100.1, false, 110)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(510, 200.2, true, 220)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(520, 300.3, false, 330 )",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(530, 400.4, false, 440)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(540, 500.5, false, 550)",
+ "flush",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(580, 100.1, false, 110)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(590, 200.2, true, 220)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(600, 300.3, false, 330 )",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(610, 400.4, false, 440)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(620, 500.5, false, 550)",
+ };
+
+ private static final String TIMESTAMP_STR = "Time";
+ private static final String TEMPERATURE_STR = "root.ln.wf01.wt01.temperature";
+ private static final String STATUS_STR = "root.ln.wf01.wt01.status";
+ private static final String HARDWARE_STR = "root.ln.wf01.wt01.hardware";
+
+ @Before
+ public void setUp() throws Exception {
+ EnvironmentUtils.closeStatMonitor();
+ EnvironmentUtils.closeMemControl();
+ daemon = IoTDB.getInstance();
+ daemon.active();
+ EnvironmentUtils.envSetUp();
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ prepareData();
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ daemon.stop();
+ EnvironmentUtils.cleanEnv();
+ }
+
+ @Test
+ public void LinearFillTest() throws SQLException {
+ String[] retArray1 = new String[]{
+ "3,3.3,false,33",
+ "70,70.34,false,374",
+ "70,null,null,null",
+ "625,null,false,null"
+ };
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select temperature,status, hardware from "
+ + "root.ln.wf01.wt01 where time = 3 "
+ + "Fill(int32[linear, 5ms, 5ms], double[linear, 5ms, 5ms], boolean[previous, 5ms])");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+ + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+ System.out.println(ans);
+ Assert.assertEquals(retArray1[cnt], ans);
+ cnt++;
+ }
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute("select temperature,status, hardware "
+ + "from root.ln.wf01.wt01 where time = 70 Fill(int32[linear, 500ms, 500ms], "
+ + "double[linear, 500ms, 500ms], boolean[previous, 500ms])");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+ + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+ Assert.assertEquals(retArray1[cnt], ans);
+ cnt++;
+ System.out.println(ans);
+ }
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute("select temperature,status, hardware "
+ + "from root.ln.wf01.wt01 where time = 70 "
+ + "Fill(int32[linear, 25ms, 25ms], double[linear, 25ms, 25ms], boolean[previous, 5ms])");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+ + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+ Assert.assertEquals(retArray1[cnt], ans);
+ cnt++;
+ System.out.println(ans);
+ }
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute("select temperature,status, hardware "
+ + "from root.ln.wf01.wt01 where time = 625 "
+ + "Fill(int32[linear, 25ms, 25ms], double[linear, 25ms, 25ms], boolean[previous, 5ms])");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+ + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+ System.out.println(cnt + " " + ans);
+ Assert.assertEquals(retArray1[cnt], ans);
+ cnt++;
+ }
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test
+ public void PreviousFillTest() throws SQLException {
+ String[] retArray1 = new String[]{
+ "3,3.3,false,33",
+ "70,50.5,false,550",
+ "70,null,null,null"
+ };
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select temperature,status, hardware "
+ + "from root.ln.wf01.wt01 where time = 3 "
+ + "Fill(int32[previous, 5ms], double[previous, 5ms], boolean[previous, 5ms])");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+ + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+ Assert.assertEquals(retArray1[cnt], ans);
+ cnt++;
+ }
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute("select temperature,status, hardware "
+ + "from root.ln.wf01.wt01 where time = 70 "
+ + "Fill(int32[previous, 500ms], double[previous, 500ms], boolean[previous, 500ms])");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+ + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+ Assert.assertEquals(retArray1[cnt], ans);
+ cnt++;
+ System.out.println(ans);
+ }
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute("select temperature,status, hardware "
+ + "from root.ln.wf01.wt01 where time = 70 "
+ + "Fill(int32[previous, 15ms], double[previous, 15ms], boolean[previous, 5ms])");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+ + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+ Assert.assertEquals(retArray1[cnt], ans);
+ cnt++;
+ System.out.println(ans);
+ }
+ statement.close();
+ Assert.assertEquals(retArray1.length, cnt);
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test
+ public void EmptyTimeRangeFillTest() throws SQLException {
+ String[] retArray1 = new String[]{
+ "3,3.3,false,33",
+ "70,70.34,false,374"
+ };
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select temperature,status, hardware "
+ + "from root.ln.wf01.wt01 where time = 3 "
+ + "Fill(int32[linear], double[linear], boolean[previous])");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+ + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+ Assert.assertEquals(retArray1[cnt], ans);
+ cnt++;
+ }
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute("select temperature,status, hardware "
+ + "from root.ln.wf01.wt01 where time = 70 "
+ + "Fill(int32[linear], double[linear], boolean[previous])");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(TEMPERATURE_STR)
+ + "," + resultSet.getString(STATUS_STR) + "," + resultSet.getString(HARDWARE_STR);
+ Assert.assertEquals(retArray1[cnt], ans);
+ cnt++;
+ System.out.println(ans);
+ }
+ statement.close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void prepareData() throws SQLException {
+ Connection connection = null;
+ try {
+ connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root",
+ "root");
+ Statement statement = connection.createStatement();
+
+ for (String sql : dataSet1) {
+ statement.execute(sql);
+ }
+ statement.close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+}
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IOTDBGroupByIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IOTDBGroupByIT.java
new file mode 100644
index 0000000..7d68679
--- /dev/null
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IOTDBGroupByIT.java
@@ -0,0 +1,532 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.integration;
+
+import static org.apache.iotdb.db.integration.Constant.count;
+import static org.apache.iotdb.db.integration.Constant.first;
+import static org.apache.iotdb.db.integration.Constant.last;
+import static org.apache.iotdb.db.integration.Constant.max_time;
+import static org.apache.iotdb.db.integration.Constant.max_value;
+import static org.apache.iotdb.db.integration.Constant.mean;
+import static org.apache.iotdb.db.integration.Constant.min_time;
+import static org.apache.iotdb.db.integration.Constant.min_value;
+import static org.apache.iotdb.db.integration.Constant.sum;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class IOTDBGroupByIT {
+
+ private static IoTDB daemon;
+
+ private static String[] dataSet1 = new String[]{
+ "SET STORAGE GROUP TO root.ln.wf01.wt01",
+ "CREATE TIMESERIES root.ln.wf01.wt01.status WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.ln.wf01.wt01.temperature WITH DATATYPE=DOUBLE, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.ln.wf01.wt01.hardware WITH DATATYPE=INT32, ENCODING=PLAIN",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(1, 1.1, false, 11)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(2, 2.2, true, 22)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(3, 3.3, false, 33 )",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(4, 4.4, false, 44)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(5, 5.5, false, 55)",
+ "flush",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(100, 100.1, false, 110)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(150, 200.2, true, 220)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(200, 300.3, false, 330 )",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(250, 400.4, false, 440)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(300, 500.5, false, 550)",
+ "flush",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(10, 10.1, false, 110)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(20, 20.2, true, 220)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(30, 30.3, false, 330 )",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(40, 40.4, false, 440)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(50, 50.5, false, 550)",
+ "flush",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(500, 100.1, false, 110)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(510, 200.2, true, 220)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(520, 300.3, false, 330 )",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(530, 400.4, false, 440)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(540, 500.5, false, 550)",
+ "flush",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(580, 100.1, false, 110)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(590, 200.2, true, 220)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(600, 300.3, false, 330 )",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(610, 400.4, false, 440)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(620, 500.5, false, 550)",
+ };
+
+ private static final String TIMESTAMP_STR = "Time";
+
+ @Before
+ public void setUp() throws Exception {
+ EnvironmentUtils.closeStatMonitor();
+ EnvironmentUtils.closeMemControl();
+ daemon = IoTDB.getInstance();
+ daemon.active();
+ EnvironmentUtils.envSetUp();
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ prepareData();
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ daemon.stop();
+ EnvironmentUtils.cleanEnv();
+ }
+
+ @Test
+ public void countSumMeanTest() throws SQLException {
+ String[] retArray1 = new String[]{
+ "2,1,4.4,4.4",
+ "5,3,35.8,11.933333333333332",
+ "25,1,30.3,30.3",
+ "50,1,50.5,50.5",
+ "65,0,0.0,null",
+ "85,1,100.1,100.1",
+ "105,0,0.0,null",
+ "125,0,0.0,null",
+ "145,1,200.2,200.2",
+ "310,0,0.0,null"
+ };
+ String[] retArray2 = new String[]{
+ "2,2,7.7,3.85",
+ "5,3,35.8,11.933333333333332",
+ "25,1,30.3,30.3",
+ "50,1,50.5,50.5",
+ "65,0,0.0,null",
+ "85,1,100.1,100.1",
+ "105,0,0.0,null",
+ "125,0,0.0,null",
+ "145,1,200.2,200.2",
+ "310,0,0.0,null"
+ };
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute(
+ "select count(temperature), sum(temperature), mean(temperature) from "
+ + "root.ln.wf01.wt01 where time > 3 "
+ + "GROUP BY (20ms, 5,[2,30], [35,37], [50, 160], [310, 314])");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+ .getString(count("root.ln.wf01.wt01.temperature")) + "," +
+ resultSet.getString(sum("root.ln.wf01.wt01.temperature")) + "," + resultSet
+ .getString(mean("root.ln.wf01.wt01.temperature"));
+ Assert.assertEquals(retArray1[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(retArray1.length, cnt);
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute(
+ "select count(temperature), sum(temperature), mean(temperature) from "
+ + "root.ln.wf01.wt01 where temperature > 3 "
+ + "GROUP BY (20ms, 5,[2,30], [35,37], [50, 160], [310, 314])");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+ .getString(count("root.ln.wf01.wt01.temperature")) + "," +
+ resultSet.getString(sum("root.ln.wf01.wt01.temperature")) + "," + resultSet
+ .getString(mean("root.ln.wf01.wt01.temperature"));
+ Assert.assertEquals(retArray2[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(retArray2.length, cnt);
+ statement.close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test
+ public void maxMinValeTimeTest() throws SQLException {
+ String[] retArray1 = new String[]{
+ "2,4.4,4.4,4,4",
+ "5,20.2,5.5,20,5",
+ "25,30.3,30.3,30,30",
+ "50,50.5,50.5,50,50",
+ "65,null,null,null,null",
+ "85,100.1,100.1,100,100",
+ "105,null,null,null,null",
+ "125,null,null,null,null",
+ "145,200.2,200.2,150,150",
+ "310,null,null,null,null"
+ };
+ String[] retArray2 = new String[]{
+ "2,4.4,3.3,4,3",
+ "5,20.2,5.5,20,5",
+ "25,30.3,30.3,30,30",
+ "50,50.5,50.5,50,50",
+ "65,null,null,null,null",
+ "85,100.1,100.1,100,100",
+ "105,null,null,null,null",
+ "125,null,null,null,null",
+ "145,200.2,200.2,150,150",
+ "310,null,null,null,null"
+ };
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute(
+ "select max_value(temperature), min_value(temperature), max_time(temperature), "
+ + "min_time(temperature) from root.ln.wf01.wt01 where time > 3 "
+ + "GROUP BY (20ms, 5,[2,30], [35,37], [50, 160], [310, 314])");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+ .getString(max_value("root.ln.wf01.wt01.temperature"))
+ + "," + resultSet.getString(min_value("root.ln.wf01.wt01.temperature")) + ","
+ + resultSet.getString(max_time("root.ln.wf01.wt01.temperature"))
+ + "," + resultSet.getString(min_time("root.ln.wf01.wt01.temperature"));
+ Assert.assertEquals(retArray1[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(retArray1.length, cnt);
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute(
+ "select max_value(temperature), min_value(temperature), max_time(temperature), "
+ + "min_time(temperature) from root.ln.wf01.wt01 where temperature > 3 "
+ + "GROUP BY (20ms, 5,[2,30], [35,37], [50, 160], [310, 314])");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+ .getString(max_value("root.ln.wf01.wt01.temperature"))
+ + "," + resultSet.getString(min_value("root.ln.wf01.wt01.temperature")) + ","
+ + resultSet.getString(max_time("root.ln.wf01.wt01.temperature"))
+ + "," + resultSet.getString(min_time("root.ln.wf01.wt01.temperature"));
+ Assert.assertEquals(retArray2[cnt], ans);
+ cnt++;
+ //System.out.println(ans);
+ }
+ Assert.assertEquals(retArray2.length, cnt);
+ statement.close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test
+ public void firstLastTest() throws SQLException {
+ String[] retArray1 = new String[]{
+ "2,4.4,4.4",
+ "5,20.2,5.5",
+ "25,30.3,30.3",
+ "50,50.5,50.5",
+ "65,null,null",
+ "85,100.1,100.1",
+ "105,null,null",
+ "125,null,null",
+ "145,200.2,200.2",
+ "310,null,null"
+ };
+ String[] retArray2 = new String[]{
+ "2,4.4,3.3",
+ "5,20.2,5.5",
+ "25,30.3,30.3",
+ "50,50.5,50.5",
+ "65,null,null",
+ "85,100.1,100.1",
+ "105,null,null",
+ "125,null,null",
+ "145,200.2,200.2",
+ "310,null,null"
+ };
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute(
+ "select last(temperature), first(temperature) from root.ln.wf01.wt01 where time > 3 "
+ + "GROUP BY (20ms, 5,[2,30], [35,37], [50, 160], [310, 314])");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+ .getString(last("root.ln.wf01.wt01.temperature"))
+ + "," + resultSet.getString(first("root.ln.wf01.wt01.temperature"));
+ System.out.println(ans);
+ Assert.assertEquals(retArray1[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(retArray1.length, cnt);
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute(
+ "select first(temperature), last(temperature) from root.ln.wf01.wt01 "
+ + "where temperature > 3 "
+ + "GROUP BY (20ms, 5,[2,30], [35,37], [50, 160], [310, 314])");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+ .getString(last("root.ln.wf01.wt01.temperature"))
+ + "," + resultSet.getString(first("root.ln.wf01.wt01.temperature"));
+ System.out.println(ans);
+ Assert.assertEquals(retArray2[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(retArray2.length, cnt);
+ statement.close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test
+ public void largeIntervalTest() throws SQLException {
+ String[] retArray1 = new String[]{
+ "2,4.4,4,20,4",
+ "30,30.3,16,610,30",
+ "620,500.5,1,620,620"
+ };
+ String[] retArray2 = new String[]{
+ "2,3.3,5,20,3",
+ "30,30.3,16,610,30",
+ "620,500.5,1,620,620"
+ };
+
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute(
+ "select min_value(temperature), count(temperature), max_time(temperature), "
+ + "min_time(temperature) from root.ln.wf01.wt01 where time > 3 GROUP BY "
+ + "(590ms, 30, [2, 30], [30, 120], [100, 120], [123, 125], [155, 550], [540, 680])");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+ .getString(min_value("root.ln.wf01.wt01.temperature"))
+ + "," + resultSet.getString(count("root.ln.wf01.wt01.temperature")) + "," +
+ resultSet.getString(max_time("root.ln.wf01.wt01.temperature"))
+ + "," + resultSet.getString(min_time("root.ln.wf01.wt01.temperature"));
+ Assert.assertEquals(retArray1[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(retArray1.length, cnt);
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute(
+ "select min_value(temperature), count (temperature), max_time(temperature), "
+ + "min_time(temperature) from root.ln.wf01.wt01 where temperature > 3 GROUP BY "
+ + "(590ms, 30, [2, 30], [30, 120], [100, 120], [123, 125], [155, 550],[540, 680])");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+ .getString(min_value("root.ln.wf01.wt01.temperature"))
+ + "," + resultSet.getString(count("root.ln.wf01.wt01.temperature")) + ","
+ + resultSet.getString(max_time("root.ln.wf01.wt01.temperature"))
+ + "," + resultSet.getString(min_time("root.ln.wf01.wt01.temperature"));
+ Assert.assertEquals(retArray2[cnt], ans);
+ cnt++;
+ //System.out.println(ans);
+ }
+ Assert.assertEquals(retArray2.length, cnt);
+ statement.close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test
+ public void smallPartitionTest() throws SQLException {
+ String[] retArray1 = new String[]{
+ "50,100.1,50.5,150.6",
+ "615,500.5,500.5,500.5"
+
+ };
+ String[] retArray2 = new String[]{
+ "50,100.1,50.5,150.6",
+ "585,null,null,0.0",
+ "590,500.5,200.2,700.7"
+ };
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute(
+ "select last(temperature), first(temperature), sum(temperature) from "
+ + "root.ln.wf01.wt01 where time > 3 "
+ + "GROUP BY (80ms, 30,[50,100], [615, 650])");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+ .getString(last("root.ln.wf01.wt01.temperature"))
+ + "," + resultSet.getString(first("root.ln.wf01.wt01.temperature")) + ","
+ + resultSet.getString(sum("root.ln.wf01.wt01.temperature"));
+ System.out.println(ans);
+ Assert.assertEquals(retArray1[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(retArray1.length, cnt);
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute(
+ "select first(temperature), last(temperature), sum(temperature) from "
+ + "root.ln.wf01.wt01 where temperature > 3 "
+ + "GROUP BY (80ms, 30,[50,100], [585,590], [615, 650])");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet
+ .getString(last("root.ln.wf01.wt01.temperature"))
+ + "," + resultSet.getString(first("root.ln.wf01.wt01.temperature")) + ","
+ + resultSet.getString(sum("root.ln.wf01.wt01.temperature"));
+ System.out.println(ans);
+ Assert.assertEquals(retArray2[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(retArray2.length, cnt);
+ statement.close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ public void prepareData() throws SQLException {
+ Connection connection = null;
+ try {
+ connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root",
+ "root");
+ Statement statement = connection.createStatement();
+
+ for (String sql : dataSet1) {
+ statement.execute(sql);
+ }
+ statement.close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+}
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationIT.java
new file mode 100644
index 0000000..47ff18c
--- /dev/null
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationIT.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.db.integration;
+
+import static org.apache.iotdb.db.integration.Constant.count;
+import static org.apache.iotdb.db.integration.Constant.first;
+import static org.apache.iotdb.db.integration.Constant.last;
+import static org.apache.iotdb.db.integration.Constant.max_time;
+import static org.apache.iotdb.db.integration.Constant.max_value;
+import static org.apache.iotdb.db.integration.Constant.mean;
+import static org.apache.iotdb.db.integration.Constant.min_time;
+import static org.apache.iotdb.db.integration.Constant.min_value;
+import static org.apache.iotdb.db.integration.Constant.sum;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class IoTDBAggregationIT {
+
+ private static IoTDB daemon;
+
+ private static String[] creationSqls = new String[]{
+ "SET STORAGE GROUP TO root.vehicle.d0",
+ "SET STORAGE GROUP TO root.vehicle.d1",
+
+ "CREATE TIMESERIES root.vehicle.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d0.s1 WITH DATATYPE=INT64, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d0.s2 WITH DATATYPE=FLOAT, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d0.s3 WITH DATATYPE=TEXT, ENCODING=PLAIN"
+ };
+
+ private static String[] dataSet2 = new String[]{
+ "SET STORAGE GROUP TO root.ln.wf01.wt01",
+ "CREATE TIMESERIES root.ln.wf01.wt01.status WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.ln.wf01.wt01.temperature WITH DATATYPE=FLOAT, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.ln.wf01.wt01.hardware WITH DATATYPE=INT32, ENCODING=PLAIN",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(1, 1.1, false, 11)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(2, 2.2, true, 22)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(3, 3.3, false, 33 )",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(4, 4.4, false, 44)",
+ "INSERT INTO root.ln.wf01.wt01(timestamp,temperature,status, hardware) "
+ + "values(5, 5.5, false, 55)"
+ };
+
+ private String insertTemplate = "INSERT INTO root.vehicle.d0(timestamp,s0,s1,s2,s3)"
+ + " VALUES(%d,%d,%d,%f,%s)";
+
+ private static final String TIMESTAMP_STR = "Time";
+ private final String d0s0 = "root.vehicle.d0.s0";
+ private final String d0s1 = "root.vehicle.d0.s1";
+ private final String d0s2 = "root.vehicle.d0.s2";
+ private final String d0s3 = "root.vehicle.d0.s3";
+ private static final String TEMPERATURE_STR = "root.ln.wf01.wt01.temperature";
+
+ @Before
+ public void setUp() throws Exception {
+ EnvironmentUtils.closeStatMonitor();
+ EnvironmentUtils.closeMemControl();
+ daemon = IoTDB.getInstance();
+ daemon.active();
+ EnvironmentUtils.envSetUp();
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ prepareData();
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ daemon.stop();
+ EnvironmentUtils.cleanEnv();
+ }
+
+ //add test for part of points in page don't satisfy filter
+ //details in: https://issues.apache.org/jira/projects/IOTDB/issues/IOTDB-54
+ @Test
+ public void test() throws SQLException {
+ String[] retArray = new String[]{
+ "0,2",
+ "0,4",
+ "0,3"
+ };
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute(
+ "select count(temperature) from root.ln.wf01.wt01 where time > 3");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+ resultSet.getString(count(TEMPERATURE_STR));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute(
+ "select min_time(temperature) from root.ln.wf01.wt01 where time > 3");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+ resultSet.getString(min_time(TEMPERATURE_STR));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(2, cnt);
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute(
+ "select min_time(temperature) from root.ln.wf01.wt01 where temperature > 3");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+ resultSet.getString(min_time(TEMPERATURE_STR));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(3, cnt);
+ statement.close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test
+ public void countTest() throws SQLException {
+ String[] retArray = new String[]{
+ "0,2001,2001,2001,2001",
+ "0,7500,7500,7500,7500"
+ };
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select count(s0),count(s1),count(s2),count(s3) " +
+ "from root.vehicle.d0 where time >= 6000 and time <= 9000");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(count(d0s0))
+ + "," + resultSet.getString(count(d0s1)) + "," + resultSet.getString(count(d0s2))
+ + "," + resultSet.getString(count(d0s3));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute("select count(s0),count(s1),count(s2),count(s3) " +
+ "from root.vehicle.d0");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(count(d0s0))
+ + "," + resultSet.getString(count(d0s1)) + "," + resultSet.getString(count(d0s2))
+ + "," + resultSet.getString(count(d0s3));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(2, cnt);
+ statement.close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test
+ public void firstTest() throws SQLException {
+ String[] retArray = new String[]{
+ "0,2000,2000,2000.0,2000",
+ "0,500,500,500.0,500"
+ };
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select first(s0),first(s1),first(s2),first(s3) " +
+ "from root.vehicle.d0 where time >= 1500 and time <= 9000");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(first(d0s0))
+ + "," + resultSet.getString(first(d0s1)) + "," + resultSet.getString(first(d0s2))
+ + "," + resultSet.getString(first(d0s3));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute("select first(s0),first(s1),first(s2),first(s3) " +
+ "from root.vehicle.d0");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(first(d0s0))
+ + "," + resultSet.getString(first(d0s1)) + "," + resultSet.getString(first(d0s2))
+ + "," + resultSet.getString(first(d0s3));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(2, cnt);
+ statement.close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test
+ public void lastTest() throws SQLException {
+ String[] retArray = new String[]{
+ "0,8499,8499.0",
+ "0,1499,1499.0",
+ "0,2200,2200.0"
+ };
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select last(s0),last(s2) " +
+ "from root.vehicle.d0 where time >= 1500 and time < 9000");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(last(d0s0))
+ + "," + resultSet.getString(last(d0s2));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute("select last(s0),last(s2) " +
+ "from root.vehicle.d0 where time <= 1600");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(last(d0s0))
+ + "," + resultSet.getString(last(d0s2));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(2, cnt);
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute("select last(s0),last(s2) " +
+ "from root.vehicle.d0 where time <= 2200");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(last(d0s0))
+ + "," + resultSet.getString(last(d0s2));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(3, cnt);
+ statement.close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test
+ public void maxminTimeTest() throws SQLException {
+ String[] retArray = new String[]{
+ "0,8499,500",
+ "0,2499,2000"
+ };
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select max_time(s0),min_time(s2) " +
+ "from root.vehicle.d0 where time >= 100 and time < 9000");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_time(d0s0))
+ + "," + resultSet.getString(min_time(d0s2));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute("select max_time(s0),min_time(s2) " +
+ "from root.vehicle.d0 where time <= 2500 and time > 1800");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_time(d0s0))
+ + "," + resultSet.getString(min_time(d0s2));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(2, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test
+ public void maxminValueTest() throws SQLException {
+ String[] retArray = new String[]{
+ "0,8499,500.0",
+ "0,2499,500.0"
+ };
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select max_value(s0),min_value(s2) " +
+ "from root.vehicle.d0 where time >= 100 and time < 9000");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_value(d0s0))
+ + "," + resultSet.getString(min_value(d0s2));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute("select max_value(s0),min_value(s2) " +
+ "from root.vehicle.d0 where time < 2500");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_value(d0s0))
+ + "," + resultSet.getString(min_value(d0s2));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(2, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test
+ public void meanSumTest() throws SQLException {
+ String[] retArray = new String[]{
+ "0,1.4508E7,7250.374812593703",
+ "0,626750.0,1250.998003992016"
+ };
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select sum(s0),mean(s2)" +
+ "from root.vehicle.d0 where time >= 6000 and time <= 9000");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(sum(d0s0))
+ + "," + resultSet.getString(mean(d0s2));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+
+ statement = connection.createStatement();
+ hasResultSet = statement.execute("select sum(s0),mean(s2)" +
+ "from root.vehicle.d0 where time >= 1000 and time <= 2000");
+
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(sum(d0s0))
+ + "," + resultSet.getString(mean(d0s2));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(2, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void prepareData() throws SQLException {
+ Connection connection = null;
+ try {
+ connection = DriverManager
+ .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root",
+ "root");
+ Statement statement = connection.createStatement();
+ for (String sql : creationSqls) {
+ statement.execute(sql);
+ }
+
+ for (String sql : dataSet2) {
+ statement.execute(sql);
+ }
+ statement.close();
+
+ statement = connection.createStatement();
+ // prepare BufferWrite file
+ for (int i = 5000; i < 7000; i++) {
+ statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'"));
+ }
+ statement.execute("flush");
+ for (int i = 7500; i < 8500; i++) {
+ statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'"));
+ }
+ statement.execute("flush");
+ // prepare Unseq-File
+ for (int i = 500; i < 1500; i++) {
+ statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'"));
+ }
+ statement.execute("flush");
+ for (int i = 3000; i < 6500; i++) {
+ statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'"));
+ }
+ statement.execute("merge");
+
+ // prepare BufferWrite cache
+ for (int i = 9000; i < 10000; i++) {
+ statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'"));
+ }
+ // prepare Overflow cache
+ for (int i = 2000; i < 2500; i++) {
+ statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'"));
+ }
+ statement.close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+}
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationLargeDataIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationLargeDataIT.java
new file mode 100644
index 0000000..d0f782d
--- /dev/null
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationLargeDataIT.java
@@ -0,0 +1,900 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.integration;
+
+import static org.apache.iotdb.db.integration.Constant.count;
+import static org.apache.iotdb.db.integration.Constant.first;
+import static org.apache.iotdb.db.integration.Constant.max_time;
+import static org.apache.iotdb.db.integration.Constant.max_value;
+import static org.apache.iotdb.db.integration.Constant.mean;
+import static org.apache.iotdb.db.integration.Constant.min_value;
+import static org.apache.iotdb.db.integration.Constant.sum;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class IoTDBAggregationLargeDataIT {
+
+ private static final String TIMESTAMP_STR = "Time";
+ private final String d0s0 = "root.vehicle.d0.s0";
+ private final String d0s1 = "root.vehicle.d0.s1";
+ private final String d0s2 = "root.vehicle.d0.s2";
+ private final String d0s3 = "root.vehicle.d0.s3";
+ private final String d0s4 = "root.vehicle.d0.s4";
+
+ private static String[] createSql = new String[]{
+ "SET STORAGE GROUP TO root.vehicle",
+ "CREATE TIMESERIES root.vehicle.d1.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d0.s2 WITH DATATYPE=FLOAT, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d0.s3 WITH DATATYPE=TEXT, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.vehicle.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d0.s1 WITH DATATYPE=INT64, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d0.s4 WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
+ };
+
+ private static String[] insertSql = new String[]{
+ "insert into root.vehicle.d0(timestamp,s0) values(1,101)",
+ "insert into root.vehicle.d0(timestamp,s0) values(2,198)",
+ "insert into root.vehicle.d0(timestamp,s0) values(100,99)",
+ "insert into root.vehicle.d0(timestamp,s0) values(101,99)",
+ "insert into root.vehicle.d0(timestamp,s0) values(102,80)",
+ "insert into root.vehicle.d0(timestamp,s0) values(103,99)",
+ "insert into root.vehicle.d0(timestamp,s0) values(104,90)",
+ "insert into root.vehicle.d0(timestamp,s0) values(105,99)",
+ "insert into root.vehicle.d0(timestamp,s0) values(106,99)",
+ "insert into root.vehicle.d0(timestamp,s0) values(2,10000)",
+ "insert into root.vehicle.d0(timestamp,s0) values(50,10000)",
+ "insert into root.vehicle.d0(timestamp,s0) values(1000,22222)",
+ "insert into root.vehicle.d0(timestamp,s0) values(106,199)",
+ "DELETE FROM root.vehicle.d0.s0 WHERE time < 104",
+
+ "insert into root.vehicle.d0(timestamp,s1) values(1,1101)",
+ "insert into root.vehicle.d0(timestamp,s1) values(2,198)",
+ "insert into root.vehicle.d0(timestamp,s1) values(100,199)",
+ "insert into root.vehicle.d0(timestamp,s1) values(101,199)",
+ "insert into root.vehicle.d0(timestamp,s1) values(102,180)",
+ "insert into root.vehicle.d0(timestamp,s1) values(103,199)",
+ "insert into root.vehicle.d0(timestamp,s1) values(104,190)",
+ "insert into root.vehicle.d0(timestamp,s1) values(105,199)",
+ "insert into root.vehicle.d0(timestamp,s1) values(2,40000)",
+ "insert into root.vehicle.d0(timestamp,s1) values(50,50000)",
+ "insert into root.vehicle.d0(timestamp,s1) values(1000,55555)",
+
+ "insert into root.vehicle.d0(timestamp,s2) values(1000,55555)",
+ "insert into root.vehicle.d0(timestamp,s2) values(2,2.22)",
+ "insert into root.vehicle.d0(timestamp,s2) values(3,3.33)",
+ "insert into root.vehicle.d0(timestamp,s2) values(4,4.44)",
+ "insert into root.vehicle.d0(timestamp,s2) values(102,10.00)",
+ "insert into root.vehicle.d0(timestamp,s2) values(105,11.11)",
+ "insert into root.vehicle.d0(timestamp,s2) values(1000,1000.11)",
+
+ "insert into root.vehicle.d0(timestamp,s3) values(60,'aaaaa')",
+ "insert into root.vehicle.d0(timestamp,s3) values(70,'bbbbb')",
+ "insert into root.vehicle.d0(timestamp,s3) values(80,'ccccc')",
+ "insert into root.vehicle.d0(timestamp,s3) values(101,'ddddd')",
+ "insert into root.vehicle.d0(timestamp,s3) values(102,'fffff')",
+
+ "insert into root.vehicle.d1(timestamp,s0) values(1,999)",
+ "insert into root.vehicle.d1(timestamp,s0) values(1000,888)",
+
+ "insert into root.vehicle.d0(timestamp,s4) values(100, false)",
+ "insert into root.vehicle.d0(timestamp,s4) values(100, true)",
+ };
+
+ private IoTDB daemon;
+
+ @Before
+ public void setUp() throws Exception {
+ EnvironmentUtils.closeStatMonitor();
+ EnvironmentUtils.closeMemControl();
+ daemon = IoTDB.getInstance();
+ daemon.active();
+ EnvironmentUtils.envSetUp();
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ daemon.stop();
+ EnvironmentUtils.cleanEnv();
+ }
+
+ @Test
+ public void test() throws ClassNotFoundException, SQLException {
+ insertSQL();
+
+ Connection connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ lastAggreWithSingleFilterTest();
+ meanAggreWithSingleFilterTest();
+ sumAggreWithSingleFilterTest();
+ firstAggreWithSingleFilterTest();
+ countAggreWithSingleFilterTest();
+ minMaxTimeAggreWithSingleFilterTest();
+ minValueAggreWithSingleFilterTest();
+ maxValueAggreWithSingleFilterTest();
+
+ lastAggreWithMultiFilterTest();
+ countAggreWithMultiFilterTest();
+ minTimeAggreWithMultiFilterTest();
+ maxTimeAggreWithMultiFilterTest();
+ minValueAggreWithMultiFilterTest();
+ maxValueAggreWithMultiFilterTest();
+ meanAggreWithMultiFilterTest();
+ sumAggreWithMultiFilterTest();
+ firstAggreWithMultiFilterTest();
+ connection.close();
+
+ }
+
+ private void lastAggreWithMultiFilterTest() throws SQLException {
+ String[] retArray = new String[]{
+ "0,9,39,63.0,E,true"
+ };
+
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute(
+ "select last(s0),last(s1),last(s2),last(s3),last(s4)" +
+ " from root.vehicle.d0 where s1 >= 0");
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+ resultSet.getString(Constant.last(d0s0)) + "," + resultSet
+ .getString(Constant.last(d0s1))
+ + "," + resultSet.getString(Constant.last(d0s2)) + "," +
+ resultSet.getString(Constant.last(d0s3)) + "," + resultSet
+ .getString(Constant.last(d0s4));
+ // System.out.println("============ " + ans);
+ Assert.assertEquals(ans, retArray[cnt]);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void lastAggreWithSingleFilterTest() throws SQLException {
+ String[] retArray = new String[]{
+ "0,9,39,63.0"
+ };
+
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select last(s0),last(s1),last(s2)" +
+ " from root.vehicle.d0 where s1 >= 0");
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+ resultSet.getString(Constant.last(d0s0)) + ","
+ + resultSet.getString(Constant.last(d0s1)) + "," +
+ resultSet.getString(Constant.last(d0s2));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void sumAggreWithSingleFilterTest() throws SQLException {
+ String[] retArray = new String[]{
+ "0,55061.0,156752.0,20254"
+ };
+
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select sum(s0),sum(s1),sum(s2)" +
+ " from root.vehicle.d0 where s1 >= 0");
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(sum(d0s0)) + ","
+ + resultSet.getString(sum(d0s1)) + "," + Math
+ .round(resultSet.getDouble(Constant.sum(d0s2)));
+ Assert.assertEquals(ans, retArray[cnt]);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void firstAggreWithSingleFilterTest() throws SQLException {
+ String[] retArray = new String[]{
+ "0,90,1101,2.22,ddddd,true"
+ };
+
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select first(s0),first(s1),first(s2),first(s3),"
+ + "first(s4) from root.vehicle.d0 where s1 >= 0");
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+ resultSet.getString(Constant.first(d0s0)) + ","
+ + resultSet.getString(Constant.first(d0s1)) + "," +
+ resultSet.getString(Constant.first(d0s2))
+ + "," + resultSet.getString(Constant.first(d0s3)) + "," +
+ resultSet.getString(Constant.first(d0s4));
+ Assert.assertEquals(ans, retArray[cnt]);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void meanAggreWithSingleFilterTest() throws SQLException {
+ String[] retArray = new String[]{
+ "0,75,212,28"
+ };
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute(
+ "select mean(s0),mean(s1),mean(s2) from root.vehicle.d0 where s1 >= 0");
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans =
+ resultSet.getString(TIMESTAMP_STR) + "," + Math.round(resultSet.getDouble(mean(d0s0)))
+ + "," + Math.round(resultSet.getDouble(mean(d0s1))) + "," +
+ Math.round(resultSet.getDouble(mean(d0s2)));
+ //System.out.println("!!!!!============ " + ans);
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void countAggreWithSingleFilterTest() throws SQLException {
+ String[] retArray = new String[]{
+ "0,733,740,734"
+ };
+
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute(
+ "select count(s0),count(s1),count(s2) from root.vehicle.d0 where s1 >= 0");
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(count(d0s0))
+ + "," + resultSet.getString(count(d0s1)) + "," + resultSet.getString(count(d0s2));
+ //System.out.println("!!!!!============ " + ans);
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void minMaxTimeAggreWithSingleFilterTest() throws ClassNotFoundException, SQLException {
+ String[] retArray = new String[]{
+ "0,104,1,2,101,100"
+ };
+
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute(
+ "select min_time(s0),min_time(s1),min_time(s2),min_time(s3),min_time(s4)" +
+ " from root.vehicle.d0 where s1 >= 0");
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+ resultSet.getString(Constant.min_time(d0s0)) + "," +
+ resultSet.getString(Constant.min_time(d0s1)) + "," +
+ resultSet.getString(Constant.min_time(d0s2)) +
+ "," + resultSet.getString(Constant.min_time(d0s3)) +
+ "," + resultSet.getString(Constant.min_time(d0s4));
+ Assert.assertEquals(ans, retArray[cnt]);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+
+ retArray = new String[]{
+ "0,3999,3999,3999,3599,100"
+ };
+ statement = connection.createStatement();
+ hasResultSet = statement.execute("select max_time(s0),max_time(s1),max_time(s2),"
+ + "max_time(s3),max_time(s4) from root.vehicle.d0 " +
+ "where s1 < 50000 and s1 != 100");
+ Assert.assertTrue(hasResultSet);
+ resultSet = statement.getResultSet();
+ cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_time(d0s0))
+ + "," + resultSet.getString(max_time(d0s1)) + "," + resultSet.getString(max_time(d0s2))
+ + "," + resultSet.getString(max_time(d0s3)) + "," + resultSet.getString(max_time(d0s4));
+ Assert.assertEquals(ans, retArray[cnt]);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void minValueAggreWithSingleFilterTest() throws ClassNotFoundException, SQLException {
+ String[] retArray = new String[]{
+ "0,0,0,0.0,B,true"
+ };
+
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ Connection connection = null;
+ try {
+ connection = DriverManager.getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select min_value(s0),min_value(s1),min_value(s2),"
+ + "min_value(s3),min_value(s4) from root.vehicle.d0 " +
+ "where s1 < 50000 and s1 != 100");
+
+ if (hasResultSet) {
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans =
+ resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(min_value(d0s0))
+ + "," + resultSet.getString(min_value(d0s1)) + "," + resultSet
+ .getString(min_value(d0s2))
+ + "," + resultSet.getString(min_value(d0s3)) + "," + resultSet
+ .getString(min_value(d0s4));
+ //System.out.println("============ " + ans);
+ Assert.assertEquals(ans, retArray[cnt]);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ }
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void maxValueAggreWithSingleFilterTest() throws ClassNotFoundException, SQLException {
+ String[] retArray = new String[]{
+ "0,99,40000,122.0,fffff,true"
+ };
+
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select max_value(s0),max_value(s1),max_value(s2),"
+ + "max_value(s3),max_value(s4) from root.vehicle.d0 " +
+ "where s1 < 50000 and s1 != 100");
+
+ if (hasResultSet) {
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans =
+ resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_value(d0s0))
+ + "," + resultSet.getString(max_value(d0s1)) + "," + resultSet
+ .getString(max_value(d0s2))
+ + "," + resultSet.getString(max_value(d0s3)) + "," + resultSet
+ .getString(max_value(d0s4));
+ //System.out.println("============ " + ans);
+ Assert.assertEquals(ans, retArray[cnt]);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ }
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void meanAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+ String[] retArray = new String[]{
+ "0,55061.0,733,75,212,28"
+ };
+
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select sum(s0),count(s0),mean(s0),mean(s1),"
+ + "mean(s2) from root.vehicle.d0 " +
+ "where s1 >= 0 or s2 < 10");
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans =
+ resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(sum(d0s0)) + "," +
+ resultSet.getString(count(d0s0)) + "," + Math.round(resultSet.getDouble(mean(d0s0)))
+ + "," +
+ Math.round(resultSet.getDouble(mean(d0s1))) + "," + Math
+ .round(resultSet.getDouble(mean(d0s2)));
+ //System.out.println("!!!!!============ " + ans);
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void sumAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+ String[] retArray = new String[]{
+ "0,55061.0,156752.0,20262"
+ };
+
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select sum(s0),sum(s1),sum(s2) from root.vehicle.d0"
+ + " where s1 >= 0 or s2 < 10");
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(sum(d0s0))
+ + "," + resultSet.getString(sum(d0s1)) + "," + Math
+ .round(resultSet.getDouble(sum(d0s2)));
+ //String ans = resultSet.getString(sum(d0s3));
+ //System.out.println("!!!!!============ " + ans);
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void firstAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+ String[] retArray = new String[]{
+ "0,90,1101,2.22,ddddd,true"
+ };
+
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select first(s0),first(s1),first(s2),first(s3),"
+ + "first(s4) from root.vehicle.d0 where s1 >= 0 or s2 < 10");
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(first(d0s0))
+ + "," + resultSet.getString(first(d0s1)) + "," + resultSet.getString(first(d0s2))
+ + "," + resultSet.getString(first(d0s3)) + "," + resultSet.getString(first(d0s4));
+ //String ans = resultSet.getString(first(d0s3));
+ //System.out.println("!!!!!============ " + ans);
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void countAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+ String[] retArray = new String[]{
+ "0,733,740,736,482,1"
+ };
+
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select count(s0),count(s1),count(s2),count(s3),"
+ + "count(s4) from root.vehicle.d0 where s1 >= 0 or s2 < 10");
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(count(d0s0))
+ + "," + resultSet.getString(count(d0s1)) + "," + resultSet.getString(count(d0s2))
+ + "," + resultSet.getString(count(d0s3)) + "," + resultSet.getString(count(d0s4));
+ //String ans = resultSet.getString(count(d0s3));
+ //System.out.println("!!!!!============ " + ans);
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void minTimeAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+ String[] retArray = new String[]{
+ "0,104,1,2,101,100"
+ };
+
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select min_time(s0),min_time(s1),min_time(s2),"
+ + "min_time(s3),min_time(s4) from root.vehicle.d0 where s1 >= 0");
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," +
+ resultSet.getString(Constant.min_time(d0s0)) + "," +
+ resultSet.getString(Constant.min_time(d0s1)) + "," +
+ resultSet.getString(Constant.min_time(d0s2)) + "," +
+ resultSet.getString(Constant.min_time(d0s3)) + "," +
+ resultSet.getString(Constant.min_time(d0s4));
+ // System.out.println("============ " + ans);
+ Assert.assertEquals(ans, retArray[cnt]);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void maxTimeAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+ String[] retArray = new String[]{
+ "0,3999,3999,3999,3599,100"
+ };
+
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select max_time(s0),max_time(s1),max_time(s2),"
+ + "max_time(s3),max_time(s4) from root.vehicle.d0 " +
+ "where s1 < 50000 and s1 != 100");
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_time(d0s0))
+ + "," + resultSet.getString(max_time(d0s1)) + "," + resultSet.getString(max_time(d0s2))
+ + "," + resultSet.getString(max_time(d0s3)) + "," + resultSet.getString(max_time(d0s4));
+ //System.out.println("============ " + ans);
+ Assert.assertEquals(ans, retArray[cnt]);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void minValueAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+ String[] retArray = new String[]{
+ "0,0,0,0.0,B,true"
+ };
+
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select min_value(s0),min_value(s1),min_value(s2),"
+ + "min_value(s3),min_value(s4) from root.vehicle.d0 " +
+ "where s1 < 50000 and s1 != 100");
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(min_value(d0s0))
+ + "," + resultSet.getString(min_value(d0s1)) + "," + resultSet
+ .getString(min_value(d0s2))
+ + "," + resultSet.getString(min_value(d0s3)) + "," + resultSet
+ .getString(min_value(d0s4));
+ //System.out.println("============ " + ans);
+ Assert.assertEquals(ans, retArray[cnt]);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private void maxValueAggreWithMultiFilterTest() throws ClassNotFoundException, SQLException {
+ String[] retArray = new String[]{
+ "0,99,40000,122.0,fffff,true"
+ };
+
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute("select max_value(s0),max_value(s1),max_value(s2),"
+ + "max_value(s3),max_value(s4) from root.vehicle.d0 where s1 < 50000 and s1 != 100");
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_value(d0s0))
+ + "," + resultSet.getString(max_value(d0s1)) + "," + resultSet
+ .getString(max_value(d0s2))
+ + "," + resultSet.getString(max_value(d0s3)) + "," + resultSet
+ .getString(max_value(d0s4));
+ Assert.assertEquals(ans, retArray[cnt]);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ private static String[] stringValue = new String[]{"A", "B", "C", "D", "E"};
+
+ public static void insertSQL() throws ClassNotFoundException, SQLException {
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ Connection connection = null;
+ double d0s0sum = 0.0, d0s1sum = 0.0, d0s2sum = 0.0;
+ int cnt = 0;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ for (String sql : createSql) {
+ statement.execute(sql);
+ }
+
+ // insert large amount of data
+ for (int time = 3000; time < 3600; time++) {
+ if (time % 5 == 0) {
+ continue;
+ }
+
+ String sql = String.format("insert into root.vehicle.d0(timestamp,s0) values(%s,%s)",
+ time, time % 100);
+ statement.execute(sql);
+ sql = String.format("insert into root.vehicle.d0(timestamp,s1) values(%s,%s)",
+ time, time % 17);
+ statement.execute(sql);
+ sql = String.format("insert into root.vehicle.d0(timestamp,s2) values(%s,%s)",
+ time, time % 22);
+ statement.execute(sql);
+ sql = String.format("insert into root.vehicle.d0(timestamp,s3) values(%s,'%s')",
+ time, stringValue[time % 5]);
+ statement.execute(sql);
+ cnt++;
+ d0s0sum += time % 100;
+ d0s1sum += time % 17;
+ d0s2sum += time % 22;
+ }
+
+ statement.execute("flush");
+ System.out.println("cnt = " + cnt);
+
+ // insert large amount of data
+ for (int time = 3700; time < 4000; time++) {
+ if (time % 6 == 0) {
+ continue;
+ }
+
+ String sql = String.format("insert into root.vehicle.d0(timestamp,s0) values(%s,%s)",
+ time, time % 70);
+ statement.execute(sql);
+ sql = String.format("insert into root.vehicle.d0(timestamp,s1) values(%s,%s)",
+ time, time % 40);
+ statement.execute(sql);
+ sql = String.format("insert into root.vehicle.d0(timestamp,s2) values(%s,%s)",
+ time, time % 123);
+ statement.execute(sql);
+
+ cnt++;
+ d0s0sum += time % 70;
+ d0s1sum += time % 40;
+ d0s2sum += time % 123;
+ }
+
+ statement.execute("merge");
+
+ System.out.println("large insert cnt = " + cnt);
+ System.out
+ .println("d0s0sum = " + d0s0sum + "; d0s1sum = " + d0s1sum + "; d0s2sum = " + d0s2sum);
+ for (String sql : insertSql) {
+ statement.execute(sql);
+ }
+
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+}
diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationSmallDataIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationSmallDataIT.java
new file mode 100644
index 0000000..b9608e0
--- /dev/null
+++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAggregationSmallDataIT.java
@@ -0,0 +1,757 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.integration;
+
+import static org.apache.iotdb.db.integration.Constant.count;
+import static org.apache.iotdb.db.integration.Constant.first;
+import static org.apache.iotdb.db.integration.Constant.last;
+import static org.apache.iotdb.db.integration.Constant.max_time;
+import static org.apache.iotdb.db.integration.Constant.max_value;
+import static org.apache.iotdb.db.integration.Constant.mean;
+import static org.apache.iotdb.db.integration.Constant.min_time;
+import static org.apache.iotdb.db.integration.Constant.min_value;
+import static org.apache.iotdb.db.integration.Constant.sum;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.apache.iotdb.db.service.IoTDB;
+import org.apache.iotdb.db.utils.EnvironmentUtils;
+import org.apache.iotdb.jdbc.Config;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Multiple aggregation with filter test.
+ */
+public class IoTDBAggregationSmallDataIT {
+
+ private static final String TIMESTAMP_STR = "Time";
+ private final String d0s0 = "root.vehicle.d0.s0";
+ private final String d0s1 = "root.vehicle.d0.s1";
+ private final String d0s2 = "root.vehicle.d0.s2";
+ private final String d0s3 = "root.vehicle.d0.s3";
+ private final String d0s4 = "root.vehicle.d0.s4";
+ private final String d1s0 = "root.vehicle.d1.s0";
+ private final String d1s1 = "root.vehicle.d1.s1";
+
+ private static String[] sqls = new String[]{
+ "SET STORAGE GROUP TO root.vehicle",
+ "CREATE TIMESERIES root.vehicle.d1.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d1.s1 WITH DATATYPE=INT64, ENCODING=RLE",
+
+ "CREATE TIMESERIES root.vehicle.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d0.s1 WITH DATATYPE=INT64, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d0.s2 WITH DATATYPE=FLOAT, ENCODING=RLE",
+ "CREATE TIMESERIES root.vehicle.d0.s3 WITH DATATYPE=TEXT, ENCODING=PLAIN",
+ "CREATE TIMESERIES root.vehicle.d0.s4 WITH DATATYPE=BOOLEAN, ENCODING=PLAIN",
+
+ "insert into root.vehicle.d0(timestamp,s0) values(1,101)",
+ "insert into root.vehicle.d0(timestamp,s0) values(2,198)",
+ "insert into root.vehicle.d0(timestamp,s0) values(100,99)",
+ "insert into root.vehicle.d0(timestamp,s0) values(101,99)",
+ "insert into root.vehicle.d0(timestamp,s0) values(102,80)",
+ "insert into root.vehicle.d0(timestamp,s0) values(103,99)",
+ "insert into root.vehicle.d0(timestamp,s0) values(104,90)",
+ "insert into root.vehicle.d0(timestamp,s0) values(105,99)",
+ "insert into root.vehicle.d0(timestamp,s0) values(106,99)",
+ "insert into root.vehicle.d0(timestamp,s0) values(2,10000)",
+ "insert into root.vehicle.d0(timestamp,s0) values(50,10000)",
+ "insert into root.vehicle.d0(timestamp,s0) values(1000,22222)",
+ "insert into root.vehicle.d0(timestamp,s0) values(106,199)",
+ "DELETE FROM root.vehicle.d0.s0 WHERE time < 104",
+
+ "insert into root.vehicle.d0(timestamp,s1) values(1,1101)",
+ "insert into root.vehicle.d0(timestamp,s1) values(2,198)",
+ "insert into root.vehicle.d0(timestamp,s1) values(100,199)",
+ "insert into root.vehicle.d0(timestamp,s1) values(101,199)",
+ "insert into root.vehicle.d0(timestamp,s1) values(102,180)",
+ "insert into root.vehicle.d0(timestamp,s1) values(103,199)",
+ "insert into root.vehicle.d0(timestamp,s1) values(104,190)",
+ "insert into root.vehicle.d0(timestamp,s1) values(105,199)",
+ "insert into root.vehicle.d0(timestamp,s1) values(2,40000)",
+ "insert into root.vehicle.d0(timestamp,s1) values(50,50000)",
+ "insert into root.vehicle.d0(timestamp,s1) values(1000,55555)",
+
+ "insert into root.vehicle.d0(timestamp,s2) values(1000,55555)",
+ "insert into root.vehicle.d0(timestamp,s2) values(2,2.22)",
+ "insert into root.vehicle.d0(timestamp,s2) values(3,3.33)",
+ "insert into root.vehicle.d0(timestamp,s2) values(4,4.44)",
+ "insert into root.vehicle.d0(timestamp,s2) values(102,10.00)",
+ "insert into root.vehicle.d0(timestamp,s2) values(105,11.11)",
+ "insert into root.vehicle.d0(timestamp,s2) values(1000,1000.11)",
+
+ "insert into root.vehicle.d0(timestamp,s3) values(60,'aaaaa')",
+ "insert into root.vehicle.d0(timestamp,s3) values(70,'bbbbb')",
+ "insert into root.vehicle.d0(timestamp,s3) values(80,'ccccc')",
+ "insert into root.vehicle.d0(timestamp,s3) values(101,'ddddd')",
+ "insert into root.vehicle.d0(timestamp,s3) values(102,'fffff')",
+
+ "insert into root.vehicle.d1(timestamp,s0) values(1,999)",
+ "insert into root.vehicle.d1(timestamp,s0) values(1000,888)",
+
+ "insert into root.vehicle.d0(timestamp,s1) values(2000-01-01T08:00:00+08:00, 100)",
+ "insert into root.vehicle.d0(timestamp,s3) values(2000-01-01T08:00:00+08:00, 'good')",
+
+ "insert into root.vehicle.d0(timestamp,s4) values(100, false)",
+ "insert into root.vehicle.d0(timestamp,s4) values(100, true)"
+ };
+
+ private IoTDB daemon;
+
+ @Before
+ public void setUp() throws Exception {
+ EnvironmentUtils.closeStatMonitor();
+ EnvironmentUtils.closeMemControl();
+ daemon = IoTDB.getInstance();
+ daemon.active();
+ EnvironmentUtils.envSetUp();
+
+ Thread.sleep(5000);
+ insertSQL();
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ daemon.stop();
+ EnvironmentUtils.cleanEnv();
+ }
+
+ @Test
+ public void countOnlyTimeFilterTest() throws ClassNotFoundException, SQLException {
+ String[] retArray = new String[]{
+ "0,3,7,4,5,1"
+ };
+
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute(
+ "select count(s0),count(s1),count(s2),count(s3),count(s4) " +
+ "from root.vehicle.d0 where time >= 3 and time <= 106");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(count(d0s0))
+ + "," + resultSet.getString(count(d0s1)) + "," + resultSet.getString(count(d0s2))
+ + "," + resultSet.getString(count(d0s3)) + "," + resultSet.getString(count(d0s4));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ } finally {
+ if (connection != null) {
+ connection.close();
+ }
+ }
+ }
+
+ @Test
+ public void functionsNoFilterTest() throws ClassNotFoundException, SQLException {
+ String[] retArray = new String[]{
+ "0,4,0,6,1",
+ "0,22222,null,good",
+ "0,90,null,aaaaa",
+ "0,22222,null,good",
+ "0,22610.0,0.0"
+ };
+
+ Class.forName(Config.JDBC_DRIVER_NAME);
+ Connection connection = null;
+ try {
+ connection = DriverManager.
+ getConnection("jdbc:iotdb://127.0.0.1:6667/", "root", "root");
+
+ //select count(d0.s0),count(d1.s1),count(d0.s3),count(d0.s4) from root.vehicle
+ Statement statement = connection.createStatement();
+ boolean hasResultSet = statement.execute(
+ "select count(d0.s0),count(d1.s1),count(d0.s3),count(d0.s4) from root.vehicle");
+
+ Assert.assertTrue(hasResultSet);
+ ResultSet resultSet = statement.getResultSet();
+ int cnt = 0;
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(count(d0s0))
+ + "," + resultSet.getString(count(d1s1)) + "," + resultSet.getString(count(d0s3))
+ + "," + resultSet.getString(count(d0s4));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(1, cnt);
+ statement.close();
+
+ //select max_value(d0.s0),max_value(d1.s1),max_value(d0.s3) from root.vehicle
+ statement = connection.createStatement();
+ hasResultSet = statement.execute(
+ "select max_value(d0.s0),max_value(d1.s1),max_value(d0.s3) from root.vehicle");
+ resultSet = statement.getResultSet();
+ Assert.assertTrue(hasResultSet);
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(max_value(d0s0))
+ + "," + resultSet.getString(max_value(d1s1)) + ","
+ + resultSet.getString(max_value(d0s3));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(2, cnt);
+ statement.close();
+
+ //select first(d0.s0),first(d1.s1),first(d0.s3) from root.vehicle
+ statement = connection.createStatement();
+ hasResultSet = statement.execute(
+ "select first(d0.s0),first(d1.s1),first(d0.s3) from root.vehicle");
+ resultSet = statement.getResultSet();
+ Assert.assertTrue(hasResultSet);
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(first(d0s0))
+ + "," + resultSet.getString(first(d1s1)) + "," + resultSet.getString(first(d0s3));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(3, cnt);
+ statement.close();
+
+ //select last(d0.s0),last(d1.s1),last(d0.s3) from root.vehicle
+ statement = connection.createStatement();
+ hasResultSet = statement.execute(
+ "select last(d0.s0),last(d1.s1),last(d0.s3) from root.vehicle");
+ resultSet = statement.getResultSet();
+ Assert.assertTrue(hasResultSet);
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(last(d0s0))
+ + "," + resultSet.getString(last(d1s1)) + "," + resultSet.getString(last(d0s3));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(4, cnt);
+ statement.close();
+
+ //select sum(d0.s0),sum(d1.s1),sum(d0.s3) from root.vehicle
+ statement = connection.createStatement();
+ hasResultSet = statement.execute("select sum(d0.s0),sum(d1.s1) from root.vehicle");
+ resultSet = statement.getResultSet();
+ Assert.assertTrue(hasResultSet);
+ while (resultSet.next()) {
+ String ans = resultSet.getString(TIMESTAMP_STR) + "," + resultSet.getString(sum(d0s0))
+ + "," + resultSet.getString(sum(d1s1));
+ Assert.assertEquals(retArray[cnt], ans);
+ cnt++;
+ }
+ Assert.assertEquals(5, cnt);
+ statement.close();
+
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
... 2932 lines suppressed ...