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/01/20 07:13:19 UTC

[GitHub] [incubator-iotdb] liutaohua opened a new pull request #756: add new byTimestampReader

liutaohua opened a new pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756
 
 
   

----------------------------------------------------------------
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 #756: add new byTimestampReader

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756#discussion_r368986936
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/seriesRelated/SeriesDataReaderByTimestamp.java
 ##########
 @@ -0,0 +1,96 @@
+/*
+ * 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.seriesRelated;
+
+import java.io.IOException;
+import java.util.Objects;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+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 class SeriesDataReaderByTimestamp extends AbstractDataReader implements
+    IReaderByTimestamp {
+
+  private BatchData batchData;
+  private Filter filter;
+
+  public SeriesDataReaderByTimestamp(Path seriesPath, TSDataType dataType, QueryContext context,
+      QueryDataSource dataSource) {
+    super(seriesPath, dataType, context, dataSource.getSeqResources(),
+        dataSource.getUnseqResources());
+  }
+
+  @Override
+  public Object getValueInTimestamp(long timestamp) throws IOException {
+    this.filter = TimeFilter.gtEq(timestamp);
 
 Review comment:
   we may need to set the value in TimeEq filter, construct this object each time is costly.

----------------------------------------------------------------
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 #756: add new byTimestampReader

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756#discussion_r368998141
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/seriesRelated/SeriesDataReaderByTimestamp.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.seriesRelated;
+
+import java.io.IOException;
+import java.util.Objects;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+
+
+public class SeriesDataReaderByTimestamp extends AbstractDataReader implements
+    IReaderByTimestamp {
+
+  private BatchData batchData;
+  private long timestamp;
+
+  public SeriesDataReaderByTimestamp(Path seriesPath, TSDataType dataType, QueryContext context)
+      throws StorageEngineException {
+    super(seriesPath, dataType, null, context);
+  }
+
+  @Override
+  public Object getValueInTimestamp(long timestamp) throws IOException {
+    this.timestamp = timestamp;
+    if (batchData == null || batchData.getTimeByIndex(batchData.length() - 1) < timestamp) {
+      if (!hasNext(timestamp)) {
+        return null;
+      }
+    }
+
+    return batchData.getValueInTimestamp(timestamp);
+  }
+
+  private boolean hasNext(long timestamp) throws IOException {
+    while (super.hasNextChunk()) {
+      Statistics statistics = firstChunkMetaData.getStatistics();
+      if (statistics.getEndTime() < timestamp) {
+        hasCachedFirstChunkMetadata = false;
+        firstChunkMetaData = null;
+        continue;
+      }
+      while (super.hasNextPage()) {
+        Statistics pageStatistics = currentPageStatistics();
+        if (pageStatistics.getEndTime() < timestamp) {
+          overlappedPageReaders.poll();
+          continue;
+        }
+        if (canUseCurrentPageStatistics()) {
+          batchData = nextPage();
+        } else {
+          batchData = nextOverlappedPage();
+        }
+        if (batchData.getTimeByIndex(batchData.length() - 1) >= timestamp) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  private Statistics currentPageStatistics() throws IOException {
+    if (overlappedPageReaders.isEmpty() || overlappedPageReaders.peek().data == null) {
+      throw new IOException("No next page statistics.");
+    }
+    return overlappedPageReaders.peek().data.getStatistics();
+  }
+
+
+  @Override
+  protected boolean satisfyFilter(Statistics statistics) {
+    return true;
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
 
 Review comment:
   Currently, `ExternalSortJob` is also using `IReaderByTimestamp`, I'm not going to fix it in this pr
   
   
   

----------------------------------------------------------------
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 #756: add new byTimestampReader

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756#discussion_r369010484
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/chunkRelated/MemChunkReaderByTimestamp.java
 ##########
 @@ -19,31 +19,25 @@
 package org.apache.iotdb.db.query.reader.chunkRelated;
 
 import java.io.IOException;
-import java.util.List;
 import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
 import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
-import org.apache.iotdb.db.query.reader.fileRelated.UnSealedTsFileReaderByTimestamp;
 import org.apache.iotdb.db.query.reader.universal.PriorityMergeReader;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
 
 /**
  * To read data in memory by timestamp, this class implements an interface {@link
  * IReaderByTimestamp} based on the data source {@link ReadOnlyMemChunk}.
  * <p>
- * This class is used in {@link UnSealedTsFileReaderByTimestamp} and {@link
- * org.apache.iotdb.db.query.reader.resourceRelated.UnseqResourceReaderByTimestamp}.
  */
 public class MemChunkReaderByTimestamp implements IReaderByTimestamp {
 
   private PriorityMergeReader timeValuePairIterator;
   private boolean hasCachedTimeValuePair;
   private TimeValuePair cachedTimeValuePair;
 
-  public MemChunkReaderByTimestamp(List<ReadOnlyMemChunk> readableChunk) throws IOException {
+  public MemChunkReaderByTimestamp(ReadOnlyMemChunk memChunk) throws IOException {
     timeValuePairIterator = new PriorityMergeReader();
-    for (ReadOnlyMemChunk memChunk : readableChunk) {
-      timeValuePairIterator.addReader(memChunk.getIterator(), memChunk.getVersion());
-    }
+    timeValuePairIterator.addReader(memChunk.getIterator(), memChunk.getVersion());
 
 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 #756: add new byTimestampReader

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756#discussion_r368994222
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/seriesRelated/SeriesDataReaderByTimestamp.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.seriesRelated;
+
+import java.io.IOException;
+import java.util.Objects;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+
+
+public class SeriesDataReaderByTimestamp extends AbstractDataReader implements
+    IReaderByTimestamp {
+
+  private BatchData batchData;
+  private long timestamp;
+
+  public SeriesDataReaderByTimestamp(Path seriesPath, TSDataType dataType, QueryContext context)
+      throws StorageEngineException {
+    super(seriesPath, dataType, null, context);
+  }
+
+  @Override
+  public Object getValueInTimestamp(long timestamp) throws IOException {
+    this.timestamp = timestamp;
+    if (batchData == null || batchData.getTimeByIndex(batchData.length() - 1) < timestamp) {
+      if (!hasNext(timestamp)) {
+        return null;
+      }
+    }
+
+    return batchData.getValueInTimestamp(timestamp);
+  }
+
+  private boolean hasNext(long timestamp) throws IOException {
+    while (super.hasNextChunk()) {
+      Statistics statistics = firstChunkMetaData.getStatistics();
+      if (statistics.getEndTime() < timestamp) {
 
 Review comment:
   You're absolutely right, I need to refactor part of `AbstractReader` to complete the implementation of this method

----------------------------------------------------------------
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 merged pull request #756: [IOTDB-434] Refactor SeriesReaderByTimestamp

Posted by GitBox <gi...@apache.org>.
qiaojialin merged pull request #756: [IOTDB-434] Refactor SeriesReaderByTimestamp
URL: https://github.com/apache/incubator-iotdb/pull/756
 
 
   

----------------------------------------------------------------
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 #756: add new byTimestampReader

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756#discussion_r368998241
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/resourceRelated/SeqResourceReaderByTimestamp.java
 ##########
 @@ -1,190 +1,189 @@
-/*
- * 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.resourceRelated;
-
-import java.io.IOException;
-import java.util.List;
-import org.apache.iotdb.db.engine.cache.DeviceMetaDataCache;
-import org.apache.iotdb.db.engine.modification.Modification;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.query.control.FileReaderManager;
-import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
-import org.apache.iotdb.db.query.reader.fileRelated.FileSeriesReaderByTimestampAdapter;
-import org.apache.iotdb.db.query.reader.fileRelated.UnSealedTsFileReaderByTimestamp;
-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.IChunkLoader;
-import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl;
-import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderByTimestamp;
-
-/**
- * To read a chronologically ordered list of sequence TsFiles by timestamp, this class implements
- * <code>IReaderByTimestamp</code> for the TsFiles.
- * <p>
- * Notes: 1) The list of sequence TsFiles is in strict chronological order. 2) The data in a
- * sequence TsFile is also organized in chronological order. 3) A sequence TsFile can be either
- * sealed or unsealed. 4) An unsealed sequence TsFile consists of two parts of data in chronological
- * order: data that has been flushed to disk and data in the flushing memtable list.
- * <p>
- * This class is used in {@link org.apache.iotdb.db.query.reader.seriesRelated.SeriesReaderByTimestamp}.
- */
-
-public class SeqResourceReaderByTimestamp implements IReaderByTimestamp {
-
-  protected Path seriesPath;
-  private List<TsFileResource> seqResources;
-  private QueryContext context;
-  private int nextIntervalFileIndex;
-  private IReaderByTimestamp seriesReader;
-
-  /**
-   * Constructor function.
-   * <p>
-   *
-   * @param seriesPath the path of the series data
-   * @param seqResources a list of sequence TsFile resources in chronological order
-   * @param context query context
-   */
-  public SeqResourceReaderByTimestamp(Path seriesPath, List<TsFileResource> seqResources,
-      QueryContext context) {
-    this.seriesPath = seriesPath;
-    this.seqResources = seqResources;
-    this.context = context;
-    this.nextIntervalFileIndex = 0;
-    this.seriesReader = null;
-  }
-
-  @Override
-  public Object getValueInTimestamp(long timestamp) throws IOException {
-    Object value = null;
-
-    if (seriesReader != null) {
-      value = seriesReader.getValueInTimestamp(timestamp);
-      // if get value or no value in this timestamp but has next, return.
-      if (value != null || seriesReader.hasNext()) {
-        return value;
-      }
-    }
-
-    // Because the sequence TsFile resources are chronologically globally ordered, there exists at
-    // most one TsFile resource that overlaps this timestamp.
-    while (nextIntervalFileIndex < seqResources.size()) {
-      boolean isConstructed = constructNextReader(nextIntervalFileIndex++, timestamp);
-      if (isConstructed) {
-        value = seriesReader.getValueInTimestamp(timestamp);
-        // if get value or no value in this timestamp but has next, return.
-        if (value != null || seriesReader.hasNext()) {
-          return value;
-        }
-      }
-    }
-    return value;
-  }
-
-  @Override
-  public boolean hasNext() throws IOException {
-    if (seriesReader != null && seriesReader.hasNext()) {
-      return true;
-    }
-
-    while (nextIntervalFileIndex < seqResources.size()) {
-      TsFileResource tsFileResource = seqResources.get(nextIntervalFileIndex++);
-      if (tsFileResource.isClosed()) {
-        seriesReader = initSealedTsFileReaderByTimestamp(tsFileResource, context);
-      } else {
-        seriesReader = new UnSealedTsFileReaderByTimestamp(tsFileResource);
-      }
-      if (seriesReader.hasNext()) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * If the idx-th TsFile in the <code>seqResources</code> might overlap this
-   * <code>timestamp</code>, then construct <code>IReaderByTimestamp</code> for it, assign to the
-   * <code>currentSeriesReader</code> and return true. Otherwise, return false.
-   * <p>
-   * Note that the list of sequence TsFiles is chronologically ordered, so there will be at most one
-   * TsFile that overlaps this timestamp.
-   *
-   * @param idx the index of the TsFile in the resource list
-   * @param timestamp check whether or not to construct the reader according to this timestamp
-   * @return True if the reader is constructed; False if not.
-   */
-  private boolean constructNextReader(int idx, long timestamp) throws IOException {
-    TsFileResource tsFileResource = seqResources.get(idx);
-    if (tsFileResource.isClosed()) {
-      if (isTsFileNotSatisfied(tsFileResource, timestamp)) {
-        return false;
-      }
-      seriesReader = initSealedTsFileReaderByTimestamp(tsFileResource, context);
-      return true;
-    } else {
-      // an unsealed sequence TsFile's endTimeMap size may be equal to 0 or greater than 0
-      // If endTimeMap size is 0, conservatively assume that this TsFile might overlap this timestamp.
-      // If endTimeMap size is not 0, call isTsFileNotSatisfied to check.
-      if (tsFileResource.getEndTimeMap().size() != 0) {
-        if (isTsFileNotSatisfied(tsFileResource, timestamp)) {
-          return false;
-        }
-      }
-      seriesReader = new UnSealedTsFileReaderByTimestamp(tsFileResource);
-      return true;
-    }
-  }
-
-  /**
-   * Returns true if the end time of the series data in this sequence TsFile is smaller than this
-   * timestamp.
-   * <p>
-   * Note that <code>seqResources</code> is a list of chronologically ordered sequence TsFiles, so
-   * there will be at most one TsFile that overlaps this timestamp.
-   * <p>
-   * This method is used to in <code>constructNextReader</code> to check whether this TsFile can be
-   * skipped.
-   */
-  private boolean isTsFileNotSatisfied(TsFileResource tsFile, long timestamp) {
-    return tsFile.getEndTimeMap().get(seriesPath.getDevice()) < timestamp;
-  }
-
-  private IReaderByTimestamp initSealedTsFileReaderByTimestamp(TsFileResource sealedTsFile,
-      QueryContext context) throws IOException {
-    // prepare metaDataList
-    List<ChunkMetaData> metaDataList = DeviceMetaDataCache.getInstance()
-        .get(sealedTsFile, seriesPath);
-
-    List<Modification> pathModifications = context.getPathModifications(sealedTsFile.getModFile(),
-        seriesPath.getFullPath());
-    if (!pathModifications.isEmpty()) {
-      QueryUtils.modifyChunkMetaData(metaDataList, pathModifications);
-    }
-    // prepare chunkLoader
-    TsFileSequenceReader tsFileReader = FileReaderManager.getInstance()
-        .get(sealedTsFile, true);
-    IChunkLoader chunkLoader = new ChunkLoaderImpl(tsFileReader);
-
-    return new FileSeriesReaderByTimestampAdapter(
-        new FileSeriesReaderByTimestamp(chunkLoader, metaDataList));
-  }
-}
\ No newline at end of file
+///*
+// * 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.resourceRelated;
+//
+//import java.io.IOException;
+//import java.util.List;
+//import org.apache.iotdb.db.engine.cache.DeviceMetaDataCache;
+//import org.apache.iotdb.db.engine.modification.Modification;
+//import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+//import org.apache.iotdb.db.query.context.QueryContext;
+//import org.apache.iotdb.db.query.control.FileReaderManager;
+//import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
+//import org.apache.iotdb.db.query.reader.fileRelated.FileSeriesReaderByTimestampAdapter;
+//import org.apache.iotdb.db.query.reader.fileRelated.UnSealedTsFileReaderByTimestamp;
+//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.IChunkLoader;
+//import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl;
+//import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderByTimestamp;
+//
+///**
+// * To read a chronologically ordered list of sequence TsFiles by timestamp, this class implements
+// * <code>IReaderByTimestamp</code> for the TsFiles.
+// * <p>
+// * Notes: 1) The list of sequence TsFiles is in strict chronological order. 2) The data in a
+// * sequence TsFile is also organized in chronological order. 3) A sequence TsFile can be either
+// * sealed or unsealed. 4) An unsealed sequence TsFile consists of two parts of data in chronological
+// * order: data that has been flushed to disk and data in the flushing memtable list.
+// * <p>
+// */
+//
+//public class SeqResourceReaderByTimestamp implements IReaderByTimestamp {
 
 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 #756: add new byTimestampReader

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756#discussion_r368987618
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/seriesRelated/SeriesDataReaderByTimestamp.java
 ##########
 @@ -0,0 +1,96 @@
+/*
+ * 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.seriesRelated;
+
+import java.io.IOException;
+import java.util.Objects;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+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 class SeriesDataReaderByTimestamp extends AbstractDataReader implements
+    IReaderByTimestamp {
+
+  private BatchData batchData;
+  private Filter filter;
+
+  public SeriesDataReaderByTimestamp(Path seriesPath, TSDataType dataType, QueryContext context,
+      QueryDataSource dataSource) {
+    super(seriesPath, dataType, context, dataSource.getSeqResources(),
+        dataSource.getUnseqResources());
+  }
+
+  @Override
+  public Object getValueInTimestamp(long timestamp) throws IOException {
+    this.filter = TimeFilter.gtEq(timestamp);
+    if (batchData == null || batchData.getTimeByIndex(batchData.length() - 1) < timestamp) {
+      if (!hasNext(timestamp)) {
+        return null;
+      }
+    }
+
+    return batchData.getValueInTimestamp(timestamp);
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
 
 Review comment:
   could this method be removed from the IReaderByTimestamp?

----------------------------------------------------------------
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 #756: add new byTimestampReader

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756#discussion_r368874270
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/seriesRelated/SeriesDataReaderByTimestamp.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.seriesRelated;
+
+import java.io.IOException;
+import java.util.Objects;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+
+
+public class SeriesDataReaderByTimestamp extends AbstractDataReader implements
+    IReaderByTimestamp {
+
+  private BatchData batchData;
+  private long timestamp;
+
+  public SeriesDataReaderByTimestamp(Path seriesPath, TSDataType dataType, QueryContext context)
+      throws StorageEngineException {
+    super(seriesPath, dataType, null, context);
+  }
+
+  @Override
+  public Object getValueInTimestamp(long timestamp) throws IOException {
+    this.timestamp = timestamp;
+    if (batchData == null || batchData.getTimeByIndex(batchData.length() - 1) < timestamp) {
+      if (!hasNext(timestamp)) {
+        return null;
+      }
+    }
+
+    return batchData.getValueInTimestamp(timestamp);
+  }
+
+  private boolean hasNext(long timestamp) throws IOException {
+    while (super.hasNextChunk()) {
+      Statistics statistics = firstChunkMetaData.getStatistics();
+      if (statistics.getEndTime() < timestamp) {
+        hasCachedFirstChunkMetadata = false;
+        firstChunkMetaData = null;
+        continue;
+      }
+      while (super.hasNextPage()) {
+        Statistics pageStatistics = currentPageStatistics();
+        if (pageStatistics.getEndTime() < timestamp) {
+          overlappedPageReaders.poll();
+          continue;
+        }
+        if (canUseCurrentPageStatistics()) {
+          batchData = nextPage();
+        } else {
+          batchData = nextOverlappedPage();
+        }
+        if (batchData.getTimeByIndex(batchData.length() - 1) >= timestamp) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  private Statistics currentPageStatistics() throws IOException {
+    if (overlappedPageReaders.isEmpty() || overlappedPageReaders.peek().data == null) {
+      throw new IOException("No next page statistics.");
+    }
+    return overlappedPageReaders.peek().data.getStatistics();
+  }
+
+
+  @Override
+  protected boolean satisfyFilter(Statistics statistics) {
+    return true;
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
 
 Review comment:
   Remove the hasNext method in IReaderByTimestamp, it's a redundant method.
   
   Try to remove the SeqResourceReaderByTimestamp, UnseqResourceReaderByTimestamp and all related classes that unsless.

----------------------------------------------------------------
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 #756: add new byTimestampReader

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756#discussion_r368871279
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/seriesRelated/SeriesDataReaderByTimestamp.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.seriesRelated;
+
+import java.io.IOException;
+import java.util.Objects;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+
+
+public class SeriesDataReaderByTimestamp extends AbstractDataReader implements
+    IReaderByTimestamp {
+
+  private BatchData batchData;
+  private long timestamp;
+
+  public SeriesDataReaderByTimestamp(Path seriesPath, TSDataType dataType, QueryContext context)
+      throws StorageEngineException {
+    super(seriesPath, dataType, null, context);
+  }
+
+  @Override
+  public Object getValueInTimestamp(long timestamp) throws IOException {
+    this.timestamp = timestamp;
+    if (batchData == null || batchData.getTimeByIndex(batchData.length() - 1) < timestamp) {
+      if (!hasNext(timestamp)) {
+        return null;
+      }
+    }
+
+    return batchData.getValueInTimestamp(timestamp);
+  }
+
+  private boolean hasNext(long timestamp) throws IOException {
+    while (super.hasNextChunk()) {
+      Statistics statistics = firstChunkMetaData.getStatistics();
+      if (statistics.getEndTime() < timestamp) {
 
 Review comment:
   It's better to treat the timestamp into a time >= t or time = t filter, and set this filter into AbstractDataReader.
   Then, the loadSatisfiedChunkMetadatas method in AbstractDataReader will help filter chunks. 
   
   Futhermore, the tryToFillChunkMetadatas should be improved in this condition, the filter should be applied to the TsFileResource, if the startTime and end time of a tsfile does not satisfy the filter, we should not call loadSatisfiedChunkMetadatas for this resource.

----------------------------------------------------------------
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 #756: add new byTimestampReader

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756#discussion_r368984484
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/resourceRelated/SeqResourceReaderByTimestamp.java
 ##########
 @@ -1,190 +1,189 @@
-/*
- * 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.resourceRelated;
-
-import java.io.IOException;
-import java.util.List;
-import org.apache.iotdb.db.engine.cache.DeviceMetaDataCache;
-import org.apache.iotdb.db.engine.modification.Modification;
-import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
-import org.apache.iotdb.db.query.context.QueryContext;
-import org.apache.iotdb.db.query.control.FileReaderManager;
-import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
-import org.apache.iotdb.db.query.reader.fileRelated.FileSeriesReaderByTimestampAdapter;
-import org.apache.iotdb.db.query.reader.fileRelated.UnSealedTsFileReaderByTimestamp;
-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.IChunkLoader;
-import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl;
-import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderByTimestamp;
-
-/**
- * To read a chronologically ordered list of sequence TsFiles by timestamp, this class implements
- * <code>IReaderByTimestamp</code> for the TsFiles.
- * <p>
- * Notes: 1) The list of sequence TsFiles is in strict chronological order. 2) The data in a
- * sequence TsFile is also organized in chronological order. 3) A sequence TsFile can be either
- * sealed or unsealed. 4) An unsealed sequence TsFile consists of two parts of data in chronological
- * order: data that has been flushed to disk and data in the flushing memtable list.
- * <p>
- * This class is used in {@link org.apache.iotdb.db.query.reader.seriesRelated.SeriesReaderByTimestamp}.
- */
-
-public class SeqResourceReaderByTimestamp implements IReaderByTimestamp {
-
-  protected Path seriesPath;
-  private List<TsFileResource> seqResources;
-  private QueryContext context;
-  private int nextIntervalFileIndex;
-  private IReaderByTimestamp seriesReader;
-
-  /**
-   * Constructor function.
-   * <p>
-   *
-   * @param seriesPath the path of the series data
-   * @param seqResources a list of sequence TsFile resources in chronological order
-   * @param context query context
-   */
-  public SeqResourceReaderByTimestamp(Path seriesPath, List<TsFileResource> seqResources,
-      QueryContext context) {
-    this.seriesPath = seriesPath;
-    this.seqResources = seqResources;
-    this.context = context;
-    this.nextIntervalFileIndex = 0;
-    this.seriesReader = null;
-  }
-
-  @Override
-  public Object getValueInTimestamp(long timestamp) throws IOException {
-    Object value = null;
-
-    if (seriesReader != null) {
-      value = seriesReader.getValueInTimestamp(timestamp);
-      // if get value or no value in this timestamp but has next, return.
-      if (value != null || seriesReader.hasNext()) {
-        return value;
-      }
-    }
-
-    // Because the sequence TsFile resources are chronologically globally ordered, there exists at
-    // most one TsFile resource that overlaps this timestamp.
-    while (nextIntervalFileIndex < seqResources.size()) {
-      boolean isConstructed = constructNextReader(nextIntervalFileIndex++, timestamp);
-      if (isConstructed) {
-        value = seriesReader.getValueInTimestamp(timestamp);
-        // if get value or no value in this timestamp but has next, return.
-        if (value != null || seriesReader.hasNext()) {
-          return value;
-        }
-      }
-    }
-    return value;
-  }
-
-  @Override
-  public boolean hasNext() throws IOException {
-    if (seriesReader != null && seriesReader.hasNext()) {
-      return true;
-    }
-
-    while (nextIntervalFileIndex < seqResources.size()) {
-      TsFileResource tsFileResource = seqResources.get(nextIntervalFileIndex++);
-      if (tsFileResource.isClosed()) {
-        seriesReader = initSealedTsFileReaderByTimestamp(tsFileResource, context);
-      } else {
-        seriesReader = new UnSealedTsFileReaderByTimestamp(tsFileResource);
-      }
-      if (seriesReader.hasNext()) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * If the idx-th TsFile in the <code>seqResources</code> might overlap this
-   * <code>timestamp</code>, then construct <code>IReaderByTimestamp</code> for it, assign to the
-   * <code>currentSeriesReader</code> and return true. Otherwise, return false.
-   * <p>
-   * Note that the list of sequence TsFiles is chronologically ordered, so there will be at most one
-   * TsFile that overlaps this timestamp.
-   *
-   * @param idx the index of the TsFile in the resource list
-   * @param timestamp check whether or not to construct the reader according to this timestamp
-   * @return True if the reader is constructed; False if not.
-   */
-  private boolean constructNextReader(int idx, long timestamp) throws IOException {
-    TsFileResource tsFileResource = seqResources.get(idx);
-    if (tsFileResource.isClosed()) {
-      if (isTsFileNotSatisfied(tsFileResource, timestamp)) {
-        return false;
-      }
-      seriesReader = initSealedTsFileReaderByTimestamp(tsFileResource, context);
-      return true;
-    } else {
-      // an unsealed sequence TsFile's endTimeMap size may be equal to 0 or greater than 0
-      // If endTimeMap size is 0, conservatively assume that this TsFile might overlap this timestamp.
-      // If endTimeMap size is not 0, call isTsFileNotSatisfied to check.
-      if (tsFileResource.getEndTimeMap().size() != 0) {
-        if (isTsFileNotSatisfied(tsFileResource, timestamp)) {
-          return false;
-        }
-      }
-      seriesReader = new UnSealedTsFileReaderByTimestamp(tsFileResource);
-      return true;
-    }
-  }
-
-  /**
-   * Returns true if the end time of the series data in this sequence TsFile is smaller than this
-   * timestamp.
-   * <p>
-   * Note that <code>seqResources</code> is a list of chronologically ordered sequence TsFiles, so
-   * there will be at most one TsFile that overlaps this timestamp.
-   * <p>
-   * This method is used to in <code>constructNextReader</code> to check whether this TsFile can be
-   * skipped.
-   */
-  private boolean isTsFileNotSatisfied(TsFileResource tsFile, long timestamp) {
-    return tsFile.getEndTimeMap().get(seriesPath.getDevice()) < timestamp;
-  }
-
-  private IReaderByTimestamp initSealedTsFileReaderByTimestamp(TsFileResource sealedTsFile,
-      QueryContext context) throws IOException {
-    // prepare metaDataList
-    List<ChunkMetaData> metaDataList = DeviceMetaDataCache.getInstance()
-        .get(sealedTsFile, seriesPath);
-
-    List<Modification> pathModifications = context.getPathModifications(sealedTsFile.getModFile(),
-        seriesPath.getFullPath());
-    if (!pathModifications.isEmpty()) {
-      QueryUtils.modifyChunkMetaData(metaDataList, pathModifications);
-    }
-    // prepare chunkLoader
-    TsFileSequenceReader tsFileReader = FileReaderManager.getInstance()
-        .get(sealedTsFile, true);
-    IChunkLoader chunkLoader = new ChunkLoaderImpl(tsFileReader);
-
-    return new FileSeriesReaderByTimestampAdapter(
-        new FileSeriesReaderByTimestamp(chunkLoader, metaDataList));
-  }
-}
\ No newline at end of file
+///*
+// * 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.resourceRelated;
+//
+//import java.io.IOException;
+//import java.util.List;
+//import org.apache.iotdb.db.engine.cache.DeviceMetaDataCache;
+//import org.apache.iotdb.db.engine.modification.Modification;
+//import org.apache.iotdb.db.engine.storagegroup.TsFileResource;
+//import org.apache.iotdb.db.query.context.QueryContext;
+//import org.apache.iotdb.db.query.control.FileReaderManager;
+//import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
+//import org.apache.iotdb.db.query.reader.fileRelated.FileSeriesReaderByTimestampAdapter;
+//import org.apache.iotdb.db.query.reader.fileRelated.UnSealedTsFileReaderByTimestamp;
+//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.IChunkLoader;
+//import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl;
+//import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderByTimestamp;
+//
+///**
+// * To read a chronologically ordered list of sequence TsFiles by timestamp, this class implements
+// * <code>IReaderByTimestamp</code> for the TsFiles.
+// * <p>
+// * Notes: 1) The list of sequence TsFiles is in strict chronological order. 2) The data in a
+// * sequence TsFile is also organized in chronological order. 3) A sequence TsFile can be either
+// * sealed or unsealed. 4) An unsealed sequence TsFile consists of two parts of data in chronological
+// * order: data that has been flushed to disk and data in the flushing memtable list.
+// * <p>
+// */
+//
+//public class SeqResourceReaderByTimestamp implements IReaderByTimestamp {
 
 Review comment:
   remove 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] qiaojialin commented on a change in pull request #756: add new byTimestampReader

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756#discussion_r369006543
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/chunkRelated/MemChunkReaderByTimestamp.java
 ##########
 @@ -19,31 +19,25 @@
 package org.apache.iotdb.db.query.reader.chunkRelated;
 
 import java.io.IOException;
-import java.util.List;
 import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
 import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
-import org.apache.iotdb.db.query.reader.fileRelated.UnSealedTsFileReaderByTimestamp;
 import org.apache.iotdb.db.query.reader.universal.PriorityMergeReader;
 import org.apache.iotdb.tsfile.read.TimeValuePair;
 
 /**
  * To read data in memory by timestamp, this class implements an interface {@link
  * IReaderByTimestamp} based on the data source {@link ReadOnlyMemChunk}.
  * <p>
- * This class is used in {@link UnSealedTsFileReaderByTimestamp} and {@link
- * org.apache.iotdb.db.query.reader.resourceRelated.UnseqResourceReaderByTimestamp}.
  */
 public class MemChunkReaderByTimestamp implements IReaderByTimestamp {
 
   private PriorityMergeReader timeValuePairIterator;
   private boolean hasCachedTimeValuePair;
   private TimeValuePair cachedTimeValuePair;
 
-  public MemChunkReaderByTimestamp(List<ReadOnlyMemChunk> readableChunk) throws IOException {
+  public MemChunkReaderByTimestamp(ReadOnlyMemChunk memChunk) throws IOException {
     timeValuePairIterator = new PriorityMergeReader();
-    for (ReadOnlyMemChunk memChunk : readableChunk) {
-      timeValuePairIterator.addReader(memChunk.getIterator(), memChunk.getVersion());
-    }
+    timeValuePairIterator.addReader(memChunk.getIterator(), memChunk.getVersion());
 
 Review comment:
   no need to use a  prioritemergereader

----------------------------------------------------------------
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 #756: add new byTimestampReader

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756#discussion_r368998770
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/seriesRelated/SeriesDataReaderByTimestamp.java
 ##########
 @@ -0,0 +1,96 @@
+/*
+ * 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.seriesRelated;
+
+import java.io.IOException;
+import java.util.Objects;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+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 class SeriesDataReaderByTimestamp extends AbstractDataReader implements
+    IReaderByTimestamp {
+
+  private BatchData batchData;
+  private Filter filter;
+
+  public SeriesDataReaderByTimestamp(Path seriesPath, TSDataType dataType, QueryContext context,
+      QueryDataSource dataSource) {
+    super(seriesPath, dataType, context, dataSource.getSeqResources(),
+        dataSource.getUnseqResources());
+  }
+
+  @Override
+  public Object getValueInTimestamp(long timestamp) throws IOException {
+    this.filter = TimeFilter.gtEq(timestamp);
+    if (batchData == null || batchData.getTimeByIndex(batchData.length() - 1) < timestamp) {
+      if (!hasNext(timestamp)) {
+        return null;
+      }
+    }
+
+    return batchData.getValueInTimestamp(timestamp);
+  }
+
+  @Override
+  public boolean hasNext() throws IOException {
 
 Review comment:
   I'm not going to fix it in this pr

----------------------------------------------------------------
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 #756: add new byTimestampReader

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756#discussion_r368872644
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/seriesRelated/SeriesDataReaderByTimestamp.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.seriesRelated;
+
+import java.io.IOException;
+import java.util.Objects;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+
+
+public class SeriesDataReaderByTimestamp extends AbstractDataReader implements
+    IReaderByTimestamp {
+
+  private BatchData batchData;
+  private long timestamp;
+
+  public SeriesDataReaderByTimestamp(Path seriesPath, TSDataType dataType, QueryContext context)
+      throws StorageEngineException {
+    super(seriesPath, dataType, null, context);
+  }
+
+  @Override
+  public Object getValueInTimestamp(long timestamp) throws IOException {
+    this.timestamp = timestamp;
+    if (batchData == null || batchData.getTimeByIndex(batchData.length() - 1) < timestamp) {
+      if (!hasNext(timestamp)) {
+        return null;
+      }
+    }
+
+    return batchData.getValueInTimestamp(timestamp);
+  }
+
+  private boolean hasNext(long timestamp) throws IOException {
+    while (super.hasNextChunk()) {
+      Statistics statistics = firstChunkMetaData.getStatistics();
+      if (statistics.getEndTime() < timestamp) {
+        hasCachedFirstChunkMetadata = false;
+        firstChunkMetaData = null;
+        continue;
+      }
+      while (super.hasNextPage()) {
+        Statistics pageStatistics = currentPageStatistics();
+        if (pageStatistics.getEndTime() < timestamp) {
+          overlappedPageReaders.poll();
+          continue;
+        }
+        if (canUseCurrentPageStatistics()) {
+          batchData = nextPage();
+        } else {
+          batchData = nextOverlappedPage();
+        }
+        if (batchData.getTimeByIndex(batchData.length() - 1) >= timestamp) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  private Statistics currentPageStatistics() throws IOException {
 
 Review comment:
   Maybe this class should extend SeriesDataReaderWithoutValueFilter

----------------------------------------------------------------
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 #756: add new byTimestampReader

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756#discussion_r368850921
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java
 ##########
 @@ -75,12 +75,13 @@ public GroupByWithValueFilterDataSet(long queryId, GroupByPlan groupByPlan) {
    * init reader and aggregate function.
    */
   private void initGroupBy(QueryContext context, GroupByPlan groupByPlan)
-      throws StorageEngineException, IOException, PathException {
+      throws StorageEngineException, IOException {
 
 Review comment:
   ```suggestion
         throws StorageEngineException {
   ```

----------------------------------------------------------------
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 #756: add new byTimestampReader

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756#discussion_r368998485
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/seriesRelated/SeriesDataReaderByTimestamp.java
 ##########
 @@ -0,0 +1,96 @@
+/*
+ * 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.seriesRelated;
+
+import java.io.IOException;
+import java.util.Objects;
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+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 class SeriesDataReaderByTimestamp extends AbstractDataReader implements
+    IReaderByTimestamp {
+
+  private BatchData batchData;
+  private Filter filter;
+
+  public SeriesDataReaderByTimestamp(Path seriesPath, TSDataType dataType, QueryContext context,
+      QueryDataSource dataSource) {
+    super(seriesPath, dataType, context, dataSource.getSeqResources(),
+        dataSource.getUnseqResources());
+  }
+
+  @Override
+  public Object getValueInTimestamp(long timestamp) throws IOException {
+    this.filter = TimeFilter.gtEq(timestamp);
 
 Review comment:
   no , I think `gtEq` should be used here

----------------------------------------------------------------
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 #756: add new byTimestampReader

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #756: add new byTimestampReader
URL: https://github.com/apache/incubator-iotdb/pull/756#discussion_r368995080
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/seriesRelated/SeriesDataReaderByTimestamp.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.seriesRelated;
+
+import java.io.IOException;
+import java.util.Objects;
+import org.apache.iotdb.db.exception.StorageEngineException;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.reader.IReaderByTimestamp;
+import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.read.common.BatchData;
+import org.apache.iotdb.tsfile.read.common.Path;
+
+
+public class SeriesDataReaderByTimestamp extends AbstractDataReader implements
+    IReaderByTimestamp {
+
+  private BatchData batchData;
+  private long timestamp;
+
+  public SeriesDataReaderByTimestamp(Path seriesPath, TSDataType dataType, QueryContext context)
+      throws StorageEngineException {
+    super(seriesPath, dataType, null, context);
+  }
+
+  @Override
+  public Object getValueInTimestamp(long timestamp) throws IOException {
+    this.timestamp = timestamp;
+    if (batchData == null || batchData.getTimeByIndex(batchData.length() - 1) < timestamp) {
+      if (!hasNext(timestamp)) {
+        return null;
+      }
+    }
+
+    return batchData.getValueInTimestamp(timestamp);
+  }
+
+  private boolean hasNext(long timestamp) throws IOException {
+    while (super.hasNextChunk()) {
+      Statistics statistics = firstChunkMetaData.getStatistics();
+      if (statistics.getEndTime() < timestamp) {
+        hasCachedFirstChunkMetadata = false;
+        firstChunkMetaData = null;
+        continue;
+      }
+      while (super.hasNextPage()) {
+        Statistics pageStatistics = currentPageStatistics();
+        if (pageStatistics.getEndTime() < timestamp) {
+          overlappedPageReaders.poll();
+          continue;
+        }
+        if (canUseCurrentPageStatistics()) {
+          batchData = nextPage();
+        } else {
+          batchData = nextOverlappedPage();
+        }
+        if (batchData.getTimeByIndex(batchData.length() - 1) >= timestamp) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  private Statistics currentPageStatistics() throws IOException {
 
 Review comment:
   I think should be ` extends AbstractReader`  directly, `currentPageStatistics` is not the`SeriesDataReaderWithoutValueFilter` method

----------------------------------------------------------------
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