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 2021/01/18 16:55:03 UTC

[GitHub] [iotdb] jixuan1989 commented on a change in pull request #2184: [IOTDB-1081] New TsFile Format

jixuan1989 commented on a change in pull request #2184:
URL: https://github.com/apache/iotdb/pull/2184#discussion_r559505727



##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
##########
@@ -334,18 +342,117 @@ private void checkUnClosedTsFileV2InFolders(List<String> folders) {
           if (!partitionDir.isDirectory()) {
             continue;
           }
-          File[] tsfiles = FSFactoryProducer.getFSFactory()
-            .listFilesBySuffix(partitionDir.toString(), TsFileConstant.TSFILE_SUFFIX);
-          File[] resources = FSFactoryProducer.getFSFactory()
-            .listFilesBySuffix(partitionDir.toString(), TsFileResource.RESOURCE_SUFFIX);
+          File[] tsfiles = fsFactory
+              .listFilesBySuffix(partitionDir.toString(), TsFileConstant.TSFILE_SUFFIX);
+          File[] resources = fsFactory
+              .listFilesBySuffix(partitionDir.toString(), TsFileResource.RESOURCE_SUFFIX);
           if (tsfiles.length != resources.length) {
-            logger.error("Unclosed Version-2 TsFile detected, please run 'flush' on v0.10 IoTDB"
-              + " before upgrading to v0.11");
-            System.exit(-1);
+            // If upgrading from v0.11.2 to v0.12, there may be some Unclosed merging files.

Review comment:
       The comments belong to line 369. 

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkGroupHeader.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.tsfile.file.header;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
+import org.apache.iotdb.tsfile.file.MetaMarker;
+import org.apache.iotdb.tsfile.read.reader.TsFileInput;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+public class ChunkGroupHeader {
+
+  private static final byte MARKER = MetaMarker.CHUNK_GROUP_HEADER;
+
+  private final String deviceID;
+
+  // this field does not need to be serialized.
+  private int serializedSize;
+
+  /**
+   * constructor of CHUNK_GROUP_FOOTER.

Review comment:
       replace all "FOOTER"s in this file

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/MetadataIndexNodeType.java
##########
@@ -42,19 +66,13 @@
    * @return MetadataIndexNodeType
    */
   public static MetadataIndexNodeType deserialize(byte i) {
-    if (i >= 4) {
-      throw new IllegalArgumentException("Invalid input: " + i);
-    }
-    switch (i) {
-      case 0:
-        return INTERNAL_DEVICE;
-      case 1:
-        return LEAF_DEVICE;
-      case 2:
-        return INTERNAL_MEASUREMENT;
-      default:
-        return LEAF_MEASUREMENT;
+    for (MetadataIndexNodeType metadataIndexNodeType : MetadataIndexNodeType.values()) {
+      if (i == metadataIndexNodeType.type) {

Review comment:
       not good using for-loop

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/PageHeader.java
##########
@@ -41,25 +40,37 @@ public PageHeader(int uncompressedSize, int compressedSize, Statistics statistic
     this.statistics = statistics;
   }
 
-  public static int calculatePageHeaderSizeWithoutStatistics() {
-    return 2 * Integer.BYTES; // uncompressedSize, compressedSize
+  /**
+   * max page header size without statistics
+   */
+  public static int estimateMaxPageHeaderSizeWithoutStatistics() {
+    return 2 * (Integer.BYTES + 1); // uncompressedSize, compressedSize

Review comment:
       comment why +1?

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/LocalTsFileInput.java
##########
@@ -145,4 +146,13 @@ public void close() throws IOException {
   public int readInt() {
     throw new UnsupportedOperationException();
   }
+
+  @Override
+  public String readVarIntString(long offset) throws IOException {

Review comment:
       this method... is not thread-safe... see how other functions use the channel.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteForEncodingUtils.java
##########
@@ -98,14 +98,24 @@ public static int getLongMaxBitWidth(List<Long> list) {
   public static int readUnsignedVarInt(InputStream in) throws IOException {

Review comment:
       by the way, how about ` public static int readUnsignedVarInt(ByteBuffer buffer)` (line 125)

##########
File path: server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
##########
@@ -334,18 +342,117 @@ private void checkUnClosedTsFileV2InFolders(List<String> folders) {
           if (!partitionDir.isDirectory()) {
             continue;
           }
-          File[] tsfiles = FSFactoryProducer.getFSFactory()
-            .listFilesBySuffix(partitionDir.toString(), TsFileConstant.TSFILE_SUFFIX);
-          File[] resources = FSFactoryProducer.getFSFactory()
-            .listFilesBySuffix(partitionDir.toString(), TsFileResource.RESOURCE_SUFFIX);
+          File[] tsfiles = fsFactory
+              .listFilesBySuffix(partitionDir.toString(), TsFileConstant.TSFILE_SUFFIX);
+          File[] resources = fsFactory
+              .listFilesBySuffix(partitionDir.toString(), TsFileResource.RESOURCE_SUFFIX);
           if (tsfiles.length != resources.length) {
-            logger.error("Unclosed Version-2 TsFile detected, please run 'flush' on v0.10 IoTDB"
-              + " before upgrading to v0.11");
-            System.exit(-1);
+            // If upgrading from v0.11.2 to v0.12, there may be some Unclosed merging files.
+            // We have to delete these files before upgrading
+            File[] unmergedTsfiles = fsFactory
+                .listFilesBySuffix(partitionDir.toString(), "0" + TsFileConstant.TSFILE_SUFFIX);
+            File[] unmergedResources = fsFactory
+                .listFilesBySuffix(partitionDir.toString(), "0" + TsFileResource.RESOURCE_SUFFIX);
+            if (unmergedTsfiles.length != unmergedResources.length) {
+              logger.error("Unclosed Version-2 TsFile detected, please stop insertion, then run 'flush' "
+                  + "on v0.11 IoTDB before upgrading to v0.12");
+              System.exit(-1);
+            }
+            deleteMergeingTsFiles(tsfiles, resources);
           }
         }
       }
     }
   }
+
+  private void deleteMergeingTsFiles(File[] tsfiles, File[] resources) {

Review comment:
       Point out that the upgrade tool will remove all files that having no resources, no matter whether they are complete tsfiles or in-merging files.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/MetaMarker.java
##########
@@ -26,15 +26,24 @@
  */
 public class MetaMarker {
 
-  public static final byte CHUNK_GROUP_FOOTER = 0;
+  public static final byte CHUNK_GROUP_HEADER = 0;
+  /**
+   * means this chunk has more than one page
+   */
   public static final byte CHUNK_HEADER = 1;
   public static final byte SEPARATOR = 2;
   public static final byte VERSION = 3;
+  
   // following this marker are two longs marking the minimum and maximum indices of operations
   // involved in the last flushed MemTable, which are generally used to support checkpoint,
   // snapshot, or backup.
   public static final byte OPERATION_INDEX_RANGE = 4;
 
+  /**
+   * means this chunk has only one page

Review comment:
       Chunk header marker and this chunk has only one page.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/MetaMarker.java
##########
@@ -26,15 +26,24 @@
  */
 public class MetaMarker {
 
-  public static final byte CHUNK_GROUP_FOOTER = 0;
+  public static final byte CHUNK_GROUP_HEADER = 0;
+  /**
+   * means this chunk has more than one page

Review comment:
       Chunk header marker and this chunk has more than one page.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkHeader.java
##########
@@ -33,101 +34,119 @@
 
 public class ChunkHeader {
 
+
+  /**
+   * 1 means this chunk has more than one page, so each page has its own page statistic 4 means this
+   * chunk has only one page, and this page has no page statistic
+   */
+  private byte chunkType;
   private String measurementID;
   private int dataSize;
   private TSDataType dataType;
   private CompressionType compressionType;
   private TSEncoding encodingType;
-  private int numOfPages;
 
-  // this field does not need to be serialized.
+  // the following fields do not need to be serialized.
+  private int numOfPages;
   private int serializedSize;
 
   public ChunkHeader(String measurementID, int dataSize, TSDataType dataType,
-      CompressionType compressionType,
-      TSEncoding encoding, int numOfPages) {
-    this(measurementID, dataSize, getSerializedSize(measurementID), dataType, compressionType,
-        encoding, numOfPages);
+      CompressionType compressionType, TSEncoding encoding, int numOfPages) {
+    this(numOfPages <= 1 ? MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER : MetaMarker.CHUNK_HEADER,
+        measurementID, dataSize, getSerializedSize(measurementID, dataSize), dataType,
+        compressionType,
+        encoding);
+    this.numOfPages = numOfPages;
   }
 
-  private ChunkHeader(String measurementID, int dataSize, int headerSize, TSDataType dataType,
-      CompressionType compressionType, TSEncoding encoding, int numOfPages) {
+  public ChunkHeader(byte chunkType, String measurementID, int dataSize, TSDataType dataType,
+      CompressionType compressionType, TSEncoding encoding) {
+    this(chunkType, measurementID, dataSize, getSerializedSize(measurementID, dataSize), dataType,
+        compressionType, encoding);
+  }
+
+  public ChunkHeader(byte chunkType, String measurementID, int dataSize, int headerSize,
+      TSDataType dataType, CompressionType compressionType, TSEncoding encoding) {
+    this.chunkType = chunkType;
     this.measurementID = measurementID;
     this.dataSize = dataSize;
     this.dataType = dataType;
     this.compressionType = compressionType;
-    this.numOfPages = numOfPages;
     this.encodingType = encoding;
     this.serializedSize = headerSize;
   }
 
+  /**
+   * the exact serialized size of chunk header
+   */
+  public static int getSerializedSize(String measurementID, int dataSize) {
+    int measurementIdLength = measurementID.getBytes(TSFileConfig.STRING_CHARSET).length;
+    return Byte.BYTES // chunkType
+        + ReadWriteForEncodingUtils.varIntSize(measurementIdLength) // measurementID length
+        + measurementIdLength // measurementID
+        + ReadWriteForEncodingUtils.uVarIntSize(dataSize) // dataSize
+        + TSDataType.getSerializedSize() // dataType
+        + CompressionType.getSerializedSize() // compressionType
+        + TSEncoding.getSerializedSize(); // encodingType
+  }
+
+  /**
+   * The estimated serialized size of chunk header. Only used when we don't know the actual dataSize
+   * attribute
+   */
   public static int getSerializedSize(String measurementID) {
-    return Byte.BYTES // marker
-        + Integer.BYTES // measurementID length
-        + measurementID.getBytes(TSFileConfig.STRING_CHARSET).length // measurementID
-        + Integer.BYTES // dataSize
+
+    int measurementIdLength = measurementID.getBytes(TSFileConfig.STRING_CHARSET).length;
+    return  Byte.BYTES // chunkType
+        + ReadWriteForEncodingUtils.varIntSize(measurementIdLength) // measurementID length
+        + measurementIdLength // measurementID
+        + Integer.BYTES + 1 // uVarInt dataSize
         + TSDataType.getSerializedSize() // dataType
         + CompressionType.getSerializedSize() // compressionType
-        + TSEncoding.getSerializedSize() // encodingType
-        + Integer.BYTES; // numOfPages
+        + TSEncoding.getSerializedSize(); // encodingType
   }
 
   /**
    * deserialize from inputStream.
-   *
-   * @param markerRead Whether the marker of the CHUNK_HEADER has been read
    */
-  public static ChunkHeader deserializeFrom(InputStream inputStream, boolean markerRead)
+  public static ChunkHeader deserializeFrom(InputStream inputStream, byte chunkType)

Review comment:
       this method begins from the position behind the marker, while the deserializeFrom(ByteBufer) begins from the position of the marker.
   
   It is inconsistent and hard to maintain codes.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/header/ChunkHeader.java
##########
@@ -33,101 +34,119 @@
 
 public class ChunkHeader {
 
+
+  /**
+   * 1 means this chunk has more than one page, so each page has its own page statistic 4 means this

Review comment:
       1 and 5!
   

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
##########
@@ -208,10 +211,14 @@ public String readTailMagic() throws IOException {
    * whether the file is a complete TsFile: only if the head magic and tail magic string exists.
    */
   public boolean isComplete() throws IOException {
-    return tsFileInput.size() >= TSFileConfig.MAGIC_STRING.getBytes().length * 2
-        + TSFileConfig.VERSION_NUMBER.getBytes().length
-        && (readTailMagic().equals(readHeadMagic()) || readTailMagic()
-        .equals(TSFileConfig.VERSION_NUMBER_V1));
+    long size = tsFileInput.size();
+    if (size >= TSFileConfig.MAGIC_STRING.getBytes().length * 2 + Byte.BYTES) {

Review comment:
       comment for Byte.BYTES.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/TimeseriesMetadata.java
##########
@@ -27,10 +27,17 @@
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
 import org.apache.iotdb.tsfile.read.controller.IChunkMetadataLoader;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
 import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
 
 public class TimeseriesMetadata implements Accountable {
 
+  /**
+   * 0 means this time series has only one chunk, no need to save the statistic again in chunk metadata

Review comment:
       using two static Byte is better..

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/PlainEncoder.java
##########
@@ -66,46 +54,29 @@ public void encode(boolean value, ByteArrayOutputStream out) {
 
   @Override
   public void encode(short value, ByteArrayOutputStream out) {
-    if (this.endianType == EndianType.LITTLE_ENDIAN) {
-      out.write(value & 0xFF);
-      out.write((value >> 8) & 0xFF);
-    } else if (this.endianType == EndianType.BIG_ENDIAN) {
-      out.write((value >> 8) & 0xFF);
-      out.write(value & 0xFF);
-    }
+    out.write((value >> 8) & 0xFF);
+    out.write(value & 0xFF);
   }
 
   @Override
   public void encode(int value, ByteArrayOutputStream out) {
-    if (this.endianType == EndianType.LITTLE_ENDIAN) {
-      out.write(value & 0xFF);
-      out.write((value >> 8) & 0xFF);
-      out.write((value >> 16) & 0xFF);
-      out.write((value >> 24) & 0xFF);
-    } else if (this.endianType == EndianType.BIG_ENDIAN) {
-      out.write((value >> 24) & 0xFF);
-      out.write((value >> 16) & 0xFF);
-      out.write((value >> 8) & 0xFF);
-      out.write(value & 0xFF);
-    }
+    ReadWriteForEncodingUtils.writeVarInt(value, out);
   }
 
   @Override
   public void encode(long value, ByteArrayOutputStream out) {
-    if (this.endianType == EndianType.LITTLE_ENDIAN) {
-      for (int i = 0; i < 8; i++) {
-        out.write((byte) (((value) >> (i * 8)) & 0xFF));
-      }
-    } else if (this.endianType == EndianType.BIG_ENDIAN) {
-      for (int i = 7; i >= 0; i--) {
-        out.write((byte) (((value) >> (i * 8)) & 0xFF));
-      }
+    for (int i = 7; i >= 0; i--) {

Review comment:
       contact @jt2594838  whether LITTLE_ENDIAN is not used anymore.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/MetadataIndexConstructor.java
##########
@@ -81,7 +80,7 @@ public static MetadataIndexNode constructMetadataIndex(Map<String, List<Timeseri
     // if not exceed the max child nodes num, ignore the device index and directly point to the measurement
     if (deviceMetadataIndexMap.size() <= config.getMaxDegreeOfIndexNode()) {
       MetadataIndexNode metadataIndexNode = new MetadataIndexNode(
-          MetadataIndexNodeType.INTERNAL_MEASUREMENT);
+          MetadataIndexNodeType.LEAF_DEVICE);

Review comment:
       please double check whether the modification is correct.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSEncoding.java
##########
@@ -32,85 +46,27 @@ public static TSEncoding deserialize(short encoding) {
     return getTsEncoding(encoding);
   }
 
-  public static byte deserializeToByte(short encoding) {
-    if (encoding < 0 || 8 < encoding) {
-      throw new IllegalArgumentException("Invalid input: " + encoding);
-    }
-    return (byte) encoding;
-  }
 
   private static TSEncoding getTsEncoding(short encoding) {
-    if (encoding < 0 || 8 < encoding) {
-      throw new IllegalArgumentException("Invalid input: " + encoding);
-    }
-    switch (encoding) {
-      case 1:
-        return PLAIN_DICTIONARY;
-      case 2:
-        return RLE;
-      case 3:
-        return DIFF;
-      case 4:
-        return TS_2DIFF;
-      case 5:
-        return BITMAP;
-      case 6:
-        return GORILLA_V1;
-      case 7:
-        return REGULAR;
-      case 8:
-        return GORILLA;
-      default:
-        return PLAIN;
+    for (TSEncoding tsEncoding : TSEncoding.values()) {

Review comment:
       what's the difference of using swtch-case

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteForEncodingUtils.java
##########
@@ -134,12 +153,42 @@ public static int readUnsignedVarInt(ByteBuffer buffer) {
    * @param value value to write into stream
    * @param out   output stream
    */
-  public static void writeUnsignedVarInt(int value, ByteArrayOutputStream out) {
+  public static int writeUnsignedVarInt(int value, ByteArrayOutputStream out) {

Review comment:
       add javadoc for @return 

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteForEncodingUtils.java
##########
@@ -98,14 +98,24 @@ public static int getLongMaxBitWidth(List<Long> list) {
   public static int readUnsignedVarInt(InputStream in) throws IOException {
     int value = 0;
     int i = 0;
-    int b;
-    while (((b = in.read()) & 0x80) != 0) {
+    int b = in.read();
+    while (b != -1 && (b & 0x80) != 0) {
       value |= (b & 0x7F) << i;
       i += 7;
+      b = in.read();
     }
     return value | (b << i);
   }
 
+  public static int readVarInt(InputStream in) throws IOException {

Review comment:
       correct? how about -1?

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/file/MetaMarker.java
##########
@@ -26,15 +26,24 @@
  */
 public class MetaMarker {
 
-  public static final byte CHUNK_GROUP_FOOTER = 0;
+  public static final byte CHUNK_GROUP_HEADER = 0;
+  /**
+   * means this chunk has more than one page
+   */
   public static final byte CHUNK_HEADER = 1;
   public static final byte SEPARATOR = 2;
   public static final byte VERSION = 3;

Review comment:
       add @deprecated annotation here?

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/TsFileSequenceReader.java
##########
@@ -583,8 +593,15 @@ private int binarySearchInTimeseriesMetadataList(List<TimeseriesMetadata> timese
     // read buffer of all ChunkMetadatas of this device
     ByteBuffer buffer = readData(start, size);
     Map<String, List<ChunkMetadata>> seriesMetadata = new HashMap<>();
+    int index = 0;
+    int curSize = timeseriesMetadataMap.get(index).getDataSizeOfChunkMetaDataList();
     while (buffer.hasRemaining()) {
-      ChunkMetadata chunkMetadata = ChunkMetadata.deserializeFrom(buffer);
+      if (buffer.position() >= curSize) {

Review comment:
       run `index++` and `curSize+=` after `seeriesMetadata.computeIfAbsent`? 

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Chunk.java
##########
@@ -18,30 +18,37 @@
  */
 package org.apache.iotdb.tsfile.read.common;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
-
 import java.util.List;
 import org.apache.iotdb.tsfile.common.cache.Accountable;
+import org.apache.iotdb.tsfile.file.MetaMarker;
 import org.apache.iotdb.tsfile.file.header.ChunkHeader;
+import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
+import org.apache.iotdb.tsfile.utils.PublicBAOS;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
 
 /**
  * used in query.
  */
 public class Chunk implements Accountable {
 
   private ChunkHeader chunkHeader;
+  private Statistics chunkStatistic;

Review comment:
       why add statistics here?

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteForEncodingUtils.java
##########
@@ -98,14 +98,24 @@ public static int getLongMaxBitWidth(List<Long> list) {
   public static int readUnsignedVarInt(InputStream in) throws IOException {

Review comment:
       modify javadoc: 
   @return -1 or int(>=0)

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/PlainEncoder.java
##########
@@ -66,46 +54,29 @@ public void encode(boolean value, ByteArrayOutputStream out) {
 
   @Override
   public void encode(short value, ByteArrayOutputStream out) {
-    if (this.endianType == EndianType.LITTLE_ENDIAN) {
-      out.write(value & 0xFF);
-      out.write((value >> 8) & 0xFF);
-    } else if (this.endianType == EndianType.BIG_ENDIAN) {
-      out.write((value >> 8) & 0xFF);
-      out.write(value & 0xFF);
-    }
+    out.write((value >> 8) & 0xFF);
+    out.write(value & 0xFF);
   }
 
   @Override
   public void encode(int value, ByteArrayOutputStream out) {
-    if (this.endianType == EndianType.LITTLE_ENDIAN) {
-      out.write(value & 0xFF);
-      out.write((value >> 8) & 0xFF);
-      out.write((value >> 16) & 0xFF);
-      out.write((value >> 24) & 0xFF);
-    } else if (this.endianType == EndianType.BIG_ENDIAN) {
-      out.write((value >> 24) & 0xFF);
-      out.write((value >> 16) & 0xFF);
-      out.write((value >> 8) & 0xFF);
-      out.write(value & 0xFF);
-    }
+    ReadWriteForEncodingUtils.writeVarInt(value, out);
   }
 
   @Override
   public void encode(long value, ByteArrayOutputStream out) {
-    if (this.endianType == EndianType.LITTLE_ENDIAN) {
-      for (int i = 0; i < 8; i++) {
-        out.write((byte) (((value) >> (i * 8)) & 0xFF));
-      }
-    } else if (this.endianType == EndianType.BIG_ENDIAN) {
-      for (int i = 7; i >= 0; i--) {
-        out.write((byte) (((value) >> (i * 8)) & 0xFF));
-      }
+    for (int i = 7; i >= 0; i--) {
+      out.write((byte) (((value) >> (i * 8)) & 0xFF));
     }
   }
 
   @Override
   public void encode(float value, ByteArrayOutputStream out) {
-    encode(Float.floatToIntBits(value), out);
+    int floatInt = Float.floatToIntBits(value);

Review comment:
       e...  it is not equal with previous version:
   public void encode(short value, ByteArrayOutputStream out) uses varint while this function does not.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Chunk.java
##########
@@ -60,12 +67,51 @@ public void setDeleteIntervalList(List<TimeRange> list) {
     this.deleteIntervalList = list;
   }
 
-  public void mergeChunk(Chunk chunk) {
-    chunkHeader.mergeChunkHeader(chunk.chunkHeader);
-    ByteBuffer newChunkData = ByteBuffer
-        .allocate(chunkData.array().length + chunk.chunkData.array().length);
-    newChunkData.put(chunkData.array());
-    newChunkData.put(chunk.chunkData.array());
+  public void mergeChunk(Chunk chunk) throws IOException {

Review comment:
       this method needs a lot of comments. Will review it after supplying comments.




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