You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@arrow.apache.org by "icexelloss (via GitHub)" <gi...@apache.org> on 2023/09/08 13:59:23 UTC

[GitHub] [arrow] icexelloss opened a new issue, #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   ### Describe the bug, including details regarding any error messages, version, and platform.
   
   I have been testing "What is the max rss needed to scan through ~100G of data in a parquet stored in gcs using Arrow C++". 
   
   The current answer is about ~6G of memory which seems a bit high so I looked into it. What I observed during the process led me to think that there are some potential cache/memory issues in the dataset/parquet cpp code.
   
   Main observation:
   (1) As I am scanning through the dataset, I printed out (a) memory allocated by the memory pool from ScanOptions (b) process rss. I found that while (a) stays pretty stable throughout the scan (stays < 1G), (b) keeps increasing during the scan (looks linear to the number of files scanned).
   (2) I tested ScanNode in Arrow as well as an in-house library that implements its own "S3Dataset" similar to Arrow dataset, both showing similar rss usage. (Which led me to think the issue is more likely to be in the parquet cpp code instead of dataset code).
   (3) Scan the same dataset twice in the same process doesn't increase the max rss.
   
   Following suggestions from the mailing list, I also did a memory profiling of the test program and the results seem to indicate potential memory leak.
   ![Screenshot from 2023-09-08 09-54-31](https://github.com/apache/arrow/assets/793516/c6b0ada6-fe93-4a8d-848e-d5540848c82e)
   
   
   ### Component(s)
   
   C++, Parquet


-- 
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] icexelloss commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   Btw is there a way that I can attach the massif output in github issue? It says unsupported format when I tried.


-- 
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] westonpace commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   > I believe [I might be be experiencing this same problem through the Python API](https://github.com/apache/arrow/issues/37820). Having to maintain a local build of Arrow doesn't sound like the right solution, so I wonder if there are ideas of how to achieve the same result as @icexelloss 's last comment, but through Arrow's normal APIs?
   
   We should definitely make metadata caching an optional feature of the scanner and/or dataset.  I think the API could be as simple as...
   
   ```
   my dataset = pyarrow.dataset.dataset(..., cache_metadata=False)
   ```
   
   Any place that is using a dataset "temporarily" should also set this to false (e.g. when we run pyarrow.parquet.read_table it creates a dataset and scans it.  That temporary dataset should NOT cache metadata)


-- 
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] westonpace commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   Yes, the fragments are part of the dataset.  So they stay around until the dataset is destroyed which has to be after the scan.


-- 
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] icexelloss commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   Not the expert on this but does this mean that 
   
   > You could upload it in a gist and link that here, perhaps?
   
   I ended up changing the file extension and that worked :)


-- 
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] westonpace commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   Note that FileFragment in the datasets API caches the parquet metadata (with no option to disable this at the moment).  So if you are scanning many files you will see memory grow over the lifetime of the scan as more and more metadatas are cached.  I would expect a second scan would not grow the memory.


