You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2020/11/03 13:58:13 UTC

[GitHub] [arrow] jhorstmann opened a new pull request #8571: ARROW-10461: [Rust] Fix offset bug in remainder bits

jhorstmann opened a new pull request #8571:
URL: https://github.com/apache/arrow/pull/8571


   


----------------------------------------------------------------
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] [arrow] vertexclique commented on a change in pull request #8571: ARROW-10461: [Rust] Fix offset bug in remainder bits

Posted by GitBox <gi...@apache.org>.
vertexclique commented on a change in pull request #8571:
URL: https://github.com/apache/arrow/pull/8571#discussion_r516513055



##########
File path: rust/arrow/src/util/bit_chunk_iterator.rs
##########
@@ -214,10 +214,37 @@ mod tests {
 
         let result = bitchunks.into_iter().collect::<Vec<_>>();
 
-        //assert_eq!(vec![0b00010000, 0b00100000, 0b01000000, 0b10000000, 0b00000000, 0b00000001, 0b00000010, 0b11110100], result);
         assert_eq!(
             vec![0b1111010000000010000000010000000010000000010000000010000000010000],
             result
         );
     }
+
+    #[test]
+    fn test_iter_unaligned_remainder_bits_across_bytes() {
+        let input: &[u8] = &[0b00000000, 0b11111111];
+        let buffer: Buffer = Buffer::from(input);
+
+        let bitchunks = buffer.bit_chunks(6, 7);

Review comment:
       What is 6, what is 7? Is 6 from left ? is 6 from right? Why there are 7 bits in the assertion? Why there are 5 bits there?

##########
File path: rust/arrow/src/util/bit_chunk_iterator.rs
##########
@@ -72,22 +72,23 @@ impl<'a> BitChunks<'a> {
         if bit_len == 0 {
             0
         } else {
-            let byte_len = ceil(bit_len, 8);
-
-            let mut bits = 0;
-            for i in 0..byte_len {
-                let byte = unsafe {
-                    std::ptr::read(
-                        self.raw_data
-                            .add(self.chunk_len * std::mem::size_of::<u64>() + i),
-                    )
-                };
-                bits |= (byte as u64) << (i * 8);
-            }
+            let bit_offset = self.offset;
+            // number of bytes to read
+            // might be one more than sizeof(u64) if the offset is in the middle of a byte
+            let byte_len = ceil(bit_len + bit_offset, 8);
+            // pointer to remainder bytes after all complete chunks
+            let base = unsafe {
+                self.raw_data
+                    .add(self.chunk_len * std::mem::size_of::<u64>())

Review comment:
       Where chunk_len is created, chunk_bits is obviously: `8 * std::mem:size_of::<u64>()`
   Any reason to not write that?

##########
File path: rust/arrow/src/util/bit_chunk_iterator.rs
##########
@@ -214,10 +214,37 @@ mod tests {
 
         let result = bitchunks.into_iter().collect::<Vec<_>>();
 
-        //assert_eq!(vec![0b00010000, 0b00100000, 0b01000000, 0b10000000, 0b00000000, 0b00000001, 0b00000010, 0b11110100], result);
         assert_eq!(
             vec![0b1111010000000010000000010000000010000000010000000010000000010000],
             result
         );
     }
+
+    #[test]
+    fn test_iter_unaligned_remainder_bits_across_bytes() {
+        let input: &[u8] = &[0b00000000, 0b11111111];
+        let buffer: Buffer = Buffer::from(input);
+
+        let bitchunks = buffer.bit_chunks(6, 7);
+
+        assert_eq!(7, bitchunks.remainder_len());

Review comment:
       None of these methods have documentation about what they do. e.g. remainder len in which form? bits? bytes?

##########
File path: rust/arrow/src/util/bit_chunk_iterator.rs
##########
@@ -214,10 +214,37 @@ mod tests {
 
         let result = bitchunks.into_iter().collect::<Vec<_>>();
 
-        //assert_eq!(vec![0b00010000, 0b00100000, 0b01000000, 0b10000000, 0b00000000, 0b00000001, 0b00000010, 0b11110100], result);
         assert_eq!(
             vec![0b1111010000000010000000010000000010000000010000000010000000010000],
             result
         );
     }
+
+    #[test]
+    fn test_iter_unaligned_remainder_bits_across_bytes() {
+        let input: &[u8] = &[0b00000000, 0b11111111];
+        let buffer: Buffer = Buffer::from(input);
+
+        let bitchunks = buffer.bit_chunks(6, 7);
+
+        assert_eq!(7, bitchunks.remainder_len());
+        assert_eq!(0b1111100, bitchunks.remainder_bits());
+    }
+
+    #[test]
+    fn test_iter_unaligned_remainder_bits_large() {
+        let input: &[u8] = &[
+            0b11111111, 0b00000000, 0b11111111, 0b00000000, 0b11111111, 0b00000000,
+            0b11111111, 0b00000000, 0b11111111,
+        ];
+        let buffer: Buffer = Buffer::from(input);
+
+        let bitchunks = buffer.bit_chunks(2, 63);
+
+        assert_eq!(63, bitchunks.remainder_len());
+        assert_eq!(
+            0b01000000_00111111_11000000_00111111_11000000_00111111_11000000_00111111,

Review comment:
       Why this forms these bit pattern? How a reader can be ensure that it is forming the correct pattern while reading?




----------------------------------------------------------------
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] [arrow] jhorstmann commented on a change in pull request #8571: ARROW-10461: [Rust] Fix offset bug in remainder bits

Posted by GitBox <gi...@apache.org>.
jhorstmann commented on a change in pull request #8571:
URL: https://github.com/apache/arrow/pull/8571#discussion_r518825331



##########
File path: rust/arrow-flight/Cargo.toml
##########
@@ -27,7 +27,10 @@ license = "Apache-2.0"
 
 [dependencies]
 arrow = { path = "../arrow", version = "3.0.0-SNAPSHOT" }
-tonic = "0.3"
+tonic = "0.3.1"

Review comment:
       I had compilation failures otherwise, but it's strange that other PRs do not seem to have the same issue. I'll check again and maybe move this to a separate PR.




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

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



[GitHub] [arrow] github-actions[bot] commented on pull request #8571: ARROW-10461: [Rust] Fix offset bug in remainder bits

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #8571:
URL: https://github.com/apache/arrow/pull/8571#issuecomment-720566906


   https://issues.apache.org/jira/browse/ARROW-10461


----------------------------------------------------------------
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] [arrow] jhorstmann commented on a change in pull request #8571: ARROW-10461: [Rust] Fix offset bug in remainder bits

Posted by GitBox <gi...@apache.org>.
jhorstmann commented on a change in pull request #8571:
URL: https://github.com/apache/arrow/pull/8571#discussion_r518830136



##########
File path: rust/arrow/src/util/bit_chunk_iterator.rs
##########
@@ -55,48 +58,52 @@ impl<'a> BitChunks<'a> {
 pub struct BitChunkIterator<'a> {
     buffer: &'a Buffer,
     raw_data: *const u8,
-    offset: usize,
+    bit_offset: usize,
     chunk_len: usize,
     index: usize,
 }
 
 impl<'a> BitChunks<'a> {
+    /// Returns the number of remaining bits, guaranteed to be between 0 and 63 (inclusive)
     #[inline]
     pub const fn remainder_len(&self) -> usize {
         self.remainder_len
     }
 
+    /// Returns the bitmask of remaining bits
     #[inline]
     pub fn remainder_bits(&self) -> u64 {
         let bit_len = self.remainder_len;
         if bit_len == 0 {
             0
         } else {
-            let byte_len = ceil(bit_len, 8);

Review comment:
       Exactly. If we have 7 remainder bits for example, if the offset starts at 0 it is enough to read `ceil(7, 8) = 1` byte, but if we start for example at offset 4 we need to combine bits from `ceil(4+7, 8) = 2` bytes. The further refactoring of the loop was then necessary because we may now need to combine bits from 9 bytes, and the previous code (which only shifted after the loop) would have overflowed an `u64`.




----------------------------------------------------------------
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] [arrow] alamb commented on pull request #8571: ARROW-10461: [Rust] Fix offset bug in remainder bits

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #8571:
URL: https://github.com/apache/arrow/pull/8571#issuecomment-724641767


   I agree @jhorstmann  -- I think we should merge this. I don't yet have the permissions set up to merge things, so I need to wait for one of the other committer such as @andygrove , @nevi-me  or @jorgecarleitao  to do so


----------------------------------------------------------------
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] [arrow] jorgecarleitao commented on pull request #8571: ARROW-10461: [Rust] Fix offset bug in remainder bits

Posted by GitBox <gi...@apache.org>.
jorgecarleitao commented on pull request #8571:
URL: https://github.com/apache/arrow/pull/8571#issuecomment-724694901


   @jhorstmann thanks a lot for this fix!
   
   
   @alamb, congrats on your push!!!!! 🎉 🎉 🎉 🎉 🎉


----------------------------------------------------------------
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] [arrow] jhorstmann commented on pull request #8571: ARROW-10461: [Rust] Fix offset bug in remainder bits

Posted by GitBox <gi...@apache.org>.
jhorstmann commented on pull request #8571:
URL: https://github.com/apache/arrow/pull/8571#issuecomment-724609649


   @nevi-me @alamb @jorgecarleitao I had a quick chat with @vertexclique and we think we can merge this PR with the bugfix first and then rebase and integrate his refactoring. Those PRs shouldn't block each other.


----------------------------------------------------------------
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] [arrow] alamb commented on a change in pull request #8571: ARROW-10461: [Rust] Fix offset bug in remainder bits

Posted by GitBox <gi...@apache.org>.
alamb commented on a change in pull request #8571:
URL: https://github.com/apache/arrow/pull/8571#discussion_r518757983



##########
File path: rust/arrow-flight/Cargo.toml
##########
@@ -27,7 +27,10 @@ license = "Apache-2.0"
 
 [dependencies]
 arrow = { path = "../arrow", version = "3.0.0-SNAPSHOT" }
-tonic = "0.3"
+tonic = "0.3.1"

Review comment:
       I wonder if there is some reason to upgrade tonic in this same PR?

##########
File path: rust/arrow/src/util/bit_chunk_iterator.rs
##########
@@ -55,48 +58,52 @@ impl<'a> BitChunks<'a> {
 pub struct BitChunkIterator<'a> {
     buffer: &'a Buffer,
     raw_data: *const u8,
-    offset: usize,
+    bit_offset: usize,
     chunk_len: usize,
     index: usize,
 }
 
 impl<'a> BitChunks<'a> {
+    /// Returns the number of remaining bits, guaranteed to be between 0 and 63 (inclusive)
     #[inline]
     pub const fn remainder_len(&self) -> usize {
         self.remainder_len
     }
 
+    /// Returns the bitmask of remaining bits
     #[inline]
     pub fn remainder_bits(&self) -> u64 {
         let bit_len = self.remainder_len;
         if bit_len == 0 {
             0
         } else {
-            let byte_len = ceil(bit_len, 8);

Review comment:
       Just to confirm my understanding, the bug is that the previous code assumed the remainder bits always started at a byte boundary like:
   
   ```
   +--------+--------+--------+----+
   |        |        |        |    |
   +--------+--------+--------+----+
            ^
            (remainder starts)
   ```
   
   But produces incorrect behavior if the remainder starts in the middle of a byte
   ```
   +--------+--------+--------+----+
   |        |        |        |    |
   +--------+--------+--------+----+
               ^
               (remainder starts)
   ```
   




----------------------------------------------------------------
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] [arrow] jhorstmann commented on pull request #8571: ARROW-10461: [Rust] Fix offset bug in remainder bits

Posted by GitBox <gi...@apache.org>.
jhorstmann commented on pull request #8571:
URL: https://github.com/apache/arrow/pull/8571#issuecomment-722265285


   @jorgecarleitao @nevi-me Can you take a look at this bugfix?


----------------------------------------------------------------
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] [arrow] alamb edited a comment on pull request #8571: ARROW-10461: [Rust] Fix offset bug in remainder bits

Posted by GitBox <gi...@apache.org>.
alamb edited a comment on pull request #8571:
URL: https://github.com/apache/arrow/pull/8571#issuecomment-724641767


   I agree @jhorstmann  -- I think we should merge this. ~~I don't yet have the permissions set up to merge things, so I need to wait for one of the other committer such as @andygrove , @nevi-me  or @jorgecarleitao  to do so~~
   
   Update: the bits came through and I have merged this PR. FYI @andygrove @nevi-me and @jorgecarleitao 


----------------------------------------------------------------
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] [arrow] jorgecarleitao commented on pull request #8571: ARROW-10461: [Rust] Fix offset bug in remainder bits

Posted by GitBox <gi...@apache.org>.
jorgecarleitao commented on pull request #8571:
URL: https://github.com/apache/arrow/pull/8571#issuecomment-722904601


   @jhorstmann , bit operations is one of my weaknesses; I can't review this due to my lack of knowledge on that topic.


----------------------------------------------------------------
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] [arrow] alamb commented on pull request #8571: ARROW-10461: [Rust] Fix offset bug in remainder bits

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #8571:
URL: https://github.com/apache/arrow/pull/8571#issuecomment-723439185


   FYI @jorgecarleitao  / @nevi-me  I think we should merge this PR.


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

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



[GitHub] [arrow] alamb closed pull request #8571: ARROW-10461: [Rust] Fix offset bug in remainder bits

Posted by GitBox <gi...@apache.org>.
alamb closed pull request #8571:
URL: https://github.com/apache/arrow/pull/8571


   


----------------------------------------------------------------
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] [arrow] jhorstmann commented on a change in pull request #8571: ARROW-10461: [Rust] Fix offset bug in remainder bits

Posted by GitBox <gi...@apache.org>.
jhorstmann commented on a change in pull request #8571:
URL: https://github.com/apache/arrow/pull/8571#discussion_r518997813



##########
File path: rust/arrow-flight/Cargo.toml
##########
@@ -27,7 +27,10 @@ license = "Apache-2.0"
 
 [dependencies]
 arrow = { path = "../arrow", version = "3.0.0-SNAPSHOT" }
-tonic = "0.3"
+tonic = "0.3.1"

Review comment:
       Removed the dependency again and CI still works, not sure what the issue was locally.




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