You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@arrow.apache.org by "coady (via GitHub)" <gi...@apache.org> on 2023/07/16 16:49:53 UTC

[GitHub] [arrow] coady opened a new issue, #36708: `run_end_encode` segfaults on chunked arrays.

coady opened a new issue, #36708:
URL: https://github.com/apache/arrow/issues/36708

   ### Describe the bug, including details regarding any error messages, version, and platform.
   
   ```python
   import pyarrow as pa, pyarrow.compute as pc, pyarrow.dataset as ds
   
   ds.write_dataset(pa.table({'': pa.repeat(1, 10)}), 'temp', format='parquet')
   (column,) = ds.dataset('temp').to_table()
   
   assert pc.run_end_encode(column.combine_chunks())
   pc.run_end_encode(column)  # <- segfaults
   ```
   Seems to only reproduce when the array is read. Tested on the latest nightly build: 13.0.0.dev516.
   
   ### Component(s)
   
   C++, Python


-- 
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: issues-unsubscribe@arrow.apache.org.apache.org

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


[GitHub] [arrow] kou commented on issue #36708: [C++] `run_end_encode` segfaults on arrays with an all-set validity buffer

Posted by "kou (via GitHub)" <gi...@apache.org>.
kou commented on issue #36708:
URL: https://github.com/apache/arrow/issues/36708#issuecomment-1642927947

   Updated the title.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] felipecrv commented on issue #36708: `run_end_encode` segfaults on chunked arrays.

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on issue #36708:
URL: https://github.com/apache/arrow/issues/36708#issuecomment-1640326536

   What if `null_count()` returns `0` in this branch of the conditional? This is where a `11..11` bitmap can creep in.
   
   ```cpp
       std::vector<std::shared_ptr<Buffer>> buffers = {reader->ReleaseIsValid(),
                                                       reader->ReleaseValues()};
       data = std::make_shared<::arrow::ArrayData>(field->type(), reader->values_written(),
                                                   std::move(buffers), reader->null_count());
   ```
   


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] felipecrv commented on issue #36708: `run_end_encode` segfaults on chunked arrays.

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on issue #36708:
URL: https://github.com/apache/arrow/issues/36708#issuecomment-1639314404

   But to be clear: validity bitmap set when `null_count` is zero is totally valid. Sometimes it's hard to know the null_count is zero upfront (e.g. slicing an array producing a range that has no nulls).


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] kou closed issue #36708: [C++] `run_end_encode` segfaults on arrays with an all-set validity buffer

Posted by "kou (via GitHub)" <gi...@apache.org>.
kou closed issue #36708: [C++] `run_end_encode` segfaults on arrays with an all-set validity buffer
URL: https://github.com/apache/arrow/issues/36708


