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/10/28 13:37:43 UTC

[GitHub] [iotdb] 5Reasons opened a new pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

5Reasons opened a new pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890


   In the enum class: TSEncoding, there's  DIFF encoding, so when user want to choose a type of encoding algorithm(or input the wrong encoding type name), DIFF encoding is in the output hint and is one of IoTDB's recommending encoding type, **but in IoTDB, DIFF encoding has not been added before,**which I think may cause some problems. I think it's necessary to finish it.
   Both DIFF and TS_2DIFF encoding is based on delta encoding algorithm, so I finish DIFF using the similar interfaces and structure like TS_2DIFF to make it easier to understand.


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



[GitHub] [iotdb] kr11 commented on a change in pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
kr11 commented on a change in pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#discussion_r514743214



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DiffEncoder.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * <p>
+ * DiffEncoder is a encoder for compressing data in type of INT32(integer) and
+ * INT64(long). It is based in delta-encoding arithmetic.We adapt a hypothesis
+ * that contiguous data points have similar values.Thus the difference value of
+ * two adjacent points is smaller than those two point values. One integer in
+ * java takes 32-bits. If a positive number is less than 2^m, the bits of this
+ * integer which index from m to 31 are all 0. Given an array which length is n,
+ * if all values in input data array are all positive and less than 2^m, we need
+ * actually m*n, but not 32*n bits to store the array.
+ * </p>
+ * <p>
+ * DiffEncoder calculates difference between two adjacent. Then it saves the delta
+ * value. Then it statistics the longest bit length {@code m} it takes for each
+ * delta value, which means the bit length that maximum delta value takes. Only
+ * the low m bits are saved into result byte array for all delta values.

Review comment:
       m -> {@code m}
   It's better to be consistent with the above.




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



