You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by ha...@apache.org on 2023/01/30 12:11:16 UTC

[iotdb] branch master updated: Chimp compression (#8766)

This is an automated email from the ASF dual-hosted git repository.

haonan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 2f63c44787 Chimp compression (#8766)
2f63c44787 is described below

commit 2f63c447874fa7932d8c3fc0fe3ef1b80c642e46
Author: Panagiotis Liakos <p....@di.uoa.gr>
AuthorDate: Mon Jan 30 14:11:09 2023 +0200

    Chimp compression (#8766)
    
    Co-authored-by: Haonan <hh...@outlook.com>
---
 client-cpp/src/main/Session.h                      |   3 +-
 client-py/iotdb/utils/IoTDBConstants.py            |   1 +
 docs/UserGuide/Data-Concept/Encoding.md            |   6 +
 docs/zh/UserGuide/Data-Concept/Encoding.md         |   6 +
 .../org/apache/iotdb/db/utils/SchemaUtils.java     |   2 +
 .../iotdb/tsfile/encoding/decoder/Decoder.java     |  14 +
 .../decoder/DoublePrecisionChimpDecoder.java       |  50 ++
 .../tsfile/encoding/decoder/IntChimpDecoder.java   | 156 ++++++
 .../tsfile/encoding/decoder/LongChimpDecoder.java  | 155 ++++++
 .../decoder/SinglePrecisionChimpDecoder.java       |  50 ++
 .../encoder/DoublePrecisionChimpEncoder.java       |  55 ++
 .../tsfile/encoding/encoder/IntChimpEncoder.java   | 188 +++++++
 .../tsfile/encoding/encoder/LongChimpEncoder.java  | 199 +++++++
 .../encoder/SinglePrecisionChimpEncoder.java       |  55 ++
 .../tsfile/encoding/encoder/TSEncodingBuilder.java |  29 +-
 .../tsfile/file/metadata/enums/TSEncoding.java     |   5 +-
 .../tsfile/encoding/decoder/ChimpDecoderTest.java  | 594 +++++++++++++++++++++
 17 files changed, 1565 insertions(+), 3 deletions(-)

diff --git a/client-cpp/src/main/Session.h b/client-cpp/src/main/Session.h
index 2f098be71a..079fb7b990 100644
--- a/client-cpp/src/main/Session.h
+++ b/client-cpp/src/main/Session.h
@@ -169,7 +169,8 @@ namespace TSEncoding {
         REGULAR = (char) 7,
         GORILLA = (char) 8,
         ZIGZAG = (char) 9,
-        FREQ = (char) 10
+        FREQ = (char) 10,
+	CHIMP = (char) 11
     };
 }
 
diff --git a/client-py/iotdb/utils/IoTDBConstants.py b/client-py/iotdb/utils/IoTDBConstants.py
index ef66741fec..f902ac6d58 100644
--- a/client-py/iotdb/utils/IoTDBConstants.py
+++ b/client-py/iotdb/utils/IoTDBConstants.py
@@ -61,6 +61,7 @@ class TSEncoding(Enum):
     GORILLA = 8
     ZIGZAG = 9
     FREQ = 10
+    CHIMP = 11
 
     # this method is implemented to avoid the issue reported by:
     # https://bugs.python.org/issue30545
diff --git a/docs/UserGuide/Data-Concept/Encoding.md b/docs/UserGuide/Data-Concept/Encoding.md
index 21306e4bab..505325d094 100644
--- a/docs/UserGuide/Data-Concept/Encoding.md
+++ b/docs/UserGuide/Data-Concept/Encoding.md
@@ -64,6 +64,12 @@ FREQ encoding is lossy. It transforms the time sequence to the frequency domain
   
 ZIGZAG encoding maps signed integers to unsigned integers so that numbers with a small absolute value (for instance, -1) have a small variant encoded value too. It does this in a way that "zig-zags" back and forth through the positive and negative integers.
 
+* CHIMP
+
+CHIMP encoding is lossless. It is the state-of-the-art compression algorithm for streaming floating point data, providing impressive savings compared to earlier approaches. It is suitable for any numerical sequence with similar values and works best for sequence data without large fluctuations and/or random noise.
+
+Usage restrictions: When using CHIMP to encode INT32 data, you need to ensure that there is no data point with the value `Integer.MIN_VALUE` in the sequence. When using CHIMP to encode INT64 data, you need to ensure that there is no data point with the value `Long.MIN_VALUE` in the sequence.
+
 ## Correspondence between data type and encoding
 
 The five encodings described in the previous sections are applicable to different data types. If the correspondence is wrong, the time series cannot be created correctly. 
diff --git a/docs/zh/UserGuide/Data-Concept/Encoding.md b/docs/zh/UserGuide/Data-Concept/Encoding.md
index 475f3a13b5..2d1d438340 100644
--- a/docs/zh/UserGuide/Data-Concept/Encoding.md
+++ b/docs/zh/UserGuide/Data-Concept/Encoding.md
@@ -63,6 +63,12 @@ GORILLA 编码是一种无损编码,它比较适合编码前后值比较接近
 
 ZigZag编码将有符号整型映射到无符号整型,适合比较小的整数。
 
+* CHIMP
+
+CHIMP 是一种无损编码。它是一种新的流式浮点数据压缩算法,可以节省存储空间。这个编码适用于前后值比较接近的数值序列,对波动小和随机噪声少的序列数据更加友好。
+
+使用限制:如果对 INT32 类型数据使用 CHIMP 编码,需要确保数据点中没有 `Integer.MIN_VALUE`。 如果对 INT64 类型数据使用 CHIMP 编码,需要确保数据点中没有 `Long.MIN_VALUE`。
+
 ## 数据类型与编码的对应关系
 
 前文介绍的五种编码适用于不同的数据类型,若对应关系错误,则无法正确创建时间序列。数据类型与支持其编码的编码方式对应关系总结如下表所示。
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java
index 034d4c7cf1..d7e4c2c531 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java
@@ -56,6 +56,7 @@ public class SchemaUtils {
     intSet.add(TSEncoding.GORILLA);
     intSet.add(TSEncoding.ZIGZAG);
     intSet.add(TSEncoding.FREQ);
+    intSet.add(TSEncoding.CHIMP);
     schemaChecker.put(TSDataType.INT32, intSet);
     schemaChecker.put(TSDataType.INT64, intSet);
 
@@ -66,6 +67,7 @@ public class SchemaUtils {
     floatSet.add(TSEncoding.GORILLA_V1);
     floatSet.add(TSEncoding.GORILLA);
     floatSet.add(TSEncoding.FREQ);
+    floatSet.add(TSEncoding.CHIMP);
     schemaChecker.put(TSDataType.FLOAT, floatSet);
     schemaChecker.put(TSDataType.DOUBLE, floatSet);
 
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java
index d0cf2f3595..c61c147145 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java
@@ -123,6 +123,20 @@ public abstract class Decoder {
         }
       case FREQ:
         return new FreqDecoder();
+      case CHIMP:
+        switch (dataType) {
+          case FLOAT:
+            return new SinglePrecisionChimpDecoder();
+          case DOUBLE:
+            return new DoublePrecisionChimpDecoder();
+          case INT32:
+            return new IntChimpDecoder();
+          case INT64:
+          case VECTOR:
+            return new LongChimpDecoder();
+          default:
+            throw new TsFileDecodingException(String.format(ERROR_MSG, encoding, dataType));
+        }
       default:
         throw new TsFileDecodingException(String.format(ERROR_MSG, encoding, dataType));
     }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/DoublePrecisionChimpDecoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/DoublePrecisionChimpDecoder.java
new file mode 100644
index 0000000000..a6e8ddf4f1
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/DoublePrecisionChimpDecoder.java
@@ -0,0 +1,50 @@
+/*
+ * 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.encoding.decoder;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This class includes code modified from Panagiotis Liakos chimp project.
+ *
+ * <p>Copyright: 2022- Panagiotis Liakos, Katia Papakonstantinopoulou and Yannis Kotidis
+ *
+ * <p>Project page: https://github.com/panagiotisl/chimp
+ *
+ * <p>License: http://www.apache.org/licenses/LICENSE-2.0
+ */
+public class DoublePrecisionChimpDecoder extends LongChimpDecoder {
+
+  private static final long CHIMP_ENCODING_ENDING = Double.doubleToRawLongBits(Double.NaN);
+
+  @Override
+  public final double readDouble(ByteBuffer in) {
+    return Double.longBitsToDouble(readLong(in));
+  }
+
+  @Override
+  protected long cacheNext(ByteBuffer in) {
+    readNext(in);
+    if (storedValues[current] == CHIMP_ENCODING_ENDING) {
+      hasNext = false;
+    }
+    return storedValues[current];
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/IntChimpDecoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/IntChimpDecoder.java
new file mode 100644
index 0000000000..41bae7e6e6
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/IntChimpDecoder.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.tsfile.encoding.decoder;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+
+import java.nio.ByteBuffer;
+
+import static org.apache.iotdb.tsfile.common.conf.TSFileConfig.VALUE_BITS_LENGTH_32BIT;
+
+/**
+ * This class includes code modified from Panagiotis Liakos chimp project.
+ *
+ * <p>Copyright: 2022- Panagiotis Liakos, Katia Papakonstantinopoulou and Yannis Kotidis
+ *
+ * <p>Project page: https://github.com/panagiotisl/chimp
+ *
+ * <p>License: http://www.apache.org/licenses/LICENSE-2.0
+ */
+public class IntChimpDecoder extends GorillaDecoderV2 {
+
+  private static final short[] LEADING_REPRESENTATION = {0, 8, 12, 16, 18, 20, 22, 24};
+  private static final int PREVIOUS_VALUES = 64;
+  private static final int PREVIOUS_VALUES_LOG2 = (int) (Math.log(PREVIOUS_VALUES) / Math.log(2));
+  private static final int CASE_ONE_METADATA_LENGTH = PREVIOUS_VALUES_LOG2 + 8;
+
+  private int storedValue = 0;
+  protected int storedValues[] = new int[PREVIOUS_VALUES];
+  protected int current = 0;
+
+  public IntChimpDecoder() {
+    this.setType(TSEncoding.CHIMP);
+    this.hasNext = true;
+    firstValueWasRead = false;
+    storedLeadingZeros = Integer.MAX_VALUE;
+    storedTrailingZeros = 0;
+    this.current = 0;
+    this.storedValue = 0;
+    this.storedValues = new int[PREVIOUS_VALUES];
+  }
+
+  @Override
+  public void reset() {
+    super.reset();
+
+    this.current = 0;
+    this.storedValue = 0;
+    this.storedValues = new int[PREVIOUS_VALUES];
+  }
+
+  @Override
+  public final int readInt(ByteBuffer in) {
+    int returnValue = storedValue;
+    if (!firstValueWasRead) {
+      flipByte(in);
+      storedValue = (int) readLong(VALUE_BITS_LENGTH_32BIT, in);
+      storedValues[current] = storedValue;
+      firstValueWasRead = true;
+      returnValue = storedValue;
+    }
+    cacheNext(in);
+    return returnValue;
+  }
+
+  protected int cacheNext(ByteBuffer in) {
+    readNext(in);
+    if (storedValues[current] == Integer.MIN_VALUE) {
+      hasNext = false;
+    }
+    return storedValues[current];
+  }
+
+  protected int readNext(ByteBuffer in) {
+    // read the two control bits
+    byte controlBits = readNextNBits(2, in);
+    int value;
+    switch (controlBits) {
+      case 3:
+        // case 11: read the length of the number of leading
+        // zeros in the next 3 bits, then read the
+        // meaningful bits of the XORed value.
+        storedLeadingZeros = LEADING_REPRESENTATION[(int) readLong(3, in)];
+        value = (int) readLong(VALUE_BITS_LENGTH_32BIT - storedLeadingZeros, in);
+        storedValue = storedValue ^ value;
+        current = (current + 1) % PREVIOUS_VALUES;
+        storedValues[current] = storedValue;
+        return storedValue;
+        // case 10: use the previous leading zeros and
+        // and just read the meaningful XORed value.
+      case 2:
+        value = (int) readLong(VALUE_BITS_LENGTH_32BIT - storedLeadingZeros, in);
+        storedValue = storedValue ^ value;
+        current = (current + 1) % PREVIOUS_VALUES;
+        storedValues[current] = storedValue;
+        return storedValue;
+        // case 01:  read the index of the previous value, the length of
+        // the number of leading zeros in the next 3 bits, then read
+        // the length of the meaningful XORed value in the next 5
+        // bits. Finally read the meaningful bits of the XORed value.
+      case 1:
+        int fill = CASE_ONE_METADATA_LENGTH;
+        int temp = (int) readLong(fill, in);
+        int index = temp >>> (fill -= PREVIOUS_VALUES_LOG2) & (1 << PREVIOUS_VALUES_LOG2) - 1;
+        storedLeadingZeros = LEADING_REPRESENTATION[temp >>> (fill -= 3) & (1 << 3) - 1];
+        int significantBits = temp >>> (fill -= 5) & (1 << 5) - 1;
+        storedValue = storedValues[index];
+        if (significantBits == 0) {
+          significantBits = VALUE_BITS_LENGTH_32BIT;
+        }
+        storedTrailingZeros = VALUE_BITS_LENGTH_32BIT - significantBits - storedLeadingZeros;
+        value =
+            (int) readLong(VALUE_BITS_LENGTH_32BIT - storedLeadingZeros - storedTrailingZeros, in);
+        value <<= storedTrailingZeros;
+        storedValue = storedValue ^ value;
+        current = (current + 1) % PREVIOUS_VALUES;
+        storedValues[current] = storedValue;
+        return storedValue;
+        // case 00: the values are identical, just read
+        // the index of the previous value
+      default:
+        int previousIndex = (int) readLong(PREVIOUS_VALUES_LOG2, in);
+        storedValue = storedValues[previousIndex];
+        current = (current + 1) % PREVIOUS_VALUES;
+        storedValues[current] = storedValue;
+        return storedValue;
+    }
+  }
+
+  private byte readNextNBits(int n, ByteBuffer in) {
+    byte value = 0x00;
+    for (int i = 0; i < n; i++) {
+      value <<= 1;
+      if (readBit(in)) {
+        value |= 0x01;
+      }
+    }
+    return value;
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/LongChimpDecoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/LongChimpDecoder.java
new file mode 100644
index 0000000000..59d063ceb5
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/LongChimpDecoder.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.tsfile.encoding.decoder;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+
+import java.nio.ByteBuffer;
+
+import static org.apache.iotdb.tsfile.common.conf.TSFileConfig.VALUE_BITS_LENGTH_64BIT;
+
+/**
+ * This class includes code modified from Panagiotis Liakos chimp project.
+ *
+ * <p>Copyright: 2022- Panagiotis Liakos, Katia Papakonstantinopoulou and Yannis Kotidis
+ *
+ * <p>Project page: https://github.com/panagiotisl/chimp
+ *
+ * <p>License: http://www.apache.org/licenses/LICENSE-2.0
+ */
+public class LongChimpDecoder extends GorillaDecoderV2 {
+
+  private static final short[] LEADING_REPRESENTATION = {0, 8, 12, 16, 18, 20, 22, 24};
+  private static final int PREVIOUS_VALUES = 128;
+  private static final int PREVIOUS_VALUES_LOG2 = (int) (Math.log(PREVIOUS_VALUES) / Math.log(2));
+  private static final int CASE_ONE_METADATA_LENGTH = PREVIOUS_VALUES_LOG2 + 9;
+
+  private long storedValue = 0;
+  protected long storedValues[] = new long[PREVIOUS_VALUES];
+  protected int current = 0;
+
+  public LongChimpDecoder() {
+    this.setType(TSEncoding.CHIMP);
+    this.hasNext = true;
+    firstValueWasRead = false;
+    storedLeadingZeros = Integer.MAX_VALUE;
+    storedTrailingZeros = 0;
+    this.current = 0;
+    this.storedValue = 0;
+    this.storedValues = new long[PREVIOUS_VALUES];
+  }
+
+  @Override
+  public void reset() {
+    super.reset();
+
+    this.current = 0;
+    this.storedValue = 0;
+    this.storedValues = new long[PREVIOUS_VALUES];
+  }
+
+  @Override
+  public final long readLong(ByteBuffer in) {
+    long returnValue = storedValue;
+    if (!firstValueWasRead) {
+      flipByte(in);
+      storedValue = readLong(VALUE_BITS_LENGTH_64BIT, in);
+      storedValues[current] = storedValue;
+      firstValueWasRead = true;
+      returnValue = storedValue;
+    }
+    cacheNext(in);
+    return returnValue;
+  }
+
+  protected long cacheNext(ByteBuffer in) {
+    readNext(in);
+    if (storedValues[current] == Long.MIN_VALUE) {
+      hasNext = false;
+    }
+    return storedValues[current];
+  }
+
+  protected long readNext(ByteBuffer in) {
+    // read the two control bits
+    byte controlBits = readNextNBits(2, in);
+    long value;
+    switch (controlBits) {
+        // case 11: read the length of the number of leading
+        // zeros in the next 3 bits, then read the
+        // meaningful bits of the XORed value.
+      case 3:
+        storedLeadingZeros = LEADING_REPRESENTATION[(int) readLong(3, in)];
+        value = readLong(VALUE_BITS_LENGTH_64BIT - storedLeadingZeros, in);
+        storedValue = storedValue ^ value;
+        current = (current + 1) % PREVIOUS_VALUES;
+        storedValues[current] = storedValue;
+        return storedValue;
+        // case 10: use the previous leading zeros and
+        // and just read the meaningful XORed value.
+      case 2:
+        value = readLong(VALUE_BITS_LENGTH_64BIT - storedLeadingZeros, in);
+        storedValue = storedValue ^ value;
+        current = (current + 1) % PREVIOUS_VALUES;
+        storedValues[current] = storedValue;
+        return storedValue;
+        // case 01:  read the index of the previous value, the length of
+        // the number of leading zeros in the next 3 bits, then read
+        // the length of the meaningful XORed value in the next 6
+        // bits. Finally read the meaningful bits of the XORed value.
+      case 1:
+        int fill = CASE_ONE_METADATA_LENGTH;
+        int temp = (int) readLong(fill, in);
+        int index = temp >>> (fill -= PREVIOUS_VALUES_LOG2) & (1 << PREVIOUS_VALUES_LOG2) - 1;
+        storedLeadingZeros = LEADING_REPRESENTATION[temp >>> (fill -= 3) & (1 << 3) - 1];
+        int significantBits = temp >>> (fill -= 6) & (1 << 6) - 1;
+        storedValue = storedValues[index];
+        if (significantBits == 0) {
+          significantBits = VALUE_BITS_LENGTH_64BIT;
+        }
+        storedTrailingZeros = VALUE_BITS_LENGTH_64BIT - significantBits - storedLeadingZeros;
+        value = readLong(VALUE_BITS_LENGTH_64BIT - storedLeadingZeros - storedTrailingZeros, in);
+        value <<= storedTrailingZeros;
+        storedValue = storedValue ^ value;
+        current = (current + 1) % PREVIOUS_VALUES;
+        storedValues[current] = storedValue;
+        return storedValue;
+        // case 00: the values are identical, just read
+        // the index of the previous value
+      default:
+        int previousIndex = (int) readLong(PREVIOUS_VALUES_LOG2, in);
+        storedValue = storedValues[previousIndex];
+        current = (current + 1) % PREVIOUS_VALUES;
+        storedValues[current] = storedValue;
+        return storedValue;
+    }
+  }
+
+  private byte readNextNBits(int n, ByteBuffer in) {
+    byte value = 0x00;
+    for (int i = 0; i < n; i++) {
+      value <<= 1;
+      if (readBit(in)) {
+        value |= 0x01;
+      }
+    }
+    return value;
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/SinglePrecisionChimpDecoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/SinglePrecisionChimpDecoder.java
new file mode 100644
index 0000000000..33a2290cab
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/SinglePrecisionChimpDecoder.java
@@ -0,0 +1,50 @@
+/*
+ * 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.encoding.decoder;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This class includes code modified from Panagiotis Liakos chimp project.
+ *
+ * <p>Copyright: 2022- Panagiotis Liakos, Katia Papakonstantinopoulou and Yannis Kotidis
+ *
+ * <p>Project page: https://github.com/panagiotisl/chimp
+ *
+ * <p>License: http://www.apache.org/licenses/LICENSE-2.0
+ */
+public class SinglePrecisionChimpDecoder extends IntChimpDecoder {
+
+  private static final int CHIMP_ENCODING_ENDING = Float.floatToRawIntBits(Float.NaN);
+
+  @Override
+  public final float readFloat(ByteBuffer in) {
+    return Float.intBitsToFloat(readInt(in));
+  }
+
+  @Override
+  protected int cacheNext(ByteBuffer in) {
+    readNext(in);
+    if (storedValues[current] == CHIMP_ENCODING_ENDING) {
+      hasNext = false;
+    }
+    return storedValues[current];
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DoublePrecisionChimpEncoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DoublePrecisionChimpEncoder.java
new file mode 100644
index 0000000000..cf762de0dc
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DoublePrecisionChimpEncoder.java
@@ -0,0 +1,55 @@
+/*
+ * 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.encoding.encoder;
+
+import java.io.ByteArrayOutputStream;
+
+/**
+ * This class includes code modified from Panagiotis Liakos chimp project.
+ *
+ * <p>Copyright: 2022- Panagiotis Liakos, Katia Papakonstantinopoulou and Yannis Kotidis
+ *
+ * <p>Project page: https://github.com/panagiotisl/chimp
+ *
+ * <p>License: http://www.apache.org/licenses/LICENSE-2.0
+ */
+public class DoublePrecisionChimpEncoder extends LongChimpEncoder {
+
+  private static final long CHIMP_ENCODING_ENDING = Double.doubleToRawLongBits(Double.NaN);
+
+  @Override
+  public final void encode(double value, ByteArrayOutputStream out) {
+    encode(Double.doubleToRawLongBits(value), out);
+  }
+
+  @Override
+  public void flush(ByteArrayOutputStream out) {
+    // ending stream
+    encode(CHIMP_ENCODING_ENDING, out);
+
+    // flip the byte no matter it is empty or not
+    // the empty ending byte is necessary when decoding
+    bitsLeft = 0;
+    flipByte(out);
+
+    // the encoder may be reused, so let us reset it
+    reset();
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/IntChimpEncoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/IntChimpEncoder.java
new file mode 100644
index 0000000000..1f792491b6
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/IntChimpEncoder.java
@@ -0,0 +1,188 @@
+/*
+ * 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.encoding.encoder;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+
+import java.io.ByteArrayOutputStream;
+
+import static org.apache.iotdb.tsfile.common.conf.TSFileConfig.LEADING_ZERO_BITS_LENGTH_32BIT;
+import static org.apache.iotdb.tsfile.common.conf.TSFileConfig.MEANINGFUL_XOR_BITS_LENGTH_32BIT;
+import static org.apache.iotdb.tsfile.common.conf.TSFileConfig.VALUE_BITS_LENGTH_32BIT;
+
+/**
+ * This class includes code modified from Panagiotis Liakos chimp project.
+ *
+ * <p>Copyright: 2022- Panagiotis Liakos, Katia Papakonstantinopoulou and Yannis Kotidis
+ *
+ * <p>Project page: https://github.com/panagiotisl/chimp
+ *
+ * <p>License: http://www.apache.org/licenses/LICENSE-2.0
+ */
+public class IntChimpEncoder extends GorillaEncoderV2 {
+
+  private static final int PREVIOUS_VALUES = 64;
+  private static final int PREVIOUS_VALUES_LOG2 = (int) (Math.log(PREVIOUS_VALUES) / Math.log(2));
+  private static final int THRESHOLD = 5 + PREVIOUS_VALUES_LOG2;
+  private static final int SET_LSB = (int) Math.pow(2, THRESHOLD + 1) - 1;
+  private static final int CASE_ZERO_METADATA_LENGTH = PREVIOUS_VALUES_LOG2 + 2;
+  private static final int CASE_ONE_METADATA_LENGTH = PREVIOUS_VALUES_LOG2 + 10;
+  public static final short[] LEADING_REPRESENTATION = {
+    0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 2, 2, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 7, 7, 7, 7, 7, 7, 7,
+    7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+  };
+
+  public static final short[] LEADING_ROUND = {
+    0, 0, 0, 0, 0, 0, 0, 0, 8, 8, 8, 8, 12, 12, 12, 12, 16, 16, 18, 18, 20, 20, 22, 22, 24, 24, 24,
+    24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24,
+    24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24
+  };
+
+  private int storedValues[];
+  private int[] indices;
+  private int index = 0;
+  private int current = 0;
+
+  public IntChimpEncoder() {
+    this.setType(TSEncoding.CHIMP);
+    this.indices = new int[(int) Math.pow(2, THRESHOLD + 1)];
+    this.storedValues = new int[PREVIOUS_VALUES];
+  }
+
+  private static final int ONE_ITEM_MAX_SIZE =
+      (2
+                  + LEADING_ZERO_BITS_LENGTH_32BIT
+                  + MEANINGFUL_XOR_BITS_LENGTH_32BIT
+                  + VALUE_BITS_LENGTH_32BIT)
+              / Byte.SIZE
+          + 1;
+
+  @Override
+  public final int getOneItemMaxSize() {
+    return ONE_ITEM_MAX_SIZE;
+  }
+
+  @Override
+  protected void reset() {
+    super.reset();
+    this.current = 0;
+    this.index = 0;
+    this.indices = new int[(int) Math.pow(2, THRESHOLD + 1)];
+    this.storedValues = new int[PREVIOUS_VALUES];
+  }
+
+  @Override
+  public void flush(ByteArrayOutputStream out) {
+    // ending stream
+    encode(Integer.MIN_VALUE, out);
+
+    // flip the byte no matter it is empty or not
+    // the empty ending byte is necessary when decoding
+    bitsLeft = 0;
+    flipByte(out);
+
+    // the encoder may be reused, so let us reset it
+    reset();
+  }
+
+  @Override
+  public final void encode(int value, ByteArrayOutputStream out) {
+    if (firstValueWasWritten) {
+      compressValue(value, out);
+    } else {
+      writeFirst(value, out);
+      firstValueWasWritten = true;
+    }
+  }
+
+  // the first value is stored with no compression
+  private void writeFirst(int value, ByteArrayOutputStream out) {
+    storedValues[current] = value;
+    writeBits(value, VALUE_BITS_LENGTH_32BIT, out);
+    indices[value & SET_LSB] = index;
+  }
+
+  private void compressValue(int value, ByteArrayOutputStream out) {
+    // find the best previous value
+    int key = value & SET_LSB;
+    int xor;
+    int previousIndex;
+    int trailingZeros = 0;
+    int currIndex = indices[key];
+    if ((index - currIndex) < PREVIOUS_VALUES) {
+      int tempXor = value ^ storedValues[currIndex % PREVIOUS_VALUES];
+      trailingZeros = Integer.numberOfTrailingZeros(tempXor);
+      if (trailingZeros > THRESHOLD) {
+        previousIndex = currIndex % PREVIOUS_VALUES;
+        xor = tempXor;
+      } else {
+        previousIndex = index % PREVIOUS_VALUES;
+        xor = storedValues[previousIndex] ^ value;
+      }
+    } else {
+      previousIndex = index % PREVIOUS_VALUES;
+      xor = storedValues[previousIndex] ^ value;
+    }
+
+    // case 00: the values are identical, write 00 control bits
+    // and the index of the previous value
+    if (xor == 0) {
+      writeBits(previousIndex, CASE_ZERO_METADATA_LENGTH, out);
+      storedLeadingZeros = VALUE_BITS_LENGTH_32BIT + 1;
+    } else {
+      int leadingZeros = LEADING_ROUND[Integer.numberOfLeadingZeros(xor)];
+      // case 01:  store the index, the length of
+      // the number of leading zeros in the next 3 bits, then store
+      // the length of the meaningful XORed value in the next 5
+      // bits. Finally store the meaningful bits of the XORed value.
+      if (trailingZeros > THRESHOLD) {
+        int significantBits = VALUE_BITS_LENGTH_32BIT - leadingZeros - trailingZeros;
+        writeBits(
+            256 * (PREVIOUS_VALUES + previousIndex)
+                + 32 * LEADING_REPRESENTATION[leadingZeros]
+                + significantBits,
+            CASE_ONE_METADATA_LENGTH,
+            out);
+        writeBits(xor >>> trailingZeros, significantBits, out); // Store the meaningful bits of XOR
+        storedLeadingZeros = VALUE_BITS_LENGTH_32BIT + 1;
+        // case 10: If the number of leading zeros is exactly
+        // equal to the previous leading zeros, use that information
+        // and just store 01 control bits and the meaningful XORed value.
+      } else if (leadingZeros == storedLeadingZeros) {
+        writeBit(out);
+        skipBit(out);
+        int significantBits = VALUE_BITS_LENGTH_32BIT - leadingZeros;
+        writeBits(xor, significantBits, out);
+        // case 11: store 11 control bits, the length of the number of leading
+        // zeros in the next 3 bits, then store the
+        // meaningful bits of the XORed value.
+      } else {
+        storedLeadingZeros = leadingZeros;
+        int significantBits = VALUE_BITS_LENGTH_32BIT - leadingZeros;
+        writeBits(24 + LEADING_REPRESENTATION[leadingZeros], 5, out);
+        writeBits(xor, significantBits, out);
+      }
+    }
+    current = (current + 1) % PREVIOUS_VALUES;
+    storedValues[current] = value;
+    index++;
+    indices[key] = index;
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/LongChimpEncoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/LongChimpEncoder.java
new file mode 100644
index 0000000000..170d34d8b2
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/LongChimpEncoder.java
@@ -0,0 +1,199 @@
+/*
+ * 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.encoding.encoder;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+
+import java.io.ByteArrayOutputStream;
+
+import static org.apache.iotdb.tsfile.common.conf.TSFileConfig.LEADING_ZERO_BITS_LENGTH_64BIT;
+import static org.apache.iotdb.tsfile.common.conf.TSFileConfig.MEANINGFUL_XOR_BITS_LENGTH_64BIT;
+import static org.apache.iotdb.tsfile.common.conf.TSFileConfig.VALUE_BITS_LENGTH_64BIT;
+
+/**
+ * This class includes code modified from Panagiotis Liakos chimp project.
+ *
+ * <p>Copyright: 2022- Panagiotis Liakos, Katia Papakonstantinopoulou and Yannis Kotidis
+ *
+ * <p>Project page: https://github.com/panagiotisl/chimp
+ *
+ * <p>License: http://www.apache.org/licenses/LICENSE-2.0
+ */
+public class LongChimpEncoder extends GorillaEncoderV2 {
+
+  private static final int PREVIOUS_VALUES = 128;
+  private static final int PREVIOUS_VALUES_LOG2 = (int) (Math.log(PREVIOUS_VALUES) / Math.log(2));
+  private static final int THRESHOLD = 6 + PREVIOUS_VALUES_LOG2;
+  private static final int SET_LSB = (int) Math.pow(2, THRESHOLD + 1) - 1;
+  private static final int CASE_ZERO_METADATA_LENGTH = PREVIOUS_VALUES_LOG2 + 2;
+  private static final int CASE_ONE_METADATA_LENGTH = PREVIOUS_VALUES_LOG2 + 11;
+  public static final short[] LEADING_REPRESENTATION = {
+    0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 2, 2, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 7, 7, 7, 7, 7, 7, 7,
+    7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+  };
+
+  public static final short[] LEADING_ROUND = {
+    0, 0, 0, 0, 0, 0, 0, 0, 8, 8, 8, 8, 12, 12, 12, 12, 16, 16, 18, 18, 20, 20, 22, 22, 24, 24, 24,
+    24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24,
+    24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24
+  };
+
+  private long storedValues[];
+  private int[] indices;
+  private int index = 0;
+  private int current = 0;
+
+  public LongChimpEncoder() {
+    this.setType(TSEncoding.CHIMP);
+    this.indices = new int[(int) Math.pow(2, THRESHOLD + 1)];
+    this.storedValues = new long[PREVIOUS_VALUES];
+  }
+
+  private static final int ONE_ITEM_MAX_SIZE =
+      (2
+                  + LEADING_ZERO_BITS_LENGTH_64BIT
+                  + MEANINGFUL_XOR_BITS_LENGTH_64BIT
+                  + VALUE_BITS_LENGTH_64BIT)
+              / Byte.SIZE
+          + 1;
+
+  public static final short[] leadingRepresentation = {
+    0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 2, 2, 2, 2, 3, 3, 4, 4, 5, 5, 6, 6, 7, 7, 7, 7, 7, 7, 7, 7,
+    7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7
+  };
+
+  public static final short[] leadingRound = {
+    0, 0, 0, 0, 0, 0, 0, 0, 8, 8, 8, 8, 12, 12, 12, 12, 16, 16, 18, 18, 20, 20, 22, 22, 24, 24, 24,
+    24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24,
+    24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24
+  };
+
+  @Override
+  public final int getOneItemMaxSize() {
+    return ONE_ITEM_MAX_SIZE;
+  }
+
+  @Override
+  protected void reset() {
+    super.reset();
+    this.current = 0;
+    this.index = 0;
+    this.indices = new int[(int) Math.pow(2, THRESHOLD + 1)];
+    this.storedValues = new long[PREVIOUS_VALUES];
+  }
+
+  @Override
+  public void flush(ByteArrayOutputStream out) {
+    // ending stream
+    encode(Long.MIN_VALUE, out);
+
+    // flip the byte no matter it is empty or not
+    // the empty ending byte is necessary when decoding
+    bitsLeft = 0;
+    flipByte(out);
+
+    // the encoder may be reused, so let us reset it
+    reset();
+  }
+
+  @Override
+  public final void encode(long value, ByteArrayOutputStream out) {
+    if (firstValueWasWritten) {
+      compressValue(value, out);
+    } else {
+      writeFirst(value, out);
+      firstValueWasWritten = true;
+    }
+  }
+
+  // the first value is stored with no compression
+  private void writeFirst(long value, ByteArrayOutputStream out) {
+    storedValues[current] = value;
+    writeBits(value, VALUE_BITS_LENGTH_64BIT, out);
+    indices[(int) value & SET_LSB] = index;
+  }
+
+  private void compressValue(long value, ByteArrayOutputStream out) {
+    // find the best previous value
+    int key = (int) value & SET_LSB;
+    long xor;
+    int previousIndex;
+    int trailingZeros = 0;
+    int currIndex = indices[key];
+    if ((index - currIndex) < PREVIOUS_VALUES) {
+      long tempXor = value ^ storedValues[currIndex % PREVIOUS_VALUES];
+      trailingZeros = Long.numberOfTrailingZeros(tempXor);
+      if (trailingZeros > THRESHOLD) {
+        previousIndex = currIndex % PREVIOUS_VALUES;
+        xor = tempXor;
+      } else {
+        previousIndex = index % PREVIOUS_VALUES;
+        xor = storedValues[previousIndex] ^ value;
+      }
+    } else {
+      previousIndex = index % PREVIOUS_VALUES;
+      xor = storedValues[previousIndex] ^ value;
+    }
+
+    // case 00: the values are identical, write 00 control bits
+    // and the index of the previous value
+    if (xor == 0) {
+      writeBits(previousIndex, CASE_ZERO_METADATA_LENGTH, out);
+      storedLeadingZeros = VALUE_BITS_LENGTH_64BIT + 1;
+    } else {
+      int leadingZeros = leadingRound[Long.numberOfLeadingZeros(xor)];
+      // case 01:  store the index, the length of
+      // the number of leading zeros in the next 3 bits, then store
+      // the length of the meaningful XORed value in the next 6
+      // bits. Finally store the meaningful bits of the XORed value.
+      if (trailingZeros > THRESHOLD) {
+        int significantBits = VALUE_BITS_LENGTH_64BIT - leadingZeros - trailingZeros;
+        writeBits(
+            512 * (PREVIOUS_VALUES + previousIndex)
+                + 64 * leadingRepresentation[leadingZeros]
+                + significantBits,
+            CASE_ONE_METADATA_LENGTH,
+            out);
+        writeBits(xor >>> trailingZeros, significantBits, out); // Store the meaningful bits of XOR
+        storedLeadingZeros = VALUE_BITS_LENGTH_64BIT + 1;
+        // case 10: If the number of leading zeros is exactly
+        // equal to the previous leading zeros, use that information
+        // and just store 01 control bits and the meaningful XORed value.
+      } else if (leadingZeros == storedLeadingZeros) {
+        writeBit(out);
+        skipBit(out);
+        int significantBits = VALUE_BITS_LENGTH_64BIT - leadingZeros;
+        writeBits(xor, significantBits, out);
+        // case 11: store 11 control bits, the length of the number of leading
+        // zeros in the next 3 bits, then store the
+        // meaningful bits of the XORed value.
+      } else {
+        storedLeadingZeros = leadingZeros;
+        int significantBits = VALUE_BITS_LENGTH_64BIT - leadingZeros;
+        writeBits(24 + leadingRepresentation[leadingZeros], 5, out);
+        writeBits(xor, significantBits, out);
+      }
+    }
+    current = (current + 1) % PREVIOUS_VALUES;
+    storedValues[current] = value;
+    index++;
+    indices[key] = index;
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/SinglePrecisionChimpEncoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/SinglePrecisionChimpEncoder.java
new file mode 100644
index 0000000000..b428c3dfe9
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/SinglePrecisionChimpEncoder.java
@@ -0,0 +1,55 @@
+/*
+ * 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.encoding.encoder;
+
+import java.io.ByteArrayOutputStream;
+
+/**
+ * This class includes code modified from Panagiotis Liakos chimp project.
+ *
+ * <p>Copyright: 2022- Panagiotis Liakos, Katia Papakonstantinopoulou and Yannis Kotidis
+ *
+ * <p>Project page: https://github.com/panagiotisl/chimp
+ *
+ * <p>License: http://www.apache.org/licenses/LICENSE-2.0
+ */
+public class SinglePrecisionChimpEncoder extends IntChimpEncoder {
+
+  private static final int CHIMP_ENCODING_ENDING = Float.floatToRawIntBits(Float.NaN);
+
+  @Override
+  public final void encode(float value, ByteArrayOutputStream out) {
+    encode(Float.floatToRawIntBits(value), out);
+  }
+
+  @Override
+  public void flush(ByteArrayOutputStream out) {
+    // ending stream
+    encode(CHIMP_ENCODING_ENDING, out);
+
+    // flip the byte no matter it is empty or not
+    // the empty ending byte is necessary when decoding
+    bitsLeft = 0;
+    flipByte(out);
+
+    // the encoder may be reused, so let us reset it
+    reset();
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/TSEncodingBuilder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/TSEncodingBuilder.java
index 913e23a699..4bd29a0e9f 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/TSEncodingBuilder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/TSEncodingBuilder.java
@@ -73,6 +73,8 @@ public abstract class TSEncodingBuilder {
         return new Freq();
       case ZIGZAG:
         return new Zigzag();
+      case CHIMP:
+        return new Chimp();
       default:
         throw new UnsupportedOperationException(type.toString());
     }
@@ -390,7 +392,7 @@ public abstract class TSEncodingBuilder {
         case INT64:
           return new LongZigzagEncoder();
         default:
-          throw new UnSupportedDataTypeException("GORILLA doesn't support data type: " + type);
+          throw new UnSupportedDataTypeException("ZIGZAG doesn't support data type: " + type);
       }
     }
 
@@ -399,4 +401,29 @@ public abstract class TSEncodingBuilder {
       // do nothing
     }
   }
+
+  /** for FLOAT, DOUBLE, INT, LONG. */
+  public static class Chimp extends TSEncodingBuilder {
+
+    @Override
+    public Encoder getEncoder(TSDataType type) {
+      switch (type) {
+        case FLOAT:
+          return new SinglePrecisionChimpEncoder();
+        case DOUBLE:
+          return new DoublePrecisionChimpEncoder();
+        case INT32:
+          return new IntChimpEncoder();
+        case INT64:
+          return new LongChimpEncoder();
+        default:
+          throw new UnSupportedDataTypeException("CHIMP doesn't support data type: " + type);
+      }
+    }
+
+    @Override
+    public void initFromProps(Map<String, String> props) {
+      // allowed do nothing
+    }
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSEncoding.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSEncoding.java
index 59f7573c38..aad9c01226 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSEncoding.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSEncoding.java
@@ -29,7 +29,8 @@ public enum TSEncoding {
   REGULAR((byte) 7),
   GORILLA((byte) 8),
   ZIGZAG((byte) 9),
-  FREQ((byte) 10);
+  FREQ((byte) 10),
+  CHIMP((byte) 11);
 
   private final byte type;
 
@@ -71,6 +72,8 @@ public enum TSEncoding {
         return TSEncoding.ZIGZAG;
       case 10:
         return TSEncoding.FREQ;
+      case 11:
+        return TSEncoding.CHIMP;
       default:
         throw new IllegalArgumentException("Invalid input: " + encoding);
     }
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/ChimpDecoderTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/ChimpDecoderTest.java
new file mode 100644
index 0000000000..5912e2893a
--- /dev/null
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/ChimpDecoderTest.java
@@ -0,0 +1,594 @@
+/*
+ * 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.encoding.decoder;
+
+import org.apache.iotdb.tsfile.encoding.encoder.DoublePrecisionChimpEncoder;
+import org.apache.iotdb.tsfile.encoding.encoder.Encoder;
+import org.apache.iotdb.tsfile.encoding.encoder.IntChimpEncoder;
+import org.apache.iotdb.tsfile.encoding.encoder.LongChimpEncoder;
+import org.apache.iotdb.tsfile.encoding.encoder.SinglePrecisionChimpEncoder;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class ChimpDecoderTest {
+
+  private static final double DELTA = 0;
+  private static final int FLOAT_MAX_POINT_VALUE = 10000;
+  private static final long DOUBLE_MAX_POINT_VALUE = 1000000000000000L;
+
+  private static final List<Float> floatList = new ArrayList<>();
+  private static final List<Double> doubleList = new ArrayList<>();
+  private static final List<Integer> intList = new ArrayList<>();
+  private static final List<Long> longList = new ArrayList<>();
+
+  private static final List<Integer> iterations = new ArrayList<>();
+
+  @BeforeClass
+  public static void setUp() {
+    int hybridCount = 11;
+    int hybridNum = 50;
+    int hybridStart = 2000;
+    for (int i = 0; i < hybridNum; i++) {
+      for (int j = 0; j < hybridCount; j++) {
+        floatList.add((float) hybridStart / FLOAT_MAX_POINT_VALUE);
+        doubleList.add((double) hybridStart / DOUBLE_MAX_POINT_VALUE);
+        intList.add(hybridStart);
+        longList.add((long) hybridStart);
+      }
+      for (int j = 0; j < hybridCount; j++) {
+        floatList.add((float) hybridStart / FLOAT_MAX_POINT_VALUE);
+        doubleList.add((double) hybridStart / DOUBLE_MAX_POINT_VALUE);
+        intList.add(hybridStart);
+        longList.add((long) hybridStart);
+        hybridStart += 3;
+      }
+      hybridCount += 2;
+    }
+
+    iterations.add(1);
+    iterations.add(3);
+    iterations.add(8);
+    iterations.add(16);
+    iterations.add(1000);
+    iterations.add(10000);
+  }
+
+  @Test
+  public void testIntSingleValue() throws IOException {
+    Encoder encoder = new IntChimpEncoder();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    encoder.encode(777, baos);
+    encoder.flush(baos);
+
+    ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+
+    Decoder decoder = new IntChimpDecoder();
+    if (decoder.hasNext(buffer)) {
+      assertEquals(777, decoder.readInt(buffer));
+    }
+    if (decoder.hasNext(buffer)) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testFloatSingleValue() throws IOException {
+    Encoder encoder = new SinglePrecisionChimpEncoder();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    encoder.encode(Float.MAX_VALUE, baos);
+    encoder.flush(baos);
+
+    ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+
+    Decoder decoder = new SinglePrecisionChimpDecoder();
+    if (decoder.hasNext(buffer)) {
+      assertEquals(Float.MAX_VALUE, decoder.readFloat(buffer), DELTA);
+    }
+    if (decoder.hasNext(buffer)) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testLongSingleValue() throws IOException {
+    Encoder encoder = new LongChimpEncoder();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    encoder.encode((long) Integer.MAX_VALUE + 10, baos);
+    encoder.flush(baos);
+
+    ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+
+    Decoder decoder = new LongChimpDecoder();
+    if (decoder.hasNext(buffer)) {
+      assertEquals((long) Integer.MAX_VALUE + 10, decoder.readLong(buffer));
+    }
+    if (decoder.hasNext(buffer)) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testDoubleSingleValue() throws IOException {
+    Encoder encoder = new DoublePrecisionChimpEncoder();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    encoder.encode(Double.MAX_VALUE, baos);
+    encoder.flush(baos);
+
+    ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+
+    Decoder decoder = new DoublePrecisionChimpDecoder();
+    if (decoder.hasNext(buffer)) {
+      assertEquals(Double.MAX_VALUE, decoder.readDouble(buffer), DELTA);
+    }
+    if (decoder.hasNext(buffer)) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testIntZeroNumber() throws IOException {
+    Encoder encoder = new IntChimpEncoder();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    int value = 0;
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.flush(baos);
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.flush(baos);
+    ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+    for (int i = 0; i < 2; i++) {
+      Decoder decoder = new IntChimpDecoder();
+      if (decoder.hasNext(buffer)) {
+        assertEquals(value, decoder.readInt(buffer), DELTA);
+      }
+      if (decoder.hasNext(buffer)) {
+        assertEquals(value, decoder.readInt(buffer), DELTA);
+      }
+      if (decoder.hasNext(buffer)) {
+        assertEquals(value, decoder.readInt(buffer), DELTA);
+      }
+    }
+  }
+
+  @Test
+  public void testFloatZeroNumber() throws IOException {
+    Encoder encoder = new SinglePrecisionChimpEncoder();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    float value = 0f;
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.flush(baos);
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.flush(baos);
+    ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+    for (int i = 0; i < 2; i++) {
+      Decoder decoder = new SinglePrecisionChimpDecoder();
+      if (decoder.hasNext(buffer)) {
+        assertEquals(value, decoder.readFloat(buffer), DELTA);
+      }
+      if (decoder.hasNext(buffer)) {
+        assertEquals(value, decoder.readFloat(buffer), DELTA);
+      }
+      if (decoder.hasNext(buffer)) {
+        assertEquals(value, decoder.readFloat(buffer), DELTA);
+      }
+    }
+  }
+
+  @Test
+  public void testLongZeroNumber() throws IOException {
+    Encoder encoder = new LongChimpEncoder();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    long value = 0;
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.flush(baos);
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.flush(baos);
+    ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+    for (int i = 0; i < 2; i++) {
+      Decoder decoder = new LongChimpDecoder();
+      if (decoder.hasNext(buffer)) {
+        assertEquals(value, decoder.readLong(buffer), DELTA);
+      }
+      if (decoder.hasNext(buffer)) {
+        assertEquals(value, decoder.readLong(buffer), DELTA);
+      }
+      if (decoder.hasNext(buffer)) {
+        assertEquals(value, decoder.readLong(buffer), DELTA);
+      }
+    }
+  }
+
+  @Test
+  public void testDoubleZeroNumber() throws IOException {
+    Encoder encoder = new DoublePrecisionChimpEncoder();
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    double value = 0f;
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.flush(baos);
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.encode(value, baos);
+    encoder.flush(baos);
+    ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+    for (int i = 0; i < 2; i++) {
+      Decoder decoder = new DoublePrecisionChimpDecoder();
+      if (decoder.hasNext(buffer)) {
+        assertEquals(value, decoder.readDouble(buffer), DELTA);
+      }
+      if (decoder.hasNext(buffer)) {
+        assertEquals(value, decoder.readDouble(buffer), DELTA);
+      }
+      if (decoder.hasNext(buffer)) {
+        assertEquals(value, decoder.readDouble(buffer), DELTA);
+      }
+    }
+  }
+
+  @Test
+  public void testInteger() throws IOException {
+    for (Integer num : iterations) {
+      Encoder encoder = new IntChimpEncoder();
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      int value = 7;
+      for (int i = 0; i < num; i++) {
+        encoder.encode(value + 2 * i, baos);
+      }
+      encoder.flush(baos);
+
+      ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+
+      Decoder decoder = new IntChimpDecoder();
+      for (int i = 0; i < num; i++) {
+        if (decoder.hasNext(buffer)) {
+          assertEquals(value + 2 * i, decoder.readInt(buffer));
+          continue;
+        }
+        fail();
+      }
+      if (decoder.hasNext(buffer)) {
+        fail();
+      }
+    }
+  }
+
+  @Test
+  public void testFloat() throws IOException {
+    for (Integer num : iterations) {
+      Encoder encoder = new SinglePrecisionChimpEncoder();
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      float value = 7.101f;
+      for (int i = 0; i < num; i++) {
+        encoder.encode(value + 2 * i, baos);
+      }
+      encoder.flush(baos);
+
+      ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+
+      Decoder decoder = new SinglePrecisionChimpDecoder();
+      for (int i = 0; i < num; i++) {
+        if (decoder.hasNext(buffer)) {
+          assertEquals(value + 2 * i, decoder.readFloat(buffer), DELTA);
+          continue;
+        }
+        fail();
+      }
+      if (decoder.hasNext(buffer)) {
+        fail();
+      }
+    }
+  }
+
+  @Test
+  public void testLong() throws IOException {
+    for (Integer num : iterations) {
+      Encoder encoder = new LongChimpEncoder();
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      long value = 7;
+      for (int i = 0; i < num; i++) {
+        encoder.encode(value + 2 * i, baos);
+      }
+      encoder.flush(baos);
+
+      ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+
+      Decoder decoder = new LongChimpDecoder();
+      for (int i = 0; i < num; i++) {
+        if (decoder.hasNext(buffer)) {
+          long temp = decoder.readLong(buffer);
+          assertEquals(value + 2 * i, temp);
+          continue;
+        }
+        fail();
+      }
+      if (decoder.hasNext(buffer)) {
+        fail();
+      }
+    }
+  }
+
+  @Test
+  public void testDouble() throws IOException {
+    for (Integer num : iterations) {
+      Encoder encoder = new DoublePrecisionChimpEncoder();
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      double value = 7.101f;
+      for (int i = 0; i < num; i++) {
+        encoder.encode(value + 2 * i, baos);
+      }
+      encoder.flush(baos);
+
+      ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+
+      Decoder decoder = new DoublePrecisionChimpDecoder();
+      for (int i = 0; i < num; i++) {
+        if (decoder.hasNext(buffer)) {
+          assertEquals(value + 2 * i, decoder.readDouble(buffer), DELTA);
+          continue;
+        }
+        fail();
+      }
+      if (decoder.hasNext(buffer)) {
+        fail();
+      }
+    }
+  }
+
+  @Test
+  public void testIntegerRepeat() throws Exception {
+    for (int i = 1; i <= 10; i++) {
+      testInteger(i);
+    }
+  }
+
+  private void testInteger(int repeatCount) throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    Encoder encoder = new IntChimpEncoder();
+    for (int i = 0; i < repeatCount; i++) {
+      for (int value : ChimpDecoderTest.intList) {
+        encoder.encode(value, baos);
+      }
+      encoder.flush(baos);
+    }
+
+    ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+
+    for (int i = 0; i < repeatCount; i++) {
+      Decoder decoder = new IntChimpDecoder();
+      for (int expected : ChimpDecoderTest.intList) {
+        if (decoder.hasNext(buffer)) {
+          int actual = decoder.readInt(buffer);
+          assertEquals(expected, actual);
+          continue;
+        }
+        fail();
+      }
+    }
+
+    baos = new ByteArrayOutputStream();
+    encoder = new IntChimpEncoder();
+    for (int i = 0; i < repeatCount; i++) {
+      for (int value : ChimpDecoderTest.intList) {
+        encoder.encode(-value, baos);
+      }
+      encoder.flush(baos);
+    }
+
+    buffer = ByteBuffer.wrap(baos.toByteArray());
+
+    for (int i = 0; i < repeatCount; i++) {
+      Decoder decoder = new IntChimpDecoder();
+      for (int expected : ChimpDecoderTest.intList) {
+        if (decoder.hasNext(buffer)) {
+          int actual = decoder.readInt(buffer);
+          assertEquals(expected, -actual);
+          continue;
+        }
+        fail();
+      }
+    }
+  }
+
+  @Test
+  public void testFloatRepeat() throws Exception {
+    for (int i = 1; i <= 10; i++) {
+      testFloat(i);
+    }
+  }
+
+  private void testFloat(int repeatCount) throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    Encoder encoder = new SinglePrecisionChimpEncoder();
+    for (int i = 0; i < repeatCount; i++) {
+      for (float value : ChimpDecoderTest.floatList) {
+        encoder.encode(value, baos);
+      }
+      encoder.flush(baos);
+    }
+
+    ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+
+    for (int i = 0; i < repeatCount; i++) {
+      Decoder decoder = new SinglePrecisionChimpDecoder();
+      for (float expected : ChimpDecoderTest.floatList) {
+        if (decoder.hasNext(buffer)) {
+          float actual = decoder.readFloat(buffer);
+          assertEquals(expected, actual, DELTA);
+          continue;
+        }
+        fail();
+      }
+    }
+
+    baos = new ByteArrayOutputStream();
+    encoder = new SinglePrecisionChimpEncoder();
+    for (int i = 0; i < repeatCount; i++) {
+      for (float value : ChimpDecoderTest.floatList) {
+        encoder.encode(-value, baos);
+      }
+      encoder.flush(baos);
+    }
+
+    buffer = ByteBuffer.wrap(baos.toByteArray());
+
+    for (int i = 0; i < repeatCount; i++) {
+      Decoder decoder = new SinglePrecisionChimpDecoder();
+      for (float expected : ChimpDecoderTest.floatList) {
+        if (decoder.hasNext(buffer)) {
+          float actual = decoder.readFloat(buffer);
+          assertEquals(expected, -actual, DELTA);
+          continue;
+        }
+        fail();
+      }
+    }
+  }
+
+  @Test
+  public void testLongRepeat() throws Exception {
+    for (int i = 1; i <= 10; i++) {
+      testLong(i);
+    }
+  }
+
+  private void testLong(int repeatCount) throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    Encoder encoder = new LongChimpEncoder();
+    for (int i = 0; i < repeatCount; i++) {
+      for (long value : ChimpDecoderTest.longList) {
+        encoder.encode(value, baos);
+      }
+      encoder.flush(baos);
+    }
+
+    ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+
+    for (int i = 0; i < repeatCount; i++) {
+      Decoder decoder = new LongChimpDecoder();
+      for (long expected : ChimpDecoderTest.longList) {
+        if (decoder.hasNext(buffer)) {
+          long actual = decoder.readLong(buffer);
+          assertEquals(expected, actual);
+          continue;
+        }
+        fail();
+      }
+    }
+
+    baos = new ByteArrayOutputStream();
+    for (int i = 0; i < repeatCount; i++) {
+      encoder = new LongChimpEncoder();
+      for (long value : ChimpDecoderTest.longList) {
+        encoder.encode(-value, baos);
+      }
+      encoder.flush(baos);
+    }
+
+    buffer = ByteBuffer.wrap(baos.toByteArray());
+
+    for (int i = 0; i < repeatCount; i++) {
+      Decoder decoder = new LongChimpDecoder();
+      for (long expected : ChimpDecoderTest.longList) {
+        if (decoder.hasNext(buffer)) {
+          long actual = decoder.readLong(buffer);
+          assertEquals(expected, -actual);
+          continue;
+        }
+        fail();
+      }
+    }
+  }
+
+  @Test
+  public void testDoubleRepeat() throws Exception {
+    for (int i = 1; i <= 10; i++) {
+      testDouble(i);
+    }
+  }
+
+  private void testDouble(int repeatCount) throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    Encoder encoder = new DoublePrecisionChimpEncoder();
+    for (int i = 0; i < repeatCount; i++) {
+      for (double value : ChimpDecoderTest.doubleList) {
+        encoder.encode(value, baos);
+      }
+      encoder.flush(baos);
+    }
+
+    ByteBuffer buffer = ByteBuffer.wrap(baos.toByteArray());
+
+    for (int i = 0; i < repeatCount; i++) {
+      Decoder decoder = new DoublePrecisionChimpDecoder();
+      for (double expected : ChimpDecoderTest.doubleList) {
+        if (decoder.hasNext(buffer)) {
+          double actual = decoder.readDouble(buffer);
+          assertEquals(expected, actual, DELTA);
+          continue;
+        }
+        fail();
+      }
+    }
+    baos = new ByteArrayOutputStream();
+    encoder = new DoublePrecisionChimpEncoder();
+    for (int i = 0; i < repeatCount; i++) {
+      for (double value : ChimpDecoderTest.doubleList) {
+        encoder.encode(-value, baos);
+      }
+      encoder.flush(baos);
+    }
+
+    buffer = ByteBuffer.wrap(baos.toByteArray());
+
+    for (int i = 0; i < repeatCount; i++) {
+      Decoder decoder = new DoublePrecisionChimpDecoder();
+      for (double expected : ChimpDecoderTest.doubleList) {
+        if (decoder.hasNext(buffer)) {
+          double actual = decoder.readDouble(buffer);
+          assertEquals(expected, -actual, DELTA);
+          continue;
+        }
+        fail();
+      }
+    }
+  }
+}