-- 
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: issues-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] felipecrv commented on issue #36708: `run_end_encode` segfaults on chunked arrays.

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on issue #36708:
URL: https://github.com/apache/arrow/issues/36708#issuecomment-1639118127

   Chunked arrays are handled by the kernel and even part of the unit tests, but the repro code above does indeed trigger the SIGSEGV.
   
   I isolated the cause of the SIGSEGV to some expectations in the allocation code not being preserved.
   
   ```cpp
   Result<std::shared_ptr<ArrayData>> PreallocateValuesArray(
       const std::shared_ptr<DataType>& value_type, bool has_validity_buffer, int64_t length,
       int64_t null_count, MemoryPool* pool, int64_t data_buffer_size) {
     std::vector<std::shared_ptr<Buffer>> values_data_buffers;
     std::shared_ptr<Buffer> validity_buffer = NULLPTR;
     if (has_validity_buffer) {
       ARROW_ASSIGN_OR_RAISE(validity_buffer, AllocateEmptyBitmap(length, pool));
       DCHECK(validity_buffer);
     }
     ARROW_ASSIGN_OR_RAISE(auto values_buffer, AllocateValuesBuffer(length, *value_type,
                                                                    pool, data_buffer_size));
     if (is_base_binary_like(value_type->id())) {
       const int offset_byte_width = offset_bit_width(value_type->id()) / 8;
       ARROW_ASSIGN_OR_RAISE(auto offsets_buffer,
                             AllocateBuffer((length + 1) * offset_byte_width, pool));
       // Ensure the first offset is zero
       memset(offsets_buffer->mutable_data(), 0, offset_byte_width);
       offsets_buffer->ZeroPadding();
       values_data_buffers = {validity_buffer, std::move(offsets_buffer),
                              std::move(values_buffer)};
     } else {
       values_data_buffers = {validity_buffer, std::move(values_buffer)};
     }
     auto data = ArrayData::Make(value_type, length, values_data_buffers, null_count);
     DCHECK(!has_validity_buffer || validity_buffer.use_count() == 2);
     DCHECK(!has_validity_buffer || validity_buffer != NULLPTR);
     DCHECK(!has_validity_buffer || data->buffers[0] != NULLPTR);
     return data;
   }
   ```
   
   `DCHECK(!has_validity_buffer || data->buffers[0] != NULLPTR);` fails after all the previous checks pass and I still don't understand why.
   
   
   


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] felipecrv commented on issue #36708: `run_end_encode` segfaults on chunked arrays.

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on issue #36708:
URL: https://github.com/apache/arrow/issues/36708#issuecomment-1639310748

   @coady the issue is triggered by giving an input array without nulls that has a non-null validity bitmap buffer. The PR I've sent fixes that.
   
   @mapleFU it might be nice to investigate if the reading of Parquet files is unnecessarily producing validity buffers when `null_count` is zero. Setting those buffers to `NULLPTR` when `null_count` is known to be zero can avoid this kind of confusion and save some cost of calculating `null_count` by scanning all the bits down the road.


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] felipecrv commented on issue #36708: [C++] `run_end_encode` segfaults on chunked arrays.

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on issue #36708:
URL: https://github.com/apache/arrow/issues/36708#issuecomment-1642725304

   @westonpace for reference, the title of the issue should probably be: "run_end_encode segfaults on arrays with an all-set validity buffer"


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] felipecrv commented on issue #36708: `run_end_encode` segfaults on chunked arrays.

Posted by "felipecrv (via GitHub)" <gi...@apache.org>.
felipecrv commented on issue #36708:
URL: https://github.com/apache/arrow/issues/36708#issuecomment-1639133737

   Found the reason: `AdjustNonNullable` zeroes the validity buffer if `null_count` is passed as `0`.
   
   ```cpp
   static inline void AdjustNonNullable(Type::type type_id, int64_t length,
                                        std::vector<std::shared_ptr<Buffer>>* buffers,
                                        int64_t* null_count) {
     if (type_id == Type::NA) {
       *null_count = length;
       (*buffers)[0] = nullptr;
     } else if (internal::HasValidityBitmap(type_id)) {
       if (*null_count == 0) {
         // In case there are no nulls, don't keep an allocated null bitmap around
         (*buffers)[0] = nullptr;
       } else if (*null_count == kUnknownNullCount && buffers->at(0) == nullptr) {
         // Conversely, if no null bitmap is provided, set the null count to 0
         *null_count = 0;
       }
     } else {
       *null_count = 0;
     }
   }
   ```


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] mapleFU commented on issue #36708: `run_end_encode` segfaults on chunked arrays.

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on issue #36708:
URL: https://github.com/apache/arrow/issues/36708#issuecomment-1637138404

   cc @felipecrv 


-- 
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: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] mapleFU commented on issue #36708: `run_end_encode` segfaults on chunked arrays.

Posted by "mapleFU (via GitHub)" <gi...@apache.org>.
mapleFU commented on issue #36708:
URL: https://github.com/apache/arrow/issues/36708#issuecomment-1639362949

   I guess not:
   
   ```
   std::shared_ptr<Array> TransferZeroCopy(RecordReader* reader,
                                           const std::shared_ptr<Field>& field) {
     std::shared_ptr<::arrow::ArrayData> data;
     if (field->nullable()) {
       std::vector<std::shared_ptr<Buffer>> buffers = {reader->ReleaseIsValid(),
                                                       reader->ReleaseValues()};
       data = std::make_shared<::arrow::ArrayData>(field->type(), reader->values_written(),
                                                   std::move(buffers), reader->null_count());
     } else {
       std::vector<std::shared_ptr<Buffer>> buffers = {nullptr, reader->ReleaseValues()};
       data = std::make_shared<::arrow::ArrayData>(field->type(), reader->values_written(),
                                                   std::move(buffers), /*null_count=*/0);
     }
     return ::arrow::MakeArray(data);
   }
   ```
   
   `parquet/arrow` has considering these cases.


-- 
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: github-unsubscribe@arrow.apache.org

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