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/17 12:08:54 UTC

[GitHub] [incubator-iotdb] liutaohua opened a new pull request #753: write tvlist

liutaohua opened a new pull request #753: write tvlist
URL: https://github.com/apache/incubator-iotdb/pull/753
 
 
   still working

----------------------------------------------------------------
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 #753: Add iterators to TVList to prevent data copy

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #753: Add iterators to TVList to prevent data copy
URL: https://github.com/apache/incubator-iotdb/pull/753#discussion_r368265571
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
 ##########
 @@ -18,106 +18,69 @@
  */
 package org.apache.iotdb.db.engine.querycontext;
 
-import java.util.Collections;
-import java.util.Iterator;
+import java.io.IOException;
 import java.util.List;
 import java.util.Map;
-import org.apache.iotdb.db.engine.memtable.MemSeriesLazyMerger;
-import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
 import org.apache.iotdb.db.query.reader.MemChunkLoader;
-import org.apache.iotdb.db.utils.MathUtils;
-import org.apache.iotdb.tsfile.read.TimeValuePair;
+import org.apache.iotdb.db.query.reader.universal.PriorityMergeReader;
+import org.apache.iotdb.db.utils.datastructure.TVList;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsDouble;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsFloat;
+import org.apache.iotdb.tsfile.read.IPointReader;
+import org.apache.iotdb.tsfile.read.TimeValuePair;
 
 //TODO: merge ReadOnlyMemChunk and WritableMemChunk and IWritableMemChunk