-- 
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] IamJeffG commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   > maybe when users have too many columns
   
   Maybe but I think there must be more to it.  [In my example](https://github.com/apache/arrow/issues/37820) I have a partitioned parquet dataset on local disk, 8.6GB in total, with 13 columns and 38,747 fragments.  Writing this dataset to a new location on disk (i.e. to compact the fragments) consumes all 8GB of RAM on my machine and then swaps to disk.  I can't imagine that 13 columns or even 13×38,747 takes upwards of 8GB of memory.


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


Re: [I] [C++] Potential memory leak in Parquet reading with Dataset [arrow]

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

   > Sorry I have not found out the reason for the seemly leak of "arrow::Field" in [#37630 (comment)](https://github.com/apache/arrow/issues/37630#issuecomment-1714569496) post.
   
   Oh, in your graph the `arrow::Field` and schema occupied lots of memory. Release Manifest would release them. Previously I wonder if a schema cache would be better. But I think clear the pointer in scanner api would be much 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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


[GitHub] [arrow] icexelloss commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   cc @mapleFU @wgtmac @pitrou @felipecrv (Folks that have been opining on the mailing list)
   


-- 
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] icexelloss commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   > Note that FileFragment in the datasets API caches the parquet metadata (with no option to disable this at the moment). So if you are scanning many files you will see memory grow over the lifetime of the scan as more and more metadatas are cached. I would expect a second scan would not grow the memory.
   
   Thanks @westonpace, can you give a pointer to where that is happening?


-- 
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 #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   You can do `.reset()` on `shared_ptrs` to achieve the same effect. [1] I would call the method `ResetMetadata()` because "clear" in C++ usually means semantically clearing data, but not really deallocating the buffers (e.g. `std::string::clear()` and `std::vector<T>::clear()`).
   
   [1] https://en.cppreference.com/w/cpp/memory/shared_ptr/reset


-- 
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] icexelloss commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   FWIW, This is my local patch:
   ```
   diff --git a/c/src/arrow/dataset/dataset.h b/c/src/arrow/dataset/dataset.h
   index 1db230b16e9c2e52ad58c02255f87375307271d0..37c2ddca4884d32bb1721dd3a2d937f9faf6b93f 100644
   --- a/c/src/arrow/dataset/dataset.h
   +++ b/c/src/arrow/dataset/dataset.h
   @@ -199,6 +199,9 @@ class ARROW_DS_EXPORT Fragment : public std::enable_shared_from_this<Fragment> {
        return partition_expression_;
      }
    
   +  /// \brief Clear metadata in the fragment. By default this is an noop.
   +  virtual void ClearMetadata() { return; }
   +
      virtual ~Fragment() = default;
    
     protected:
   diff --git a/c/src/arrow/dataset/file_parquet.h b/c/src/arrow/dataset/file_parquet.h
   index f33190bd93347f781df91a6cda612031f09caf75..632a555894a7b91ee6ac722a6820a9eaff61f232 100644
   --- a/c/src/arrow/dataset/file_parquet.h
   +++ b/c/src/arrow/dataset/file_parquet.h
   @@ -174,6 +174,11 @@ class ARROW_DS_EXPORT ParquetFileFragment : public FileFragment {
      static std::optional<compute::Expression> EvaluateStatisticsAsExpression(
          const Field& field, const parquet::Statistics& statistics);
    
   +  void ClearMetadata() override {
   +    metadata_ = NULLPTR;
   +    manifest_ = NULLPTR;
   +  }
   +
     private:
      ParquetFileFragment(FileSource source, std::shared_ptr<FileFormat> format,
                          compute::Expression partition_expression,
   diff --git a/c/src/arrow/dataset/scanner.cc b/c/src/arrow/dataset/scanner.cc
   index 18981d1451980f4b775fb87e42dcaede89a1fc7c..8f83a7c2dc52b8f5c728b91be2f93442168002d7 100644
   --- a/c/src/arrow/dataset/scanner.cc
   +++ b/c/src/arrow/dataset/scanner.cc
   @@ -1037,6 +1037,9 @@ Result<acero::ExecNode*> MakeScanNode(acero::ExecPlan* plan,
            // unnecessarily materialized columns in batch. We could drop them now instead of
            // letting them coast through the rest of the plan.
            auto guarantee = partial.fragment.value->partition_expression();
   +        // This is safe because by the time we reach here we have already decoded
   +        // the fragment into a batch so the metadata is no longer needed.
   +        partial.fragment.value->ClearMetadata();
    
            ARROW_ASSIGN_OR_RAISE(
                std::optional<compute::ExecBatch> batch,
   ```


-- 
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 #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   Future in arrow is an shared_future, which means if reference is hold, the object might not be dtor. I guess maybe not that problem, but I'll try to run the script and reproduce the problem.


-- 
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] icexelloss commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   Test code:
   https://gist.github.com/icexelloss/88195de046962e1d043c99d96e1b8b43


-- 
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] westonpace commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   In `file_parquet.h`.  The `ParquetFileFragment` class has these variables:
   
   ```
     /// Indices of row groups selected by this fragment,
     /// or std::nullopt if all row groups are selected.
     std::optional<std::vector<int>> row_groups_;
   
     std::vector<compute::Expression> statistics_expressions_;
     std::vector<bool> statistics_expressions_complete_;
     std::shared_ptr<parquet::FileMetaData> metadata_;
     std::shared_ptr<parquet::arrow::SchemaManifest> manifest_;
   ```
   
   They initially start empty/null and are initialized during a call to `EnsureCompleteMetadata`.


-- 
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 #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   I was so busy these two weeks, sorry for late reply. Have you find out the reason? Seems that you're suffering from too many `arrow::field`? @icexelloss 