[GitHub] [iotdb] coveralls edited a comment on pull request #1890: [IOTDB-1597] Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#issuecomment-908897698


   
   [![Coverage Status](https://coveralls.io/builds/42529164/badge)](https://coveralls.io/builds/42529164)
   
   Coverage increased (+0.009%) to 67.371% when pulling **df32a0e59cc4c74cfd469043a8b4a7ba7858117e on 5Reasons:addDiff** into **fb18357edee14e60893e968aeddc3177fba868bc on apache:master**.
   


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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls commented on pull request #1890: [IOTDB-1597] Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
coveralls commented on pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#issuecomment-908897698


   
   [![Coverage Status](https://coveralls.io/builds/42528841/badge)](https://coveralls.io/builds/42528841)
   
   Coverage increased (+0.02%) to 67.386% when pulling **fd3813587c5c804bd614a86f37ef79df20376b00 on 5Reasons:addDiff** into **fb18357edee14e60893e968aeddc3177fba868bc on apache:master**.
   


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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] kr11 commented on a change in pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
kr11 commented on a change in pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#discussion_r514739053



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DiffEncoder.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * <p>
+ * DiffEncoder is a encoder for compressing data in type of INT32(integer) and
+ * INT64(long). It is based in delta-encoding arithmetic.We adapt a hypothesis
+ * that contiguous data points have similar values.Thus the difference value of
+ * two adjacent points is smaller than those two point values. One integer in
+ * java takes 32-bits. If a positive number is less than 2^m, the bits of this
+ * integer which index from m to 31 are all 0. Given an array which length is n,
+ * if all values in input data array are all positive and less than 2^m, we need
+ * actually m*n, but not 32*n bits to store the array.
+ * </p>
+ * <p>
+ * DiffEncoder calculates difference between two adjacent. Then it saves the delta
+ * value. Then it statistics the longest bit length {@code m} it takes for each

Review comment:
       maybe "values" better.
   Then it statistics ... -> Then it counts ... 




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



[GitHub] [iotdb] JackieTien97 commented on pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
JackieTien97 commented on pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#issuecomment-744143853


   Hi, please resolve the conflicts


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



[GitHub] [iotdb] jixuan1989 commented on pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
jixuan1989 commented on pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#issuecomment-720874325






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



[GitHub] [iotdb] kr11 commented on a change in pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
kr11 commented on a change in pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#discussion_r514736515



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DiffEncoder.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * <p>
+ * DiffEncoder is a encoder for compressing data in type of INT32(integer) and
+ * INT64(long). It is based in delta-encoding arithmetic.We adapt a hypothesis

Review comment:
       arithmetic.We adapt -> arithmetic. We adapt 
   Add space.




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



[GitHub] [iotdb] coveralls edited a comment on pull request #1890: [IOTDB-1597] Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#issuecomment-908897698






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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #1890: [IOTDB-1597] Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#issuecomment-908897698


   
   [![Coverage Status](https://coveralls.io/builds/42538060/badge)](https://coveralls.io/builds/42538060)
   
   Coverage increased (+0.01%) to 67.374% when pulling **d9881b2954a949ce70e22ffee979caad67757449 on 5Reasons:addDiff** into **fb18357edee14e60893e968aeddc3177fba868bc on apache:master**.
   


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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] kr11 commented on a change in pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
kr11 commented on a change in pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#discussion_r514739053



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DiffEncoder.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * <p>
+ * DiffEncoder is a encoder for compressing data in type of INT32(integer) and
+ * INT64(long). It is based in delta-encoding arithmetic.We adapt a hypothesis
+ * that contiguous data points have similar values.Thus the difference value of
+ * two adjacent points is smaller than those two point values. One integer in
+ * java takes 32-bits. If a positive number is less than 2^m, the bits of this
+ * integer which index from m to 31 are all 0. Given an array which length is n,
+ * if all values in input data array are all positive and less than 2^m, we need
+ * actually m*n, but not 32*n bits to store the array.
+ * </p>
+ * <p>
+ * DiffEncoder calculates difference between two adjacent. Then it saves the delta
+ * value. Then it statistics the longest bit length {@code m} it takes for each

Review comment:
       maybe "values" better




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



[GitHub] [iotdb] 5Reasons commented on a change in pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
5Reasons commented on a change in pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#discussion_r514885548



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DiffEncoder.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * <p>
+ * DiffEncoder is a encoder for compressing data in type of INT32(integer) and
+ * INT64(long). It is based in delta-encoding arithmetic.We adapt a hypothesis
+ * that contiguous data points have similar values.Thus the difference value of
+ * two adjacent points is smaller than those two point values. One integer in
+ * java takes 32-bits. If a positive number is less than 2^m, the bits of this
+ * integer which index from m to 31 are all 0. Given an array which length is n,
+ * if all values in input data array are all positive and less than 2^m, we need
+ * actually m*n, but not 32*n bits to store the array.
+ * </p>
+ * <p>
+ * DiffEncoder calculates difference between two adjacent. Then it saves the delta
+ * value. Then it statistics the longest bit length {@code m} it takes for each
+ * delta value, which means the bit length that maximum delta value takes. Only
+ * the low m bits are saved into result byte array for all delta values.
+ * </p>
+ */
+public abstract class DiffEncoder extends Encoder {
+
+    protected static final int BLOCK_DEFAULT_SIZE = 128;
+    private static final Logger logger = LoggerFactory.getLogger(DeltaBinaryEncoder.class);
+    protected ByteArrayOutputStream out;
+    protected int blockSize;
+    // input value is stored in deltaBlackBuffer temporarily
+    protected byte[] encodingBlockBuffer;
+
+    protected int writeIndex = -1;//defalut value?
+    protected int writeWidth = 0;
+
+    /**
+     * constructor of DeltaBinaryEncoder.

Review comment:
       I have fixed them : ) and I will pay more attention on annotation errors in the future




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



[GitHub] [iotdb] coveralls edited a comment on pull request #1890: [IOTDB-1597] Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#issuecomment-908897698


   
   [![Coverage Status](https://coveralls.io/builds/42529098/badge)](https://coveralls.io/builds/42529098)
   
   Coverage increased (+0.03%) to 67.393% when pulling **df32a0e59cc4c74cfd469043a8b4a7ba7858117e on 5Reasons:addDiff** into **fb18357edee14e60893e968aeddc3177fba868bc on apache:master**.
   


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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] kr11 commented on a change in pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
kr11 commented on a change in pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#discussion_r514745333



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DiffEncoder.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * <p>
+ * DiffEncoder is a encoder for compressing data in type of INT32(integer) and
+ * INT64(long). It is based in delta-encoding arithmetic.We adapt a hypothesis
+ * that contiguous data points have similar values.Thus the difference value of
+ * two adjacent points is smaller than those two point values. One integer in
+ * java takes 32-bits. If a positive number is less than 2^m, the bits of this
+ * integer which index from m to 31 are all 0. Given an array which length is n,
+ * if all values in input data array are all positive and less than 2^m, we need
+ * actually m*n, but not 32*n bits to store the array.
+ * </p>
+ * <p>
+ * DiffEncoder calculates difference between two adjacent. Then it saves the delta
+ * value. Then it statistics the longest bit length {@code m} it takes for each
+ * delta value, which means the bit length that maximum delta value takes. Only
+ * the low m bits are saved into result byte array for all delta values.
+ * </p>
+ */
+public abstract class DiffEncoder extends Encoder {
+
+    protected static final int BLOCK_DEFAULT_SIZE = 128;
+    private static final Logger logger = LoggerFactory.getLogger(DeltaBinaryEncoder.class);

Review comment:
       The logger should be initialized by `DiffEncoder`, not another class.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #1890: [IOTDB-1597] Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#issuecomment-908897698


   
   [![Coverage Status](https://coveralls.io/builds/42529473/badge)](https://coveralls.io/builds/42529473)
   
   Coverage increased (+0.02%) to 67.38% when pulling **df32a0e59cc4c74cfd469043a8b4a7ba7858117e on 5Reasons:addDiff** into **fb18357edee14e60893e968aeddc3177fba868bc on apache:master**.
   


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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls commented on pull request #1890: [IOTDB-1597] Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
coveralls commented on pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#issuecomment-908897698


   
   [![Coverage Status](https://coveralls.io/builds/42528841/badge)](https://coveralls.io/builds/42528841)
   
   Coverage increased (+0.02%) to 67.386% when pulling **fd3813587c5c804bd614a86f37ef79df20376b00 on 5Reasons:addDiff** into **fb18357edee14e60893e968aeddc3177fba868bc on apache:master**.
   


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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] HTHou commented on a change in pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#discussion_r698929403



##########
File path: tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/delta/DeltaBinaryEncoderIntegerTest.java
##########
@@ -96,15 +96,24 @@ private void writeData(int[] data, int length) {
   }
 
   private void shouldReadAndWrite(int[] data, int length) throws IOException {
-    // System.out.println("source data size:" + 4 * length + " byte");
+    System.out.println("source data size:" + 4 * length + " byte");
     out = new ByteArrayOutputStream();
+
+    long encodeStart=System.nanoTime();

Review comment:
       ```suggestion
   ```

##########
File path: tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/delta/DeltaBinaryEncoderIntegerTest.java
##########
@@ -96,15 +96,24 @@ private void writeData(int[] data, int length) {
   }
 
   private void shouldReadAndWrite(int[] data, int length) throws IOException {
-    // System.out.println("source data size:" + 4 * length + " byte");
+    System.out.println("source data size:" + 4 * length + " byte");
     out = new ByteArrayOutputStream();
+
+    long encodeStart=System.nanoTime();
     writeData(data, length);
+    Long encodeEnd=System.nanoTime();
+    System.out.println("encode take(ns): "+(encodeEnd-encodeStart));
+

Review comment:
       
   ```suggestion
   ```

##########
File path: tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/delta/DeltaBinaryEncoderIntegerTest.java
##########
@@ -96,15 +96,24 @@ private void writeData(int[] data, int length) {
   }
 
   private void shouldReadAndWrite(int[] data, int length) throws IOException {
-    // System.out.println("source data size:" + 4 * length + " byte");
+    System.out.println("source data size:" + 4 * length + " byte");

Review comment:
       ```suggestion
   ```

##########
File path: tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/delta/DeltaBinaryEncoderIntegerTest.java
##########
@@ -96,15 +96,24 @@ private void writeData(int[] data, int length) {
   }
 
   private void shouldReadAndWrite(int[] data, int length) throws IOException {
-    // System.out.println("source data size:" + 4 * length + " byte");
+    System.out.println("source data size:" + 4 * length + " byte");
     out = new ByteArrayOutputStream();
+
+    long encodeStart=System.nanoTime();
     writeData(data, length);
+    Long encodeEnd=System.nanoTime();
+    System.out.println("encode take(ns): "+(encodeEnd-encodeStart));
+
     byte[] page = out.toByteArray();
-    // System.out.println("encoding data size:" + page.length + " byte");
+    System.out.println("encoding data size:" + page.length + " byte");
     buffer = ByteBuffer.wrap(page);
     int i = 0;
+
+    Long decodeStart=System.nanoTime();
     while (reader.hasNext(buffer)) {
       assertEquals(data[i++], reader.readInt(buffer));
     }
+    Long decodeEnd=System.nanoTime();
+    System.out.println("decode take(ns): "+(decodeEnd-decodeStart));

Review comment:
       ```suggestion
   ```

##########
File path: tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/delta/DeltaBinaryEncoderIntegerTest.java
##########
@@ -96,15 +96,24 @@ private void writeData(int[] data, int length) {
   }
 
   private void shouldReadAndWrite(int[] data, int length) throws IOException {
-    // System.out.println("source data size:" + 4 * length + " byte");
+    System.out.println("source data size:" + 4 * length + " byte");
     out = new ByteArrayOutputStream();
+
+    long encodeStart=System.nanoTime();
     writeData(data, length);
+    Long encodeEnd=System.nanoTime();
+    System.out.println("encode take(ns): "+(encodeEnd-encodeStart));
+
     byte[] page = out.toByteArray();
-    // System.out.println("encoding data size:" + page.length + " byte");
+    System.out.println("encoding data size:" + page.length + " byte");
     buffer = ByteBuffer.wrap(page);
     int i = 0;
+
+    Long decodeStart=System.nanoTime();

Review comment:
       ```suggestion
   ```

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java
##########
@@ -112,6 +112,16 @@ public static Decoder getDecoderByType(TSEncoding encoding, TSDataType dataType)
         }
       case DICTIONARY:
         return new DictionaryDecoder();
+      case TS_2DIFF:

Review comment:
       ```suggestion
         case DIFF:
   ```

##########
File path: tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/delta/DeltaBinaryEncoderIntegerTest.java
##########
@@ -96,15 +96,24 @@ private void writeData(int[] data, int length) {
   }
 
   private void shouldReadAndWrite(int[] data, int length) throws IOException {
-    // System.out.println("source data size:" + 4 * length + " byte");
+    System.out.println("source data size:" + 4 * length + " byte");
     out = new ByteArrayOutputStream();
+
+    long encodeStart=System.nanoTime();
     writeData(data, length);
+    Long encodeEnd=System.nanoTime();
+    System.out.println("encode take(ns): "+(encodeEnd-encodeStart));
+
     byte[] page = out.toByteArray();
-    // System.out.println("encoding data size:" + page.length + " byte");
+    System.out.println("encoding data size:" + page.length + " byte");

Review comment:
       ```suggestion
   ```




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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] 5Reasons commented on pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
5Reasons commented on pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#issuecomment-720967886


   > 1. there are too many duplications between `DIFFEncoder` and `DeltaBinaryEncoder`. Better to inherit `DeltaBinaryEncoder`, rather than copy codes.
   > 2. What is the benefit to add such a method? Trying to give some experimental results to show when should we use it...
   
   I have made some experiment abouot it(https://cwiki.apache.org/confluence/display/IOTDB/DIFF+encoding%27s+perfomance).
   As I said in email-list, this method's performance is not better than TS_2DIFF (included in 'DeltaBinaryEncoder'), only when the data set is in best circumstances, the new method can save very little room (about 4 bytes, in head)
   
   I add it beacuse DIFF is a reserved method, it's one of the recommending method when user want to choose a encoding method in IoTDB, but it's not finished before. It's in the system prompt, but user can't chooose it
   ![system_prompt](https://user-images.githubusercontent.com/61086064/97960647-d05a5380-1dec-11eb-8bba-b75eb9f0198f.png)


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



[GitHub] [iotdb] kr11 commented on a change in pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
kr11 commented on a change in pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#discussion_r514751841



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DiffEncoder.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * <p>
+ * DiffEncoder is a encoder for compressing data in type of INT32(integer) and
+ * INT64(long). It is based in delta-encoding arithmetic.We adapt a hypothesis
+ * that contiguous data points have similar values.Thus the difference value of
+ * two adjacent points is smaller than those two point values. One integer in
+ * java takes 32-bits. If a positive number is less than 2^m, the bits of this
+ * integer which index from m to 31 are all 0. Given an array which length is n,
+ * if all values in input data array are all positive and less than 2^m, we need
+ * actually m*n, but not 32*n bits to store the array.
+ * </p>
+ * <p>
+ * DiffEncoder calculates difference between two adjacent. Then it saves the delta
+ * value. Then it statistics the longest bit length {@code m} it takes for each
+ * delta value, which means the bit length that maximum delta value takes. Only
+ * the low m bits are saved into result byte array for all delta values.
+ * </p>
+ */
+public abstract class DiffEncoder extends Encoder {
+
+    protected static final int BLOCK_DEFAULT_SIZE = 128;
+    private static final Logger logger = LoggerFactory.getLogger(DeltaBinaryEncoder.class);
+    protected ByteArrayOutputStream out;
+    protected int blockSize;
+    // input value is stored in deltaBlackBuffer temporarily
+    protected byte[] encodingBlockBuffer;
+
+    protected int writeIndex = -1;//defalut value?
+    protected int writeWidth = 0;
+
+    /**
+     * constructor of DeltaBinaryEncoder.

Review comment:
       All `DeltaBinaryEncoder` should be removed carefully before it's passed. 




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



[GitHub] [iotdb] coveralls edited a comment on pull request #1890: [IOTDB-1597] Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#issuecomment-908897698


   
   [![Coverage Status](https://coveralls.io/builds/42529065/badge)](https://coveralls.io/builds/42529065)
   
   Coverage increased (+0.01%) to 67.374% when pulling **df32a0e59cc4c74cfd469043a8b4a7ba7858117e on 5Reasons:addDiff** into **fb18357edee14e60893e968aeddc3177fba868bc on apache:master**.
   


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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] HTHou commented on a change in pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#discussion_r698929403



##########
File path: tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/delta/DeltaBinaryEncoderIntegerTest.java
##########
@@ -96,15 +96,24 @@ private void writeData(int[] data, int length) {
   }
 
   private void shouldReadAndWrite(int[] data, int length) throws IOException {
-    // System.out.println("source data size:" + 4 * length + " byte");
+    System.out.println("source data size:" + 4 * length + " byte");
     out = new ByteArrayOutputStream();
+
+    long encodeStart=System.nanoTime();

Review comment:
       ```suggestion
   ```

##########
File path: tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/delta/DeltaBinaryEncoderIntegerTest.java
##########
@@ -96,15 +96,24 @@ private void writeData(int[] data, int length) {
   }
 
   private void shouldReadAndWrite(int[] data, int length) throws IOException {
-    // System.out.println("source data size:" + 4 * length + " byte");
+    System.out.println("source data size:" + 4 * length + " byte");
     out = new ByteArrayOutputStream();
+
+    long encodeStart=System.nanoTime();
     writeData(data, length);
+    Long encodeEnd=System.nanoTime();
+    System.out.println("encode take(ns): "+(encodeEnd-encodeStart));
+

Review comment:
       
   ```suggestion
   ```

##########
File path: tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/delta/DeltaBinaryEncoderIntegerTest.java
##########
@@ -96,15 +96,24 @@ private void writeData(int[] data, int length) {
   }
 
   private void shouldReadAndWrite(int[] data, int length) throws IOException {
-    // System.out.println("source data size:" + 4 * length + " byte");
+    System.out.println("source data size:" + 4 * length + " byte");

Review comment:
       ```suggestion
   ```

##########
File path: tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/delta/DeltaBinaryEncoderIntegerTest.java
##########
@@ -96,15 +96,24 @@ private void writeData(int[] data, int length) {
   }
 
   private void shouldReadAndWrite(int[] data, int length) throws IOException {
-    // System.out.println("source data size:" + 4 * length + " byte");
+    System.out.println("source data size:" + 4 * length + " byte");
     out = new ByteArrayOutputStream();
+
+    long encodeStart=System.nanoTime();
     writeData(data, length);
+    Long encodeEnd=System.nanoTime();
+    System.out.println("encode take(ns): "+(encodeEnd-encodeStart));
+
     byte[] page = out.toByteArray();
-    // System.out.println("encoding data size:" + page.length + " byte");
+    System.out.println("encoding data size:" + page.length + " byte");
     buffer = ByteBuffer.wrap(page);
     int i = 0;
+
+    Long decodeStart=System.nanoTime();
     while (reader.hasNext(buffer)) {
       assertEquals(data[i++], reader.readInt(buffer));
     }
+    Long decodeEnd=System.nanoTime();
+    System.out.println("decode take(ns): "+(decodeEnd-decodeStart));

Review comment:
       ```suggestion
   ```

##########
File path: tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/delta/DeltaBinaryEncoderIntegerTest.java
##########
@@ -96,15 +96,24 @@ private void writeData(int[] data, int length) {
   }
 
   private void shouldReadAndWrite(int[] data, int length) throws IOException {
-    // System.out.println("source data size:" + 4 * length + " byte");
+    System.out.println("source data size:" + 4 * length + " byte");
     out = new ByteArrayOutputStream();
+
+    long encodeStart=System.nanoTime();
     writeData(data, length);
+    Long encodeEnd=System.nanoTime();
+    System.out.println("encode take(ns): "+(encodeEnd-encodeStart));
+
     byte[] page = out.toByteArray();
-    // System.out.println("encoding data size:" + page.length + " byte");
+    System.out.println("encoding data size:" + page.length + " byte");
     buffer = ByteBuffer.wrap(page);
     int i = 0;
+
+    Long decodeStart=System.nanoTime();

Review comment:
       ```suggestion
   ```

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java
##########
@@ -112,6 +112,16 @@ public static Decoder getDecoderByType(TSEncoding encoding, TSDataType dataType)
         }
       case DICTIONARY:
         return new DictionaryDecoder();
+      case TS_2DIFF:

Review comment:
       ```suggestion
         case DIFF:
   ```

##########
File path: tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/delta/DeltaBinaryEncoderIntegerTest.java
##########
@@ -96,15 +96,24 @@ private void writeData(int[] data, int length) {
   }
 
   private void shouldReadAndWrite(int[] data, int length) throws IOException {
-    // System.out.println("source data size:" + 4 * length + " byte");
+    System.out.println("source data size:" + 4 * length + " byte");
     out = new ByteArrayOutputStream();
+
+    long encodeStart=System.nanoTime();
     writeData(data, length);
+    Long encodeEnd=System.nanoTime();
+    System.out.println("encode take(ns): "+(encodeEnd-encodeStart));
+
     byte[] page = out.toByteArray();
-    // System.out.println("encoding data size:" + page.length + " byte");
+    System.out.println("encoding data size:" + page.length + " byte");

Review comment:
       ```suggestion
   ```




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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] HTHou commented on a change in pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
HTHou commented on a change in pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#discussion_r514010586



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DiffEncoder.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * <p>
+ * DiffEncoder is a encoder for compressing data in type of INT32(integer) and
+ * INT64(long). It is based in delta-encoding arithmetic.We adapt a hypothesis
+ * that contiguous data points have similar values.Thus the difference value of
+ * two adjacent points is smaller than those two point values. One integer in
+ * java takes 32-bits. If a positive number is less than 2^m, the bits of this
+ * integer which index from m to 31 are all 0. Given an array which length is n,
+ * if all values in input data array are all positive and less than 2^m, we need
+ * actually m*n, but not 32*n bits to store the array.
+ * </p>
+ * <p>
+ * DiffEncoder calculates difference between two adjacent. Then it saves the delta
+ * value. Then it statistics the longest bit length {@code m} it takes for each
+ * delta value, which means the bit length that maximum delta value takes. Only
+ * the low m bits are saved into result byte array for all delta values.
+ * </p>
+ */
+public abstract class DiffEncoder extends Encoder {
+
+    protected static final int BLOCK_DEFAULT_SIZE = 128;
+    private static final Logger logger = LoggerFactory.getLogger(DeltaBinaryEncoder.class);
+    protected ByteArrayOutputStream out;
+    protected int blockSize;
+    // input value is stored in deltaBlackBuffer temporarily
+    protected byte[] encodingBlockBuffer;
+
+    protected int writeIndex = -1;//defalut value?
+    protected int writeWidth = 0;
+
+    /**
+     * constructor of DeltaBinaryEncoder.
+     *
+     * @param size - the number how many numbers to be packed into a block.
+     */
+    public DiffEncoder(int size) {
+        super(TSEncoding.DIFF);
+        blockSize = size;
+    }
+
+    protected abstract void writeHeader() throws IOException;
+
+    protected abstract void writeValueToBytes(int i);
+
+    protected abstract void reset();
+
+    protected abstract int calculateBitWidthsForDeltaBlockBuffer();
+
+    /**
+     * write all data into {@code encodingBlockBuffer}.
+     */
+    private void writeDataWithMinWidth() {
+        for (int i = 0; i < writeIndex; i++) {
+            writeValueToBytes(i);
+        }
+        int encodingLength = (int) Math.ceil((double) (writeIndex * writeWidth) / 8.0);
+        out.write(encodingBlockBuffer, 0, encodingLength);
+    }
+
+    private void writeHeaderToBytes() throws IOException {
+        ReadWriteIOUtils.write(writeIndex, out);
+        ReadWriteIOUtils.write(writeWidth, out);
+        writeHeader();
+    }
+
+
+    //这其实是个写入函数,在真正flush清空之前,把东西都写到out里

Review comment:
       Better to use English.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DiffEncoder.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * <p>
+ * DiffEncoder is a encoder for compressing data in type of INT32(integer) and
+ * INT64(long). It is based in delta-encoding arithmetic.We adapt a hypothesis
+ * that contiguous data points have similar values.Thus the difference value of
+ * two adjacent points is smaller than those two point values. One integer in
+ * java takes 32-bits. If a positive number is less than 2^m, the bits of this
+ * integer which index from m to 31 are all 0. Given an array which length is n,
+ * if all values in input data array are all positive and less than 2^m, we need
+ * actually m*n, but not 32*n bits to store the array.
+ * </p>
+ * <p>
+ * DiffEncoder calculates difference between two adjacent. Then it saves the delta
+ * value. Then it statistics the longest bit length {@code m} it takes for each
+ * delta value, which means the bit length that maximum delta value takes. Only
+ * the low m bits are saved into result byte array for all delta values.
+ * </p>
+ */
+public abstract class DiffEncoder extends Encoder {
+
+    protected static final int BLOCK_DEFAULT_SIZE = 128;
+    private static final Logger logger = LoggerFactory.getLogger(DeltaBinaryEncoder.class);
+    protected ByteArrayOutputStream out;
+    protected int blockSize;
+    // input value is stored in deltaBlackBuffer temporarily
+    protected byte[] encodingBlockBuffer;
+
+    protected int writeIndex = -1;//defalut value?
+    protected int writeWidth = 0;
+
+    /**
+     * constructor of DeltaBinaryEncoder.
+     *
+     * @param size - the number how many numbers to be packed into a block.
+     */
+    public DiffEncoder(int size) {
+        super(TSEncoding.DIFF);
+        blockSize = size;
+    }
+
+    protected abstract void writeHeader() throws IOException;
+
+    protected abstract void writeValueToBytes(int i);
+
+    protected abstract void reset();
+
+    protected abstract int calculateBitWidthsForDeltaBlockBuffer();
+
+    /**
+     * write all data into {@code encodingBlockBuffer}.
+     */
+    private void writeDataWithMinWidth() {
+        for (int i = 0; i < writeIndex; i++) {
+            writeValueToBytes(i);
+        }
+        int encodingLength = (int) Math.ceil((double) (writeIndex * writeWidth) / 8.0);
+        out.write(encodingBlockBuffer, 0, encodingLength);
+    }
+
+    private void writeHeaderToBytes() throws IOException {
+        ReadWriteIOUtils.write(writeIndex, out);
+        ReadWriteIOUtils.write(writeWidth, out);
+        writeHeader();
+    }
+
+
+    //这其实是个写入函数,在真正flush清空之前,把东西都写到out里
+    private void flushBlockBuffer(ByteArrayOutputStream out) throws IOException {
+        if (writeIndex == -1) {
+            return;
+        }
+        // since we store the min delta, the deltas will be converted to be the
+        // difference to min delta and all positive
+        this.out = out;
+        writeWidth = calculateBitWidthsForDeltaBlockBuffer();
+        writeHeaderToBytes();
+        writeDataWithMinWidth();
+
+        reset();
+        writeIndex = -1;
+    }
+
+    /**
+     * calling this method to flush all values which haven't encoded to result byte
+     * array.
+     * 调用flushBlockBuffer函数

Review comment:
       Same.

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DiffEncoder.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * <p>
+ * DiffEncoder is a encoder for compressing data in type of INT32(integer) and
+ * INT64(long). It is based in delta-encoding arithmetic.We adapt a hypothesis
+ * that contiguous data points have similar values.Thus the difference value of
+ * two adjacent points is smaller than those two point values. One integer in
+ * java takes 32-bits. If a positive number is less than 2^m, the bits of this
+ * integer which index from m to 31 are all 0. Given an array which length is n,
+ * if all values in input data array are all positive and less than 2^m, we need
+ * actually m*n, but not 32*n bits to store the array.
+ * </p>
+ * <p>
+ * DiffEncoder calculates difference between two adjacent. Then it saves the delta
+ * value. Then it statistics the longest bit length {@code m} it takes for each
+ * delta value, which means the bit length that maximum delta value takes. Only
+ * the low m bits are saved into result byte array for all delta values.
+ * </p>
+ */
+public abstract class DiffEncoder extends Encoder {
+
+    protected static final int BLOCK_DEFAULT_SIZE = 128;
+    private static final Logger logger = LoggerFactory.getLogger(DeltaBinaryEncoder.class);
+    protected ByteArrayOutputStream out;
+    protected int blockSize;
+    // input value is stored in deltaBlackBuffer temporarily
+    protected byte[] encodingBlockBuffer;
+
+    protected int writeIndex = -1;//defalut value?
+    protected int writeWidth = 0;

Review comment:
       The indent of this java file is not correct. Please fix it.

##########
File path: tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/delta/DiffEncoderIntegerTest.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.delta;
+
+import org.apache.iotdb.tsfile.encoding.decoder.DiffDecoder;
+import org.apache.iotdb.tsfile.encoding.encoder.DiffEncoder;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+
+public class DiffEncoderIntegerTest {
+
+  private static final int ROW_NUM = 100_000_00;//行数,数组的大小
+  ByteArrayOutputStream out;
+  private DiffEncoder writer;
+  private DiffDecoder reader;
+  private Random ran = new Random();
+  private ByteBuffer buffer;
+
+  @Before
+  public void test() {
+    writer = new DiffEncoder.IntDeltaEncoder();
+    reader = new DiffDecoder.IntDeltaDecoder();
+  }
+
+  @Test
+  public void testBasic() throws IOException {
+    int data[] = new int[ROW_NUM];
+    for (int i = 0; i < ROW_NUM; i++) {
+      data[i] = i * i;
+    }
+    shouldReadAndWrite(data, ROW_NUM);
+  }
+
+  @Test
+  public void testBoundInt() throws IOException {
+    int data[] = new int[ROW_NUM];
+    for (int i = 0; i < 10; i++) {
+      boundInt(i, data);
+    }
+  }
+
+  private void boundInt(int power, int[] data) throws IOException {
+    for (int i = 0; i < ROW_NUM; i++) {
+      data[i] = ran.nextInt((int) Math.pow(2, power));
+    }
+    shouldReadAndWrite(data, ROW_NUM);
+  }
+
+  @Test
+  public void testRandom() throws IOException {
+    int data[] = new int[ROW_NUM];
+    for (int i = 0; i < ROW_NUM; i++) {
+      data[i] = ran.nextInt();
+      System.out.printf("%d ",data[i]);

Review comment:
       This line will print too mush lines...

##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DiffEncoder.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * <p>
+ * DiffEncoder is a encoder for compressing data in type of INT32(integer) and
+ * INT64(long). It is based in delta-encoding arithmetic.We adapt a hypothesis
+ * that contiguous data points have similar values.Thus the difference value of
+ * two adjacent points is smaller than those two point values. One integer in
+ * java takes 32-bits. If a positive number is less than 2^m, the bits of this
+ * integer which index from m to 31 are all 0. Given an array which length is n,
+ * if all values in input data array are all positive and less than 2^m, we need
+ * actually m*n, but not 32*n bits to store the array.
+ * </p>
+ * <p>
+ * DiffEncoder calculates difference between two adjacent. Then it saves the delta
+ * value. Then it statistics the longest bit length {@code m} it takes for each
+ * delta value, which means the bit length that maximum delta value takes. Only
+ * the low m bits are saved into result byte array for all delta values.
+ * </p>
+ */
+public abstract class DiffEncoder extends Encoder {
+
+    protected static final int BLOCK_DEFAULT_SIZE = 128;
+    private static final Logger logger = LoggerFactory.getLogger(DeltaBinaryEncoder.class);
+    protected ByteArrayOutputStream out;
+    protected int blockSize;
+    // input value is stored in deltaBlackBuffer temporarily
+    protected byte[] encodingBlockBuffer;
+
+    protected int writeIndex = -1;//defalut value?
+    protected int writeWidth = 0;
+
+    /**
+     * constructor of DeltaBinaryEncoder.
+     *
+     * @param size - the number how many numbers to be packed into a block.
+     */
+    public DiffEncoder(int size) {
+        super(TSEncoding.DIFF);
+        blockSize = size;
+    }
+
+    protected abstract void writeHeader() throws IOException;
+
+    protected abstract void writeValueToBytes(int i);
+
+    protected abstract void reset();
+
+    protected abstract int calculateBitWidthsForDeltaBlockBuffer();
+
+    /**
+     * write all data into {@code encodingBlockBuffer}.
+     */
+    private void writeDataWithMinWidth() {
+        for (int i = 0; i < writeIndex; i++) {
+            writeValueToBytes(i);
+        }
+        int encodingLength = (int) Math.ceil((double) (writeIndex * writeWidth) / 8.0);
+        out.write(encodingBlockBuffer, 0, encodingLength);
+    }
+
+    private void writeHeaderToBytes() throws IOException {
+        ReadWriteIOUtils.write(writeIndex, out);
+        ReadWriteIOUtils.write(writeWidth, out);
+        writeHeader();
+    }
+
+
+    //这其实是个写入函数,在真正flush清空之前,把东西都写到out里
+    private void flushBlockBuffer(ByteArrayOutputStream out) throws IOException {
+        if (writeIndex == -1) {
+            return;
+        }
+        // since we store the min delta, the deltas will be converted to be the
+        // difference to min delta and all positive
+        this.out = out;
+        writeWidth = calculateBitWidthsForDeltaBlockBuffer();
+        writeHeaderToBytes();
+        writeDataWithMinWidth();
+
+        reset();
+        writeIndex = -1;
+    }
+
+    /**
+     * calling this method to flush all values which haven't encoded to result byte
+     * array.
+     * 调用flushBlockBuffer函数
+     */
+    @Override
+    public void flush(ByteArrayOutputStream out) {
+        try {
+            flushBlockBuffer(out);
+        } catch (IOException e) {
+            logger.error("flush data to stream failed!", e);
+        }
+    }
+
+    public static class IntDeltaEncoder extends DiffEncoder {
+
+        private int[] deltaBlockBuffer;
+        private int firstValue;
+        private int previousValue;
+//        private int minDeltaBase;
+
+        public IntDeltaEncoder() {
+            this(BLOCK_DEFAULT_SIZE);
+        }
+
+        /**
+         * constructor of IntDeltaEncoder which is a sub-class of DeltaBinaryEncoder.
+         *
+         * @param size - the number how many numbers to be packed into a block.
+         */
+        public IntDeltaEncoder(int size) {
+            super(size);// choose TS_2DIFF + initialize deltaSize=size
+            deltaBlockBuffer = new int[this.blockSize]; //deltaBlockBuffer: 暂时性存储待压缩的数据
+            encodingBlockBuffer = new byte[blockSize * 4]; //目前的理解:压缩后存放的地方
+            reset();//初始化

Review comment:
       Same

##########
File path: tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/delta/DiffEncoderIntegerTest.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.delta;
+
+import org.apache.iotdb.tsfile.encoding.decoder.DiffDecoder;
+import org.apache.iotdb.tsfile.encoding.encoder.DiffEncoder;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+
+public class DiffEncoderIntegerTest {
+
+  private static final int ROW_NUM = 100_000_00;//行数,数组的大小

Review comment:
       Use English




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



[GitHub] [iotdb] 5Reasons commented on a change in pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
5Reasons commented on a change in pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#discussion_r514197949



##########
File path: tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/delta/DiffEncoderIntegerTest.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.delta;
+
+import org.apache.iotdb.tsfile.encoding.decoder.DiffDecoder;
+import org.apache.iotdb.tsfile.encoding.encoder.DiffEncoder;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+
+public class DiffEncoderIntegerTest {
+
+  private static final int ROW_NUM = 100_000_00;//行数,数组的大小
+  ByteArrayOutputStream out;
+  private DiffEncoder writer;
+  private DiffDecoder reader;
+  private Random ran = new Random();
+  private ByteBuffer buffer;
+
+  @Before
+  public void test() {
+    writer = new DiffEncoder.IntDeltaEncoder();
+    reader = new DiffDecoder.IntDeltaDecoder();
+  }
+
+  @Test
+  public void testBasic() throws IOException {
+    int data[] = new int[ROW_NUM];
+    for (int i = 0; i < ROW_NUM; i++) {
+      data[i] = i * i;
+    }
+    shouldReadAndWrite(data, ROW_NUM);
+  }
+
+  @Test
+  public void testBoundInt() throws IOException {
+    int data[] = new int[ROW_NUM];
+    for (int i = 0; i < 10; i++) {
+      boundInt(i, data);
+    }
+  }
+
+  private void boundInt(int power, int[] data) throws IOException {
+    for (int i = 0; i < ROW_NUM; i++) {
+      data[i] = ran.nextInt((int) Math.pow(2, power));
+    }
+    shouldReadAndWrite(data, ROW_NUM);
+  }
+
+  @Test
+  public void testRandom() throws IOException {
+    int data[] = new int[ROW_NUM];
+    for (int i = 0; i < ROW_NUM; i++) {
+      data[i] = ran.nextInt();
+      System.out.printf("%d ",data[i]);

Review comment:
       Sorry for not finding the problems before submitting. I will fix them.




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



[GitHub] [iotdb] kr11 commented on a change in pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
kr11 commented on a change in pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#discussion_r514751841



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DiffEncoder.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * <p>
+ * DiffEncoder is a encoder for compressing data in type of INT32(integer) and
+ * INT64(long). It is based in delta-encoding arithmetic.We adapt a hypothesis
+ * that contiguous data points have similar values.Thus the difference value of
+ * two adjacent points is smaller than those two point values. One integer in
+ * java takes 32-bits. If a positive number is less than 2^m, the bits of this
+ * integer which index from m to 31 are all 0. Given an array which length is n,
+ * if all values in input data array are all positive and less than 2^m, we need
+ * actually m*n, but not 32*n bits to store the array.
+ * </p>
+ * <p>
+ * DiffEncoder calculates difference between two adjacent. Then it saves the delta
+ * value. Then it statistics the longest bit length {@code m} it takes for each
+ * delta value, which means the bit length that maximum delta value takes. Only
+ * the low m bits are saved into result byte array for all delta values.
+ * </p>
+ */
+public abstract class DiffEncoder extends Encoder {
+
+    protected static final int BLOCK_DEFAULT_SIZE = 128;
+    private static final Logger logger = LoggerFactory.getLogger(DeltaBinaryEncoder.class);
+    protected ByteArrayOutputStream out;
+    protected int blockSize;
+    // input value is stored in deltaBlackBuffer temporarily
+    protected byte[] encodingBlockBuffer;
+
+    protected int writeIndex = -1;//defalut value?
+    protected int writeWidth = 0;
+
+    /**
+     * constructor of DeltaBinaryEncoder.

Review comment:
       All `DeltaBinaryEncoder` annotations should be checked and removed carefully 




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



[GitHub] [iotdb] kr11 commented on a change in pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
kr11 commented on a change in pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#discussion_r514736515



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DiffEncoder.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * <p>
+ * DiffEncoder is a encoder for compressing data in type of INT32(integer) and
+ * INT64(long). It is based in delta-encoding arithmetic.We adapt a hypothesis

Review comment:
       arithmetic.We adapt -> arithmetic. We adapt 
   space




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



[GitHub] [iotdb] coveralls edited a comment on pull request #1890: [IOTDB-1597] Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#issuecomment-908897698


   
   [![Coverage Status](https://coveralls.io/builds/42538060/badge)](https://coveralls.io/builds/42538060)
   
   Coverage increased (+0.01%) to 67.374% when pulling **d9881b2954a949ce70e22ffee979caad67757449 on 5Reasons:addDiff** into **fb18357edee14e60893e968aeddc3177fba868bc on apache:master**.
   


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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] coveralls edited a comment on pull request #1890: [IOTDB-1597] Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
coveralls edited a comment on pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#issuecomment-908897698


   
   [![Coverage Status](https://coveralls.io/builds/42529630/badge)](https://coveralls.io/builds/42529630)
   
   Coverage increased (+0.02%) to 67.377% when pulling **df32a0e59cc4c74cfd469043a8b4a7ba7858117e on 5Reasons:addDiff** into **fb18357edee14e60893e968aeddc3177fba868bc on apache:master**.
   


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

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [iotdb] kr11 commented on a change in pull request #1890: Finish DIFF encoding algorithm which is reserved in enum before

Posted by GitBox <gi...@apache.org>.
kr11 commented on a change in pull request #1890:
URL: https://github.com/apache/iotdb/pull/1890#discussion_r514747722



##########
File path: tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DiffEncoder.java
##########
@@ -0,0 +1,324 @@
+/*
+ * 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 org.apache.iotdb.tsfile.utils.BytesUtils;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+/**
+ * <p>
+ * DiffEncoder is a encoder for compressing data in type of INT32(integer) and
+ * INT64(long). It is based in delta-encoding arithmetic.We adapt a hypothesis
+ * that contiguous data points have similar values.Thus the difference value of
+ * two adjacent points is smaller than those two point values. One integer in
+ * java takes 32-bits. If a positive number is less than 2^m, the bits of this
+ * integer which index from m to 31 are all 0. Given an array which length is n,
+ * if all values in input data array are all positive and less than 2^m, we need
+ * actually m*n, but not 32*n bits to store the array.
+ * </p>
+ * <p>
+ * DiffEncoder calculates difference between two adjacent. Then it saves the delta
+ * value. Then it statistics the longest bit length {@code m} it takes for each
+ * delta value, which means the bit length that maximum delta value takes. Only
+ * the low m bits are saved into result byte array for all delta values.
+ * </p>
+ */
+public abstract class DiffEncoder extends Encoder {
+
+    protected static final int BLOCK_DEFAULT_SIZE = 128;
+    private static final Logger logger = LoggerFactory.getLogger(DeltaBinaryEncoder.class);
+    protected ByteArrayOutputStream out;
+    protected int blockSize;
+    // input value is stored in deltaBlackBuffer temporarily
+    protected byte[] encodingBlockBuffer;
+
+    protected int writeIndex = -1;//defalut value?

Review comment:
       What's the meaning of this annotation? We should explain something, not ask :)




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