-public class ReadOnlyMemChunk implements TimeValuePairSorter {
+public class ReadOnlyMemChunk {
 
-  private boolean initialized;
   private String measurementUid;
   private TSDataType dataType;
-  private TimeValuePairSorter memSeries;
-  private List<TimeValuePair> sortedTimeValuePairList;
 
   Map<String, String> props;
   private int floatPrecision = TSFileDescriptor.getInstance().getConfig().getFloatPrecision();
+  private ChunkMetaData cachedMetaData;
+
+  private PriorityMergeReader mergeReader;
+  private PriorityMergeReader chunkedReader;
 
   /**
    * init by TSDataType and TimeValuePairSorter.
    */
-  public ReadOnlyMemChunk(String measurementUid, TSDataType dataType, TimeValuePairSorter memSeries,
-      Map<String, String> props) {
+  public ReadOnlyMemChunk(String measurementUid, TSDataType dataType,
+      List<TVList> memSeries, Map<String, String> props) throws IOException {
     this.measurementUid = measurementUid;
     this.dataType = dataType;
-    this.memSeries = memSeries;
-    this.initialized = false;
     this.props = props;
     if (props.containsKey(Encoder.MAX_POINT_NUMBER)) {
       this.floatPrecision = Integer.parseInt(props.get(Encoder.MAX_POINT_NUMBER));
     }
-  }
-
-  private void checkInitialized() {
-    if (!initialized) {
-      init();
-    }
-  }
-
-  private void init() {
-    sortedTimeValuePairList = memSeries.getSortedTimeValuePairList();
-    if (!(memSeries instanceof MemSeriesLazyMerger)) {
-      switch (dataType) {
-        case FLOAT:
-          sortedTimeValuePairList.replaceAll(x -> new TimeValuePair(x.getTimestamp(),
-              new TsFloat(
-                  MathUtils.roundWithGivenPrecision(x.getValue().getFloat(), floatPrecision))));
-          break;
-        case DOUBLE:
-          sortedTimeValuePairList.replaceAll(x -> new TimeValuePair(x.getTimestamp(),
-              new TsDouble(
-                  MathUtils.roundWithGivenPrecision(x.getValue().getDouble(), floatPrecision))));
-          break;
-        default:
-          break;
-      }
+    mergeReader = new PriorityMergeReader(floatPrecision);
+    chunkedReader = new PriorityMergeReader(floatPrecision);
+    for (TVList pair : memSeries) {
 
 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 #753: Add iterators to TVList to prevent data copy

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #753: Add iterators to TVList to prevent data copy
URL: https://github.com/apache/incubator-iotdb/pull/753#discussion_r368263014
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileProcessor.java
 ##########
 @@ -583,28 +587,28 @@ public String getStorageGroupName() {
       String measurementId, TSDataType dataType, Map<String, String> props, QueryContext context) {
     flushQueryLock.readLock().lock();
     try {
-      MemSeriesLazyMerger memSeriesLazyMerger = new MemSeriesLazyMerger();
+      List<TVList> tvLists = new ArrayList<>();
 
 Review comment:
   replace this with List<ReadOnlyMemchunk>

----------------------------------------------------------------
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 #753: Add iterators to TVList to prevent data copy

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #753: Add iterators to TVList to prevent data copy
URL: https://github.com/apache/incubator-iotdb/pull/753#discussion_r368261745
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
 ##########
 @@ -195,19 +199,19 @@ public boolean isEmpty() {
   }
 
   @Override
-  public ReadOnlyMemChunk query(String deviceId, String measurement, TSDataType dataType,
+  public TVList query(String deviceId, String measurement, TSDataType dataType,
       Map<String, String> props, long timeLowerBound) {
-    TimeValuePairSorter sorter;
     if (!checkPath(deviceId, measurement)) {
       return null;
-    } else {
-      long undeletedTime = findUndeletedTime(deviceId, measurement, timeLowerBound);
-      IWritableMemChunk memChunk = memTableMap.get(deviceId).get(measurement);
-      IWritableMemChunk chunkCopy = new WritableMemChunk(dataType, memChunk.getTVList().clone());
-      chunkCopy.setTimeOffset(undeletedTime);
-      sorter = chunkCopy;
     }
-    return new ReadOnlyMemChunk(measurement, dataType, sorter, props);
+    long undeletedTime = findUndeletedTime(deviceId, measurement, timeLowerBound);
+    IWritableMemChunk memChunk = memTableMap.get(deviceId).get(measurement);
+    TVList chunkCopy = memChunk.getTVList().clone();
+
+    chunkCopy.setVersion(getVersion());
 
 Review comment:
   put this to clone method of TVList

----------------------------------------------------------------
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 #753: Add iterators to TVList to prevent data copy

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #753: Add iterators to TVList to prevent data copy
URL: https://github.com/apache/incubator-iotdb/pull/753#discussion_r368262728
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
 ##########
 @@ -18,106 +18,69 @@
  */
 package org.apache.iotdb.db.engine.querycontext;
 
-import java.util.Collections;
-import java.util.Iterator;
+import java.io.IOException;
 import java.util.List;
 import java.util.Map;
-import org.apache.iotdb.db.engine.memtable.MemSeriesLazyMerger;
-import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
 import org.apache.iotdb.db.query.reader.MemChunkLoader;
-import org.apache.iotdb.db.utils.MathUtils;
-import org.apache.iotdb.tsfile.read.TimeValuePair;
+import org.apache.iotdb.db.query.reader.universal.PriorityMergeReader;
+import org.apache.iotdb.db.utils.datastructure.TVList;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsDouble;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsFloat;
+import org.apache.iotdb.tsfile.read.IPointReader;
+import org.apache.iotdb.tsfile.read.TimeValuePair;
 
 //TODO: merge ReadOnlyMemChunk and WritableMemChunk and IWritableMemChunk
-public class ReadOnlyMemChunk implements TimeValuePairSorter {
+public class ReadOnlyMemChunk {
 
-  private boolean initialized;
   private String measurementUid;
   private TSDataType dataType;
-  private TimeValuePairSorter memSeries;
-  private List<TimeValuePair> sortedTimeValuePairList;
 
   Map<String, String> props;
   private int floatPrecision = TSFileDescriptor.getInstance().getConfig().getFloatPrecision();
+  private ChunkMetaData cachedMetaData;
+
+  private PriorityMergeReader mergeReader;
+  private PriorityMergeReader chunkedReader;
 
 Review comment:
   I suggest using one ReadOnlyMemChunk to store one chunk in a memtable. The TsFileProcessor return a Pair<List<ReadOnlyMemtable>, List<ChunkMetadata>>
   Init the chunk metadata in the constructor in this class and cache it. 
   Therefore, we only need one chunkreader, and it should not be a mergereader, we could directly get the  IPointReader in TVList.
   In the AbstractMemTable, the version should be init by Long.MAX_VALUE for a working memtable, the memtable version is set only when it comes to a  flushing memtable.
   
   

----------------------------------------------------------------
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 #753: Add iterators to TVList to prevent data copy

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #753: Add iterators to TVList to prevent data copy
URL: https://github.com/apache/incubator-iotdb/pull/753#discussion_r368263165
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
 ##########
 @@ -452,11 +470,62 @@ void updateMinTimeAndSorted(long[] time, int start, int end) {
     boolean inputSorted = true;
     for (int i = start; i < end; i++) {
       inPutMinTime = inPutMinTime <= time[i] ? inPutMinTime : time[i];
-      if (inputSorted && i < length - 1 && time[i] > time[i+1]) {
+      if (inputSorted && i < length - 1 && time[i] > time[i + 1]) {
         inputSorted = false;
       }
     }
     minTime = inPutMinTime < minTime ? inPutMinTime : minTime;
     sorted = sorted && inputSorted && (size == 0 || inPutMinTime >= getTime(size - 1));
   }
+
+  protected abstract TimeValuePair getTimeValuePair(int index, long time);
+
+  public IPointReader getIterator() {
+    return new Ite();
+  }
+
+  private class Ite implements IPointReader {
+
+    private TimeValuePair cachedTimeValuePair;
+    private boolean hasCachedPair;
+    private int cur;
+
+    @Override
+    public boolean hasNextTimeValuePair() {
+      if (hasCachedPair) {
+        return true;
+      }
+
+      while (cur < size) {
+        long time = getTime(cur);
+        if (time < getTimeOffset() || (cur + 1 < size() && (time == getTime(cur + 1)))) {
+          cur++;
+          continue;
+        }
+        cachedTimeValuePair = getTimeValuePair(cur, time);
+        hasCachedPair = true;
+        cur++;
+        return true;
+      }
+      return hasCachedPair;
+    }
+
+    @Override
+    public TimeValuePair nextTimeValuePair() {
+      hasCachedPair = false;
 
 Review comment:
   if (hasCachedPair || hasNextTimeValuePair) {
       hasCachedPair = false
      return cachedTimeValuePair
   } else {
      throw new IOException("no next time value 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 #753: Add iterators to TVList to prevent data copy

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #753: Add iterators to TVList to prevent data copy
URL: https://github.com/apache/incubator-iotdb/pull/753#discussion_r368268572
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
 ##########
 @@ -18,106 +18,69 @@
  */
 package org.apache.iotdb.db.engine.querycontext;
 
-import java.util.Collections;
-import java.util.Iterator;
+import java.io.IOException;
 import java.util.List;
 import java.util.Map;
-import org.apache.iotdb.db.engine.memtable.MemSeriesLazyMerger;
-import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
 import org.apache.iotdb.db.query.reader.MemChunkLoader;
-import org.apache.iotdb.db.utils.MathUtils;
-import org.apache.iotdb.tsfile.read.TimeValuePair;
+import org.apache.iotdb.db.query.reader.universal.PriorityMergeReader;
+import org.apache.iotdb.db.utils.datastructure.TVList;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsDouble;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsFloat;
+import org.apache.iotdb.tsfile.read.IPointReader;
+import org.apache.iotdb.tsfile.read.TimeValuePair;
 
 //TODO: merge ReadOnlyMemChunk and WritableMemChunk and IWritableMemChunk
-public class ReadOnlyMemChunk implements TimeValuePairSorter {
+public class ReadOnlyMemChunk {
 
-  private boolean initialized;
   private String measurementUid;
   private TSDataType dataType;
-  private TimeValuePairSorter memSeries;
-  private List<TimeValuePair> sortedTimeValuePairList;
 
   Map<String, String> props;
   private int floatPrecision = TSFileDescriptor.getInstance().getConfig().getFloatPrecision();
+  private ChunkMetaData cachedMetaData;
+
+  private PriorityMergeReader mergeReader;
+  private PriorityMergeReader chunkedReader;
 
 Review comment:
   this is a pretty nice design

----------------------------------------------------------------
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 #753: Add iterators to TVList to prevent data copy

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #753: Add iterators to TVList to prevent data copy
URL: https://github.com/apache/incubator-iotdb/pull/753#discussion_r368265522
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java
 ##########
 @@ -223,51 +223,6 @@ public void setTimeOffset(long offset) {
     list.setTimeOffset(offset);
   }
 
-  @Override
-  public synchronized List<TimeValuePair> getSortedTimeValuePairList() {
 
 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 #753: Add iterators to TVList to prevent data copy

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #753: Add iterators to TVList to prevent data copy
URL: https://github.com/apache/incubator-iotdb/pull/753#discussion_r368265613
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/utils/datastructure/TVList.java
 ##########
 @@ -452,11 +470,62 @@ void updateMinTimeAndSorted(long[] time, int start, int end) {
     boolean inputSorted = true;
     for (int i = start; i < end; i++) {
       inPutMinTime = inPutMinTime <= time[i] ? inPutMinTime : time[i];
-      if (inputSorted && i < length - 1 && time[i] > time[i+1]) {
+      if (inputSorted && i < length - 1 && time[i] > time[i + 1]) {
         inputSorted = false;
       }
     }
     minTime = inPutMinTime < minTime ? inPutMinTime : minTime;
     sorted = sorted && inputSorted && (size == 0 || inPutMinTime >= getTime(size - 1));
   }
+
+  protected abstract TimeValuePair getTimeValuePair(int index, long time);
+
+  public IPointReader getIterator() {
+    return new Ite();
+  }
+
+  private class Ite implements IPointReader {
+
+    private TimeValuePair cachedTimeValuePair;
+    private boolean hasCachedPair;
+    private int cur;
+
+    @Override
+    public boolean hasNextTimeValuePair() {
+      if (hasCachedPair) {
+        return true;
+      }
+
+      while (cur < size) {
+        long time = getTime(cur);
+        if (time < getTimeOffset() || (cur + 1 < size() && (time == getTime(cur + 1)))) {
+          cur++;
+          continue;
+        }
+        cachedTimeValuePair = getTimeValuePair(cur, time);
+        hasCachedPair = true;
+        cur++;
+        return true;
+      }
+      return hasCachedPair;
+    }
+
+    @Override
+    public TimeValuePair nextTimeValuePair() {
+      hasCachedPair = false;
 
 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 merged pull request #753: Add iterators to TVList to prevent data copy

Posted by GitBox <gi...@apache.org>.
qiaojialin merged pull request #753: Add iterators to TVList to prevent data copy
URL: https://github.com/apache/incubator-iotdb/pull/753
 
 
   

----------------------------------------------------------------
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 #753: Add iterators to TVList to prevent data copy

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #753: Add iterators to TVList to prevent data copy
URL: https://github.com/apache/incubator-iotdb/pull/753#discussion_r368265539
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java
 ##########
 @@ -195,19 +199,19 @@ public boolean isEmpty() {
   }
 
   @Override
-  public ReadOnlyMemChunk query(String deviceId, String measurement, TSDataType dataType,
+  public TVList query(String deviceId, String measurement, TSDataType dataType,
       Map<String, String> props, long timeLowerBound) {
-    TimeValuePairSorter sorter;
     if (!checkPath(deviceId, measurement)) {
       return null;
-    } else {
-      long undeletedTime = findUndeletedTime(deviceId, measurement, timeLowerBound);
-      IWritableMemChunk memChunk = memTableMap.get(deviceId).get(measurement);
-      IWritableMemChunk chunkCopy = new WritableMemChunk(dataType, memChunk.getTVList().clone());
-      chunkCopy.setTimeOffset(undeletedTime);
-      sorter = chunkCopy;
     }
-    return new ReadOnlyMemChunk(measurement, dataType, sorter, props);
+    long undeletedTime = findUndeletedTime(deviceId, measurement, timeLowerBound);
+    IWritableMemChunk memChunk = memTableMap.get(deviceId).get(measurement);
+    TVList chunkCopy = memChunk.getTVList().clone();
+
+    chunkCopy.setVersion(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] qiaojialin commented on a change in pull request #753: Add iterators to TVList to prevent data copy

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #753: Add iterators to TVList to prevent data copy
URL: https://github.com/apache/incubator-iotdb/pull/753#discussion_r368262176
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java
 ##########
 @@ -223,51 +223,6 @@ public void setTimeOffset(long offset) {
     list.setTimeOffset(offset);
   }
 
-  @Override
-  public synchronized List<TimeValuePair> getSortedTimeValuePairList() {
 
 Review comment:
   remove the sortedList field in this class and unused 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] qiaojialin commented on a change in pull request #753: Add iterators to TVList to prevent data copy

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #753: Add iterators to TVList to prevent data copy
URL: https://github.com/apache/incubator-iotdb/pull/753#discussion_r368262947
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/engine/querycontext/ReadOnlyMemChunk.java
 ##########
 @@ -18,106 +18,69 @@
  */
 package org.apache.iotdb.db.engine.querycontext;
 
-import java.util.Collections;
-import java.util.Iterator;
+import java.io.IOException;
 import java.util.List;
 import java.util.Map;
-import org.apache.iotdb.db.engine.memtable.MemSeriesLazyMerger;
-import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
 import org.apache.iotdb.db.query.reader.MemChunkLoader;
-import org.apache.iotdb.db.utils.MathUtils;
-import org.apache.iotdb.tsfile.read.TimeValuePair;
+import org.apache.iotdb.db.query.reader.universal.PriorityMergeReader;
+import org.apache.iotdb.db.utils.datastructure.TVList;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
 import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsDouble;
-import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsFloat;
+import org.apache.iotdb.tsfile.read.IPointReader;
+import org.apache.iotdb.tsfile.read.TimeValuePair;
 
 //TODO: merge ReadOnlyMemChunk and WritableMemChunk and IWritableMemChunk
-public class ReadOnlyMemChunk implements TimeValuePairSorter {
+public class ReadOnlyMemChunk {
 
-  private boolean initialized;
   private String measurementUid;
   private TSDataType dataType;
-  private TimeValuePairSorter memSeries;
-  private List<TimeValuePair> sortedTimeValuePairList;
 
   Map<String, String> props;
   private int floatPrecision = TSFileDescriptor.getInstance().getConfig().getFloatPrecision();
+  private ChunkMetaData cachedMetaData;
+
+  private PriorityMergeReader mergeReader;
+  private PriorityMergeReader chunkedReader;
 
   /**
    * init by TSDataType and TimeValuePairSorter.
    */
-  public ReadOnlyMemChunk(String measurementUid, TSDataType dataType, TimeValuePairSorter memSeries,
-      Map<String, String> props) {
+  public ReadOnlyMemChunk(String measurementUid, TSDataType dataType,
+      List<TVList> memSeries, Map<String, String> props) throws IOException {
     this.measurementUid = measurementUid;
     this.dataType = dataType;
-    this.memSeries = memSeries;
-    this.initialized = false;
     this.props = props;
     if (props.containsKey(Encoder.MAX_POINT_NUMBER)) {
       this.floatPrecision = Integer.parseInt(props.get(Encoder.MAX_POINT_NUMBER));
     }
-  }
-
-  private void checkInitialized() {
-    if (!initialized) {
-      init();
-    }
-  }
-
-  private void init() {
-    sortedTimeValuePairList = memSeries.getSortedTimeValuePairList();
-    if (!(memSeries instanceof MemSeriesLazyMerger)) {
-      switch (dataType) {
-        case FLOAT:
-          sortedTimeValuePairList.replaceAll(x -> new TimeValuePair(x.getTimestamp(),
-              new TsFloat(
-                  MathUtils.roundWithGivenPrecision(x.getValue().getFloat(), floatPrecision))));
-          break;
-        case DOUBLE:
-          sortedTimeValuePairList.replaceAll(x -> new TimeValuePair(x.getTimestamp(),
-              new TsDouble(
-                  MathUtils.roundWithGivenPrecision(x.getValue().getDouble(), floatPrecision))));
-          break;
-        default:
-          break;
-      }
+    mergeReader = new PriorityMergeReader(floatPrecision);
+    chunkedReader = new PriorityMergeReader(floatPrecision);
+    for (TVList pair : memSeries) {
 
 Review comment:
   ```suggestion
       for (TVList tvlist : memSeries) {
   ```

----------------------------------------------------------------
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 #753: Add iterators to TVList to prevent data copy

Posted by GitBox <gi...@apache.org>.
qiaojialin commented on a change in pull request #753: Add iterators to TVList to prevent data copy
URL: https://github.com/apache/incubator-iotdb/pull/753#discussion_r368263094
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/universal/PriorityMergeReader.java
 ##########
 @@ -42,6 +47,11 @@
   public PriorityMergeReader() {
   }
 
+  public PriorityMergeReader(int floatPrecision) {
 
 Review comment:
   If we use a List<ReadOnlyMemchunk> , this class should not be changed.
   is this a bug before?

----------------------------------------------------------------
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 #753: Add iterators to TVList to prevent data copy

Posted by GitBox <gi...@apache.org>.
liutaohua commented on a change in pull request #753: Add iterators to TVList to prevent data copy
URL: https://github.com/apache/incubator-iotdb/pull/753#discussion_r368268610
 
 

 ##########
 File path: server/src/main/java/org/apache/iotdb/db/query/reader/universal/PriorityMergeReader.java
 ##########
 @@ -42,6 +47,11 @@
   public PriorityMergeReader() {
   }
 
+  public PriorityMergeReader(int floatPrecision) {
 
 Review comment:
   It's for formatting data in memory

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