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

[GitHub] [incubator-iotdb] liutaohua opened a new pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

liutaohua opened a new pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394084037
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/externalsort/adapter/ByTimestampReaderAdapter.java
 ##########
 @@ -40,14 +41,18 @@ public ByTimestampReaderAdapter(IPointReader pointReader) {
   }
 
   @Override
-  public Object[] getValuesInTimestamps(long[] timestamps) throws IOException {
-    Object[] result = new Object[timestamps.length];
+  public Object[] getValuesInTimestamps(TimeColumn timestamps, long bound) throws IOException {
+    Object[] result = new Object[timestamps.size()];
 
 Review comment:
   if the result use size, do we need the bound? These two are conflict and puzzle me

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394313019
 
 

 ##########
 File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/node/OrNode.java
 ##########
 @@ -55,17 +55,21 @@ public boolean hasNextTimeColumn() throws IOException {
     }
 
     if (hasLeftValue() && !hasRightValue()) {
-      cachedTimeColumn = leftTimeColumn;
+      while (leftTimeColumn.hasCurrent()) {
 
 Review comment:
   To do this, we need to be careful where `timecolumn` is used in the system, because the `timecolumn` returned may not start at 0

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394097986
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java
 ##########
 @@ -45,14 +46,19 @@ public SeriesReaderByTimestamp(SeriesReader seriesReader) {
   }
 
   @Override
-  public Object[] getValuesInTimestamps(long[] timestamps) throws IOException {
-    Object[] result = new Object[timestamps.length];
+  public Object[] getValuesInTimestamps(TimeColumn timestamps, long bound) throws IOException {
 
 Review comment:
   Not sure. maybe in `group by` or the `leftTimeColumn`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] samperson1997 commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
samperson1997 commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394077348
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
 ##########
 @@ -73,26 +86,56 @@ protected RowRecord nextWithoutConstraint() throws IOException {
    * @return if there has next row record.
    */
   private boolean cacheRowRecord() throws IOException {
-    while (timeGenerator.hasNext()) {
-      boolean hasField = false;
-      long timestamp = timeGenerator.next();
-      RowRecord rowRecord = new RowRecord(timestamp);
-      for (int i = 0; i < seriesReaderByTimestampList.size(); i++) {
-        IReaderByTimestamp reader = seriesReaderByTimestampList.get(i);
-        Object value = reader.getValueInTimestamp(timestamp);
-        if (value == null) {
-          rowRecord.addField(null);
-        } else {
-          hasField = true;
-          rowRecord.addField(value, dataTypes.get(i));
+    int seriesNum = seriesReaderByTimestampList.size();
+    while (timeGenerator.hasNextTimeColumn()) {
+      final TimeColumn timeColumn = timeGenerator.nextTimeColumn();
+
+      Future<List<Field>>[] futures = new Future[seriesNum];
+      for (int i = 0; i < seriesNum; i++) {
+        final IReaderByTimestamp readerByTimestamp = seriesReaderByTimestampList.get(i);
+        final TSDataType tsDataType = dataTypes.get(i);
+        futures[i] = QueryTaskPoolManager.getInstance().submit(() -> {
+          List<Field> fields = new ArrayList<>();
+          Object[] values = readerByTimestamp
+              .getValuesInTimestamps(timeColumn.duplicate(), Long.MAX_VALUE);
+          for (Object value : values) {
+            if (value == null) {
+              fields.add(null);
+            } else {
+              fields.add(Field.getField(value, tsDataType));
+            }
+          }
+          values = null;
 
 Review comment:
   This `values` is not used, maybe it could be removed?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394097616
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java
 ##########
 @@ -267,22 +271,17 @@ private void aggregateWithValueFilter(List<AggregateResult> aggregateResults,
       TimeGenerator timestampGenerator, List<IReaderByTimestamp> 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++) {
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] samperson1997 commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
samperson1997 commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394076866
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
 ##########
 @@ -19,21 +19,32 @@
 package org.apache.iotdb.db.query.dataset;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
 
 Review comment:
   useless import

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394298900
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/externalsort/adapter/ByTimestampReaderAdapter.java
 ##########
 @@ -40,14 +41,18 @@ public ByTimestampReaderAdapter(IPointReader pointReader) {
   }
 
   @Override
-  public Object[] getValuesInTimestamps(long[] timestamps) throws IOException {
-    Object[] result = new Object[timestamps.length];
+  public Object[] getValuesInTimestamps(TimeColumn timestamps, long bound) throws IOException {
+    Object[] result = new Object[timestamps.size()];
 
 Review comment:
   `bound` represents the stop time, but we don't know how much data there is between the current index and bound time, and how much memory this Object[] should alloc

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394097117
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxValueAggrResult.java
 ##########
 @@ -66,16 +67,17 @@ public void updateResultFromPageData(BatchData dataInThisPage, long bound) {
   }
 
   @Override
-  public void updateResultUsingTimestamps(long[] timestamps, int length,
-      IReaderByTimestamp dataReader) throws IOException {
+  public void updateResultUsingTimestamps(TimeColumn timestamps, long bound,
+      IReaderByTimestamp dataReader)
+      throws IOException {
     Comparable<Object> maxVal = null;
-    for (int i = 0; i < length; i++) {
-      Object value = dataReader.getValueInTimestamp(timestamps[i]);
-      if (value == null) {
+    Object[] value = dataReader.getValuesInTimestamps(timestamps, bound);
+    for (int i = value.length - 1; i >= 0; i--) {
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394082454
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java
 ##########
 @@ -45,14 +46,19 @@ public SeriesReaderByTimestamp(SeriesReader seriesReader) {
   }
 
   @Override
-  public Object[] getValuesInTimestamps(long[] timestamps) throws IOException {
-    Object[] result = new Object[timestamps.length];
+  public Object[] getValuesInTimestamps(TimeColumn timestamps, long bound) throws IOException {
 
 Review comment:
   could the TimeColumn not start from 0?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] samperson1997 commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
samperson1997 commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394077614
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java
 ##########
 @@ -37,30 +37,21 @@
 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.common.TimeColumn;
 import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGenerator;
 
 public class GroupByWithValueFilterDataSet extends GroupByEngineDataSet {
 
   private List<IReaderByTimestamp> allDataReaderList;
   private GroupByPlan groupByPlan;
   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;
 
+  private TimeColumn timeColumn = new TimeColumn();
   /**
    * group by batch calculation size.
    */
   protected int timeStampFetchSize;
 
 Review comment:
   can be private. And also other methods in this class

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394300034
 
 

 ##########
 File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/dataset/DataSetWithTimeGenerator.java
 ##########
 @@ -56,12 +60,33 @@ public DataSetWithTimeGenerator(List<Path> paths, List<Boolean> cached,
 
   @Override
   protected boolean hasNextWithoutConstraint() throws IOException {
-    return timeGenerator.hasNext();
+    if (hasCache) {
+      return true;
+    }
+
+    if (timeColumn != null && timeColumn.hasCurrent()) {
+      cacheTime = timeColumn.currentTime();
+      timeColumn.next();
+      hasCache = true;
+      return true;
+    }
+
+    while (timeGenerator.hasNextTimeColumn()) {
+      timeColumn = timeGenerator.nextTimeColumn();
+      if (timeColumn.hasCurrent()) {
+        hasCache = true;
+        cacheTime = timeColumn.currentTime();
+        timeColumn.next();
+        break;
+      }
+    }
+    return hasCache;
   }
 
   @Override
   protected RowRecord nextWithoutConstraint() throws IOException {
-    long timestamp = timeGenerator.next();
+    hasCache = false;
+    long timestamp = cacheTime;
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394097147
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
 ##########
 @@ -19,21 +19,32 @@
 package org.apache.iotdb.db.query.dataset;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394076644
 
 

 ##########
 File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/timegenerator/node/OrNode.java
 ##########
 @@ -55,17 +55,21 @@ public boolean hasNextTimeColumn() throws IOException {
     }
 
     if (hasLeftValue() && !hasRightValue()) {
-      cachedTimeColumn = leftTimeColumn;
+      while (leftTimeColumn.hasCurrent()) {
 
 Review comment:
   return the leftTimeColumn and set null

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394770893
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
 ##########
 @@ -73,26 +77,62 @@ protected RowRecord nextWithoutConstraint() throws IOException {
    * @return if there has next row record.
    */
   private boolean cacheRowRecord() throws IOException {
-    while (timeGenerator.hasNext()) {
+    Object[][] results = new Object[seriesReaderByTimestampList.size()][];
+    while (timeGenerator.hasNextTimeColumn()) {
+      initResults(results);
+
+      TimeColumn timeColumn = timeGenerator.nextTimeColumn();
+
+      fillResults(results, timeColumn);
+      flatAndCache(results, timeColumn);
+    }
+    hasCachedRowRecord = !cachedRecords.isEmpty();
+    return hasCachedRowRecord;
+  }
+
+  private void flatAndCache(Object[][] results, TimeColumn timeColumn) {
+    int resultSize = timeColumn.size() - timeColumn.position();
+    for (int i = 0; i < resultSize; i++) {
+      RowRecord rowRecord = new RowRecord(timeColumn.getTimeByIndex(i + timeColumn.position()));
       boolean hasField = false;
-      long timestamp = timeGenerator.next();
-      RowRecord rowRecord = new RowRecord(timestamp);
-      for (int i = 0; i < seriesReaderByTimestampList.size(); i++) {
-        IReaderByTimestamp reader = seriesReaderByTimestampList.get(i);
-        Object value = reader.getValueInTimestamp(timestamp);
-        if (value == null) {
+      for (Object[] result : results) {
+        if (result[i] == null) {
           rowRecord.addField(null);
-        } else {
-          hasField = true;
-          rowRecord.addField(value, dataTypes.get(i));
+          continue;
         }
+        hasField = true;
+        rowRecord.addField((Field) result[i]);
       }
       if (hasField) {
-        hasCachedRowRecord = true;
-        cachedRowRecord = rowRecord;
-        break;
+        cachedRecords.add(rowRecord);
       }
     }
-    return hasCachedRowRecord;
+  }
+
+  private void fillResults(Object[][] results, TimeColumn timeColumn) throws IOException {
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394771663
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
 ##########
 @@ -73,26 +77,62 @@ protected RowRecord nextWithoutConstraint() throws IOException {
    * @return if there has next row record.
    */
   private boolean cacheRowRecord() throws IOException {
-    while (timeGenerator.hasNext()) {
+    Object[][] results = new Object[seriesReaderByTimestampList.size()][];
+    while (timeGenerator.hasNextTimeColumn()) {
+      initResults(results);
+
+      TimeColumn timeColumn = timeGenerator.nextTimeColumn();
+
+      fillResults(results, timeColumn);
+      flatAndCache(results, timeColumn);
+    }
+    hasCachedRowRecord = !cachedRecords.isEmpty();
+    return hasCachedRowRecord;
+  }
+
+  private void flatAndCache(Object[][] results, TimeColumn timeColumn) {
+    int resultSize = timeColumn.size() - timeColumn.position();
+    for (int i = 0; i < resultSize; i++) {
+      RowRecord rowRecord = new RowRecord(timeColumn.getTimeByIndex(i + timeColumn.position()));
 
 Review comment:
   flatAndCache was combined

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394092880
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java
 ##########
 @@ -116,67 +108,40 @@ protected RowRecord nextWithoutConstraint() throws IOException {
           groupByPlan.getDeduplicatedDataTypes().get(i)));
     }
 
-    long[] timestampArray = new long[timeStampFetchSize];
-    int timeArrayLength = 0;
-    if (hasCachedTimestamp) {
-      if (timestamp < curEndTime) {
-        if (timestamp >= curStartTime) {
-          hasCachedTimestamp = false;
-          timestampArray[timeArrayLength++] = timestamp;
-        }
-      } else {
+    if (timeColumn != null && timeColumn.hasCurrent()) {
+      //skip early time
+      while (timeColumn.currentTime() < curStartTime && timeColumn.hasCurrent()) {
+        timeColumn.next();
+      }
+      if (timeColumn.currentTime() >= curEndTime) {
         return constructRowRecord(aggregateResultList);
       }
     }
-    while (timestampGenerator.hasNext()) {
-      // construct timestamp array
-      timeArrayLength = constructTimeArrayForOneCal(timestampArray, timeArrayLength);
 
+    while (timestampGenerator.hasNextTimeColumn() || timeColumn.hasCurrent()) {
+      if (timeColumn == null || !timeColumn.hasCurrent()) {
+        timeColumn = timestampGenerator.nextTimeColumn();
+        if (timeColumn.currentTime() >= curEndTime) {
+          break;
+        }
 
 Review comment:
   this could be put out of the if, then the 137 line could be deleted

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394087691
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java
 ##########
 @@ -267,22 +271,17 @@ private void aggregateWithValueFilter(List<AggregateResult> aggregateResults,
       TimeGenerator timestampGenerator, List<IReaderByTimestamp> 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();
-      }
-
+    while (timestampGenerator.hasNextTimeColumn()) {
+      TimeColumn timeColumn = timestampGenerator.nextTimeColumn();
       // cal part of aggregate result
+      int index = timeColumn.currentIndex();
       for (int i = 0; i < readersOfSelectedSeries.size(); i++) {
-        aggregateResults.get(i).updateResultUsingTimestamps(timeArray, timeArrayLength,
-            readersOfSelectedSeries.get(i));
+        AggregateResult result = aggregateResults.get(i);
+        if (!result.isCalculatedAggregationResult()) {
+          timeColumn.resetIndex(index);
+          result.updateResultUsingTimestamps(timeColumn, Integer.MAX_VALUE,
 
 Review comment:
   ```suggestion
             result.updateResultUsingTimestamps(timeColumn, Long.MAX_VALUE,
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394348913
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
 ##########
 @@ -73,26 +86,56 @@ protected RowRecord nextWithoutConstraint() throws IOException {
    * @return if there has next row record.
    */
   private boolean cacheRowRecord() throws IOException {
-    while (timeGenerator.hasNext()) {
-      boolean hasField = false;
-      long timestamp = timeGenerator.next();
-      RowRecord rowRecord = new RowRecord(timestamp);
-      for (int i = 0; i < seriesReaderByTimestampList.size(); i++) {
-        IReaderByTimestamp reader = seriesReaderByTimestampList.get(i);
-        Object value = reader.getValueInTimestamp(timestamp);
-        if (value == null) {
-          rowRecord.addField(null);
-        } else {
-          hasField = true;
-          rowRecord.addField(value, dataTypes.get(i));
+    int seriesNum = seriesReaderByTimestampList.size();
+    while (timeGenerator.hasNextTimeColumn()) {
+      final TimeColumn timeColumn = timeGenerator.nextTimeColumn();
+
+      Future<List<Field>>[] futures = new Future[seriesNum];
+      for (int i = 0; i < seriesNum; i++) {
+        final IReaderByTimestamp readerByTimestamp = seriesReaderByTimestampList.get(i);
+        final TSDataType tsDataType = dataTypes.get(i);
+        futures[i] = QueryTaskPoolManager.getInstance().submit(() -> {
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394083613
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/chunk/DiskChunkReaderByTimestamp.java
 ##########
 @@ -40,21 +41,25 @@ public DiskChunkReaderByTimestamp(ChunkReaderByTimestamp chunkReaderByTimestamp)
   }
 
   @Override
-  public Object[] getValuesInTimestamps(long[] timestamps) throws IOException {
-    Object[] result = new Object[timestamps.length];
+  public Pair[] getValuesInTimestamps(TimeColumn timestamps, long bound) throws IOException {
+    Pair[] result = new Pair[timestamps.size()];
 
 Review comment:
   the result may not fully be filled, this function may cause bug

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394297040
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java
 ##########
 @@ -267,22 +271,17 @@ private void aggregateWithValueFilter(List<AggregateResult> aggregateResults,
       TimeGenerator timestampGenerator, List<IReaderByTimestamp> 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();
-      }
-
+    while (timestampGenerator.hasNextTimeColumn()) {
+      TimeColumn timeColumn = timestampGenerator.nextTimeColumn();
       // cal part of aggregate result
+      int index = timeColumn.currentIndex();
 
 Review comment:
   `LeftTimeColumn` is returned directly, so the index of `timecolumn` does not necessarily start at 0

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394095224
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
 ##########
 @@ -73,26 +86,56 @@ protected RowRecord nextWithoutConstraint() throws IOException {
    * @return if there has next row record.
    */
   private boolean cacheRowRecord() throws IOException {
-    while (timeGenerator.hasNext()) {
-      boolean hasField = false;
-      long timestamp = timeGenerator.next();
-      RowRecord rowRecord = new RowRecord(timestamp);
-      for (int i = 0; i < seriesReaderByTimestampList.size(); i++) {
-        IReaderByTimestamp reader = seriesReaderByTimestampList.get(i);
-        Object value = reader.getValueInTimestamp(timestamp);
-        if (value == null) {
-          rowRecord.addField(null);
-        } else {
-          hasField = true;
-          rowRecord.addField(value, dataTypes.get(i));
+    int seriesNum = seriesReaderByTimestampList.size();
+    while (timeGenerator.hasNextTimeColumn()) {
+      final TimeColumn timeColumn = timeGenerator.nextTimeColumn();
+
+      Future<List<Field>>[] futures = new Future[seriesNum];
+      for (int i = 0; i < seriesNum; i++) {
+        final IReaderByTimestamp readerByTimestamp = seriesReaderByTimestampList.get(i);
+        final TSDataType tsDataType = dataTypes.get(i);
+        futures[i] = QueryTaskPoolManager.getInstance().submit(() -> {
 
 Review comment:
   remove parallelization, do this in a separate PR and do some experiments to help us decide whether introduce parallelization

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394915809
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java
 ##########
 @@ -91,13 +75,13 @@ protected void initGroupBy(QueryContext context, GroupByPlan groupByPlan)
     }
   }
 
-  protected TimeGenerator getTimeGenerator(IExpression expression, QueryContext context)
+  private TimeGenerator getTimeGenerator(IExpression expression, QueryContext context)
 
 Review comment:
   protected

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394761720
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
 ##########
 @@ -73,26 +77,62 @@ protected RowRecord nextWithoutConstraint() throws IOException {
    * @return if there has next row record.
    */
   private boolean cacheRowRecord() throws IOException {
-    while (timeGenerator.hasNext()) {
+    Object[][] results = new Object[seriesReaderByTimestampList.size()][];
+    while (timeGenerator.hasNextTimeColumn()) {
+      initResults(results);
+
+      TimeColumn timeColumn = timeGenerator.nextTimeColumn();
+
+      fillResults(results, timeColumn);
+      flatAndCache(results, timeColumn);
+    }
+    hasCachedRowRecord = !cachedRecords.isEmpty();
+    return hasCachedRowRecord;
+  }
+
+  private void flatAndCache(Object[][] results, TimeColumn timeColumn) {
+    int resultSize = timeColumn.size() - timeColumn.position();
+    for (int i = 0; i < resultSize; i++) {
+      RowRecord rowRecord = new RowRecord(timeColumn.getTimeByIndex(i + timeColumn.position()));
       boolean hasField = false;
-      long timestamp = timeGenerator.next();
-      RowRecord rowRecord = new RowRecord(timestamp);
-      for (int i = 0; i < seriesReaderByTimestampList.size(); i++) {
-        IReaderByTimestamp reader = seriesReaderByTimestampList.get(i);
-        Object value = reader.getValueInTimestamp(timestamp);
-        if (value == null) {
+      for (Object[] result : results) {
+        if (result[i] == null) {
           rowRecord.addField(null);
-        } else {
-          hasField = true;
-          rowRecord.addField(value, dataTypes.get(i));
+          continue;
         }
+        hasField = true;
+        rowRecord.addField((Field) result[i]);
       }
       if (hasField) {
-        hasCachedRowRecord = true;
-        cachedRowRecord = rowRecord;
-        break;
+        cachedRecords.add(rowRecord);
       }
     }
-    return hasCachedRowRecord;
+  }
+
+  private void fillResults(Object[][] results, TimeColumn timeColumn) throws IOException {
 
 Review comment:
   combine the fillResults, initReulsts, flatAndCache, cacheRowRecord into hasNextWithoutConstraints

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394098349
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/chunk/DiskChunkReaderByTimestamp.java
 ##########
 @@ -40,21 +41,25 @@ public DiskChunkReaderByTimestamp(ChunkReaderByTimestamp chunkReaderByTimestamp)
   }
 
   @Override
-  public Object[] getValuesInTimestamps(long[] timestamps) throws IOException {
-    Object[] result = new Object[timestamps.length];
+  public Pair[] getValuesInTimestamps(TimeColumn timestamps, long bound) throws IOException {
+    Pair[] result = new Pair[timestamps.size()];
 
-    for (int i = 0; i < timestamps.length; i++) {
-      if (timestamps[i] < currentTime) {
+    for (int i = 0; i < timestamps.size(); i++) {
+      if (timestamps.currentTime() < currentTime) {
         throw new IOException("time must be increasing when use ReaderByTimestamp");
       }
-      currentTime = timestamps[i];
+      if (timestamps.currentTime() >= bound) {
+        return result;
+      }
+      currentTime = timestamps.currentTime();
+      timestamps.next();
       while (hasNext()) {
         data = next();
         if (data.getMaxTimestamp() > currentTime) {
           result[i] = null;
           break;
         }
-        result[i] = data.getValueInTimestamp(currentTime);
+        result[i] = new Pair(currentTime, data.getValueInTimestamp(currentTime));
 
 Review comment:
   sorry , forgot  
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394754500
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
 ##########
 @@ -73,26 +77,62 @@ protected RowRecord nextWithoutConstraint() throws IOException {
    * @return if there has next row record.
    */
   private boolean cacheRowRecord() throws IOException {
-    while (timeGenerator.hasNext()) {
+    Object[][] results = new Object[seriesReaderByTimestampList.size()][];
+    while (timeGenerator.hasNextTimeColumn()) {
+      initResults(results);
+
+      TimeColumn timeColumn = timeGenerator.nextTimeColumn();
+
+      fillResults(results, timeColumn);
+      flatAndCache(results, timeColumn);
+    }
+    hasCachedRowRecord = !cachedRecords.isEmpty();
+    return hasCachedRowRecord;
+  }
+
+  private void flatAndCache(Object[][] results, TimeColumn timeColumn) {
+    int resultSize = timeColumn.size() - timeColumn.position();
+    for (int i = 0; i < resultSize; i++) {
+      RowRecord rowRecord = new RowRecord(timeColumn.getTimeByIndex(i + timeColumn.position()));
 
 Review comment:
   record the position before, avoid calculating each time

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394073174
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxValueAggrResult.java
 ##########
 @@ -66,16 +67,17 @@ public void updateResultFromPageData(BatchData dataInThisPage, long bound) {
   }
 
   @Override
-  public void updateResultUsingTimestamps(long[] timestamps, int length,
-      IReaderByTimestamp dataReader) throws IOException {
+  public void updateResultUsingTimestamps(TimeColumn timestamps, long bound,
+      IReaderByTimestamp dataReader)
+      throws IOException {
     Comparable<Object> maxVal = null;
-    for (int i = 0; i < length; i++) {
-      Object value = dataReader.getValueInTimestamp(timestamps[i]);
-      if (value == null) {
+    Object[] value = dataReader.getValuesInTimestamps(timestamps, bound);
+    for (int i = value.length - 1; i >= 0; i--) {
 
 Review comment:
   ```suggestion
       for (Object value : values) {
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394083355
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/chunk/DiskChunkReaderByTimestamp.java
 ##########
 @@ -40,21 +41,25 @@ public DiskChunkReaderByTimestamp(ChunkReaderByTimestamp chunkReaderByTimestamp)
   }
 
   @Override
-  public Object[] getValuesInTimestamps(long[] timestamps) throws IOException {
-    Object[] result = new Object[timestamps.length];
+  public Pair[] getValuesInTimestamps(TimeColumn timestamps, long bound) throws IOException {
+    Pair[] result = new Pair[timestamps.size()];
 
-    for (int i = 0; i < timestamps.length; i++) {
-      if (timestamps[i] < currentTime) {
+    for (int i = 0; i < timestamps.size(); i++) {
+      if (timestamps.currentTime() < currentTime) {
         throw new IOException("time must be increasing when use ReaderByTimestamp");
       }
-      currentTime = timestamps[i];
+      if (timestamps.currentTime() >= bound) {
+        return result;
+      }
+      currentTime = timestamps.currentTime();
+      timestamps.next();
       while (hasNext()) {
         data = next();
         if (data.getMaxTimestamp() > currentTime) {
           result[i] = null;
           break;
         }
-        result[i] = data.getValueInTimestamp(currentTime);
+        result[i] = new Pair(currentTime, data.getValueInTimestamp(currentTime));
 
 Review comment:
   why return a pair?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394097139
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxValueAggrResult.java
 ##########
 @@ -66,16 +67,17 @@ public void updateResultFromPageData(BatchData dataInThisPage, long bound) {
   }
 
   @Override
-  public void updateResultUsingTimestamps(long[] timestamps, int length,
-      IReaderByTimestamp dataReader) throws IOException {
+  public void updateResultUsingTimestamps(TimeColumn timestamps, long bound,
+      IReaderByTimestamp dataReader)
+      throws IOException {
     Comparable<Object> maxVal = null;
-    for (int i = 0; i < length; i++) {
-      Object value = dataReader.getValueInTimestamp(timestamps[i]);
-      if (value == null) {
+    Object[] value = dataReader.getValuesInTimestamps(timestamps, bound);
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394097986
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java
 ##########
 @@ -45,14 +46,19 @@ public SeriesReaderByTimestamp(SeriesReader seriesReader) {
   }
 
   @Override
-  public Object[] getValuesInTimestamps(long[] timestamps) throws IOException {
-    Object[] result = new Object[timestamps.length];
+  public Object[] getValuesInTimestamps(TimeColumn timestamps, long bound) throws IOException {
 
 Review comment:
   Not sure. maybe in group by

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394754500
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
 ##########
 @@ -73,26 +77,62 @@ protected RowRecord nextWithoutConstraint() throws IOException {
    * @return if there has next row record.
    */
   private boolean cacheRowRecord() throws IOException {
-    while (timeGenerator.hasNext()) {
+    Object[][] results = new Object[seriesReaderByTimestampList.size()][];
+    while (timeGenerator.hasNextTimeColumn()) {
+      initResults(results);
+
+      TimeColumn timeColumn = timeGenerator.nextTimeColumn();
+
+      fillResults(results, timeColumn);
+      flatAndCache(results, timeColumn);
+    }
+    hasCachedRowRecord = !cachedRecords.isEmpty();
+    return hasCachedRowRecord;
+  }
+
+  private void flatAndCache(Object[][] results, TimeColumn timeColumn) {
+    int resultSize = timeColumn.size() - timeColumn.position();
+    for (int i = 0; i < resultSize; i++) {
+      RowRecord rowRecord = new RowRecord(timeColumn.getTimeByIndex(i + timeColumn.position()));
 
 Review comment:
   record the position before, avoid calculate each time

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394100153
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java
 ##########
 @@ -267,22 +271,17 @@ private void aggregateWithValueFilter(List<AggregateResult> aggregateResults,
       TimeGenerator timestampGenerator, List<IReaderByTimestamp> 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();
-      }
-
+    while (timestampGenerator.hasNextTimeColumn()) {
+      TimeColumn timeColumn = timestampGenerator.nextTimeColumn();
       // cal part of aggregate result
+      int index = timeColumn.currentIndex();
       for (int i = 0; i < readersOfSelectedSeries.size(); i++) {
-        aggregateResults.get(i).updateResultUsingTimestamps(timeArray, timeArrayLength,
-            readersOfSelectedSeries.get(i));
+        AggregateResult result = aggregateResults.get(i);
+        if (!result.isCalculatedAggregationResult()) {
+          timeColumn.resetIndex(index);
+          result.updateResultUsingTimestamps(timeColumn, Integer.MAX_VALUE,
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394303381
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java
 ##########
 @@ -116,67 +108,40 @@ protected RowRecord nextWithoutConstraint() throws IOException {
           groupByPlan.getDeduplicatedDataTypes().get(i)));
     }
 
-    long[] timestampArray = new long[timeStampFetchSize];
-    int timeArrayLength = 0;
-    if (hasCachedTimestamp) {
-      if (timestamp < curEndTime) {
-        if (timestamp >= curStartTime) {
-          hasCachedTimestamp = false;
-          timestampArray[timeArrayLength++] = timestamp;
-        }
-      } else {
+    if (timeColumn != null && timeColumn.hasCurrent()) {
+      //skip early time
+      while (timeColumn.currentTime() < curStartTime && timeColumn.hasCurrent()) {
+        timeColumn.next();
+      }
+      if (timeColumn.currentTime() >= curEndTime) {
         return constructRowRecord(aggregateResultList);
       }
     }
-    while (timestampGenerator.hasNext()) {
-      // construct timestamp array
-      timeArrayLength = constructTimeArrayForOneCal(timestampArray, timeArrayLength);
 
+    while (timestampGenerator.hasNextTimeColumn() || timeColumn.hasCurrent()) {
+      if (timeColumn == null || !timeColumn.hasCurrent()) {
+        timeColumn = timestampGenerator.nextTimeColumn();
+        if (timeColumn.currentTime() >= curEndTime) {
+          break;
+        }
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394073195
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/aggregation/impl/MaxValueAggrResult.java
 ##########
 @@ -66,16 +67,17 @@ public void updateResultFromPageData(BatchData dataInThisPage, long bound) {
   }
 
   @Override
-  public void updateResultUsingTimestamps(long[] timestamps, int length,
-      IReaderByTimestamp dataReader) throws IOException {
+  public void updateResultUsingTimestamps(TimeColumn timestamps, long bound,
+      IReaderByTimestamp dataReader)
+      throws IOException {
     Comparable<Object> maxVal = null;
-    for (int i = 0; i < length; i++) {
-      Object value = dataReader.getValueInTimestamp(timestamps[i]);
-      if (value == null) {
+    Object[] value = dataReader.getValuesInTimestamps(timestamps, bound);
 
 Review comment:
   ```suggestion
       Object[] values = dataReader.getValuesInTimestamps(timestamps, bound);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua closed pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua closed pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] samperson1997 commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
samperson1997 commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394078160
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java
 ##########
 @@ -267,22 +271,17 @@ private void aggregateWithValueFilter(List<AggregateResult> aggregateResults,
       TimeGenerator timestampGenerator, List<IReaderByTimestamp> 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++) {
 
 Review comment:
   Since `aggregateFetchSize` is not used, it could be deleted from this class.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394317297
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/chunk/DiskChunkReaderByTimestamp.java
 ##########
 @@ -40,21 +41,25 @@ public DiskChunkReaderByTimestamp(ChunkReaderByTimestamp chunkReaderByTimestamp)
   }
 
   @Override
-  public Object[] getValuesInTimestamps(long[] timestamps) throws IOException {
-    Object[] result = new Object[timestamps.length];
+  public Pair[] getValuesInTimestamps(TimeColumn timestamps, long bound) throws IOException {
+    Pair[] result = new Pair[timestamps.size()];
 
 Review comment:
   `values [I] == null `where this value is used, so I don't think there's going to be any bugs that you can imagine

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394297040
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java
 ##########
 @@ -267,22 +271,17 @@ private void aggregateWithValueFilter(List<AggregateResult> aggregateResults,
       TimeGenerator timestampGenerator, List<IReaderByTimestamp> 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();
-      }
-
+    while (timestampGenerator.hasNextTimeColumn()) {
+      TimeColumn timeColumn = timestampGenerator.nextTimeColumn();
       // cal part of aggregate result
+      int index = timeColumn.currentIndex();
 
 Review comment:
   yes, I rename the method to` position()`, looks more like the JDK provided,and here can be directly set to 0
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394097532
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java
 ##########
 @@ -73,26 +86,56 @@ protected RowRecord nextWithoutConstraint() throws IOException {
    * @return if there has next row record.
    */
   private boolean cacheRowRecord() throws IOException {
-    while (timeGenerator.hasNext()) {
-      boolean hasField = false;
-      long timestamp = timeGenerator.next();
-      RowRecord rowRecord = new RowRecord(timestamp);
-      for (int i = 0; i < seriesReaderByTimestampList.size(); i++) {
-        IReaderByTimestamp reader = seriesReaderByTimestampList.get(i);
-        Object value = reader.getValueInTimestamp(timestamp);
-        if (value == null) {
-          rowRecord.addField(null);
-        } else {
-          hasField = true;
-          rowRecord.addField(value, dataTypes.get(i));
+    int seriesNum = seriesReaderByTimestampList.size();
+    while (timeGenerator.hasNextTimeColumn()) {
+      final TimeColumn timeColumn = timeGenerator.nextTimeColumn();
+
+      Future<List<Field>>[] futures = new Future[seriesNum];
+      for (int i = 0; i < seriesNum; i++) {
+        final IReaderByTimestamp readerByTimestamp = seriesReaderByTimestampList.get(i);
+        final TSDataType tsDataType = dataTypes.get(i);
+        futures[i] = QueryTaskPoolManager.getInstance().submit(() -> {
+          List<Field> fields = new ArrayList<>();
+          Object[] values = readerByTimestamp
+              .getValuesInTimestamps(timeColumn.duplicate(), Long.MAX_VALUE);
+          for (Object value : values) {
+            if (value == null) {
+              fields.add(null);
+            } else {
+              fields.add(Field.getField(value, tsDataType));
+            }
+          }
+          values = null;
 
 Review comment:
   I think this is in order to reduce the pressure of GC, but it looks as if there is no escape. I've deleted it

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394097568
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java
 ##########
 @@ -37,30 +37,21 @@
 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.common.TimeColumn;
 import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGenerator;
 
 public class GroupByWithValueFilterDataSet extends GroupByEngineDataSet {
 
   private List<IReaderByTimestamp> allDataReaderList;
   private GroupByPlan groupByPlan;
   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;
 
+  private TimeColumn timeColumn = new TimeColumn();
   /**
    * group by batch calculation size.
    */
   protected int timeStampFetchSize;
 
 Review comment:
   done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394079645
 
 

 ##########
 File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/dataset/DataSetWithTimeGenerator.java
 ##########
 @@ -56,12 +60,33 @@ public DataSetWithTimeGenerator(List<Path> paths, List<Boolean> cached,
 
   @Override
   protected boolean hasNextWithoutConstraint() throws IOException {
-    return timeGenerator.hasNext();
+    if (hasCache) {
+      return true;
+    }
+
+    if (timeColumn != null && timeColumn.hasCurrent()) {
+      cacheTime = timeColumn.currentTime();
+      timeColumn.next();
+      hasCache = true;
+      return true;
+    }
+
+    while (timeGenerator.hasNextTimeColumn()) {
+      timeColumn = timeGenerator.nextTimeColumn();
+      if (timeColumn.hasCurrent()) {
+        hasCache = true;
+        cacheTime = timeColumn.currentTime();
+        timeColumn.next();
+        break;
+      }
+    }
+    return hasCache;
   }
 
   @Override
   protected RowRecord nextWithoutConstraint() throws IOException {
-    long timestamp = timeGenerator.next();
+    hasCache = false;
+    long timestamp = cacheTime;
 
 Review comment:
   you can use cacheTime directly

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [incubator-iotdb] qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #919: [IOTDB-483] switch all timeGenerate to use TimeColumn and make RawDataSetWithValueFilter to multi thread
URL: https://github.com/apache/incubator-iotdb/pull/919#discussion_r394087490
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java
 ##########
 @@ -267,22 +271,17 @@ private void aggregateWithValueFilter(List<AggregateResult> aggregateResults,
       TimeGenerator timestampGenerator, List<IReaderByTimestamp> 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();
-      }
-
+    while (timestampGenerator.hasNextTimeColumn()) {
+      TimeColumn timeColumn = timestampGenerator.nextTimeColumn();
       // cal part of aggregate result
+      int index = timeColumn.currentIndex();
 
 Review comment:
   I think this is always 0, just rest is ok

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services