-- 
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] icexelloss commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   Oh I see we keep a vector of Fragment, each Fragment has a shared_ptr to the schema of the file


-- 
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] icexelloss commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   In a local build I cleared 
   ```
     std::shared_ptr<parquet::FileMetaData> metadata_;
     std::shared_ptr<parquet::arrow::SchemaManifest> manifest_;
   ```
   after scanning a fragment and things appear to be in better shape. Memory usage goes from 4G -> 700+Mb.
   
   There still seem to be some leaking that I couldn't figure out but at least things seem to be in a much better shape after clearing those two fields.
   ![massif-after-fix](https://github.com/apache/arrow/assets/793516/7f529ce9-09d9-4ff3-89e3-c40ccc86b8e6)
   [massif.out.593871.txt](https://github.com/apache/arrow/files/12580295/massif.out.593871.txt)
   


-- 
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 #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   ```c++
   Result<std::vector<std::shared_ptr<FileFragment>>>
   ParquetDatasetFactory::CollectParquetFragments(const Partitioning& partitioning) {
     std::vector<std::shared_ptr<FileFragment>> fragments(paths_with_row_group_ids_.size());
   
     size_t i = 0;
     for (const auto& e : paths_with_row_group_ids_) {
       const auto& path = e.first;
       auto metadata_subset = metadata_->Subset(e.second);
   
       auto row_groups = Iota(metadata_subset->num_row_groups());
   
       auto partition_expression =
           partitioning.Parse(StripPrefix(path, options_.partition_base_dir))
               .ValueOr(compute::literal(true));
   
       ARROW_ASSIGN_OR_RAISE(
           auto fragment,
           format_->MakeFragment({path, filesystem_}, std::move(partition_expression),
                                 physical_schema_, std::move(row_groups)));
   
       RETURN_NOT_OK(fragment->SetMetadata(metadata_subset, manifest_));
       fragments[i++] = std::move(fragment);
     }
   
     return fragments;
   }
   ```
   
   I noticed that these metadata and manifest is `shared_ptr` and share among fragments, maybe when users have too many columns, this would be a huge cost?


-- 
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] icexelloss commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   > I believe [I might be be experiencing this same problem through the Python API](https://github.com/apache/arrow/issues/37820). Having to maintain a local build of Arrow doesn't sound like the right solution, so I wonder if there are ideas of how to achieve the same result as @icexelloss 's last comment, but through Arrow's normal APIs?
   
   I don't think you can at the moment. Because there is no way to tell Dataset to not to cache these metadata through normal API.


-- 
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] icexelloss commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   > In `file_parquet.h`. The `ParquetFileFragment` class has these variables:
   > 
   > ```
   >   /// Indices of row groups selected by this fragment,
   >   /// or std::nullopt if all row groups are selected.
   >   std::optional<std::vector<int>> row_groups_;
   > 
   >   std::vector<compute::Expression> statistics_expressions_;
   >   std::vector<bool> statistics_expressions_complete_;
   >   std::shared_ptr<parquet::FileMetaData> metadata_;
   >   std::shared_ptr<parquet::arrow::SchemaManifest> manifest_;
   > ```
   > 
   > They initially start empty/null and are initialized during a call to `EnsureCompleteMetadata`.
   
   Hmm, what do you mean by they are cached? Doesn't ParquetFileFragment get dtord after we scanned the fragment?


-- 
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] lidavidm commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   You could upload it in a gist and link that here, perhaps?


-- 
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] icexelloss commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   Not too familiar with the async/future code (nor massif outputs), but does this look like the Future objects are not destructed somehow (which has a reference to the parquet metadata object?)


-- 
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] icexelloss commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   > I was so busy these two weeks, sorry for late reply. Have you find out the reason? Seems that you're suffering from too many arrow::field? @icexelloss
   
   @mapleFU Sorry I have not found out the reason for the seemly leak of "arrow::Field" in https://github.com/apache/arrow/issues/37630#issuecomment-1714569496 post.
   
   


-- 
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] IamJeffG commented on issue #37630: [C++] Potential memory leak in Parquet reading with Dataset

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

   I believe [I might be be experiencing this same problem through the Python API](https://github.com/apache/arrow/issues/37820).  Having to maintain a local build of Arrow doesn't sound like the right solution, so I wonder if there are ideas of how to achieve the same result as @icexelloss 's last comment, but through Arrow's normal APIs?


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