Uploaded image for project: 'Apache Arrow'
  1. Apache Arrow
  2. ARROW-17984

pq.read_table doesn't seem to be thread safe

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 9.0.0
    • None
    • Parquet
    • None

    Description

      Before PR: https://github.com/apache/arrow/pull/13799 gets merged in master, I am using multithreading to improve read bandwidth from S3. Even after that PR gets merged, I probably will still try to use multithreading to some extent.

      However pq.read_table from S3 doesn't seem to be thread safe. Seems like it uses the new dataset reader under the hood. I cannot provide a reproduction, not a stable one anyway. But this is roughly the script I have been using 

      ~~~

      def get_next_batch(self, mapper_id, pos=None):

      def download(file):
          return pq.read_table("s3://" + self.bucket + "/" + file, columns=self.columns, filters=self.filters)
       
      executor = concurrent.futures.ThreadPoolExecutor(max_workers=self.workers)

      futures= {executor.submit(download, file): file for file in my_files}
      for future inconcurrent.futures.as_completed(futures):
          yield  future.result()
      ~~~
      The errors all have to do with malloc segfaults which makes me suspect the connection object is being reused across different pq.read_table invocations in different threads
      ```
      (InputReaderNode pid=25001, ip=172.31.60.29) malloc_consolidate(): invalid chunk size
      (InputReaderNode pid=25001, ip=172.31.60.29) *** SIGABRT received at time=1665464922 on cpu 9 ***
      (InputReaderNode pid=25001, ip=172.31.60.29) PC: @     0x7f9a480a803b  (unknown)  raise
      (InputReaderNode pid=25001, ip=172.31.60.29)     @     0x7f9a480a80c0       4160  (unknown)
      (InputReaderNode pid=25001, ip=172.31.60.29)     @     0x7f9a480fa32c  (unknown)  (unknown)
      ```
      Note, this multithreaded code is running inside a Ray actor process, but that shouldn't be a problem.

      Attachments

        1. _usr_bin_python3.8.1000.crash
          45.16 MB
          Ziheng Wang

        Activity

          People

            Unassigned Unassigned
            marsupialtail Ziheng Wang
            Votes:
            0 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

              Created:
              Updated: