Re: High memory usage with highly concurrent committers

2021-12-15 Thread Ryan Blue
ies to all files. We still want data (parquet) >>>> files to allocate 64 MiB (seems reasonable). For metadata, a smaller size >>>> is better. Is there a way to set the property based on file suffix or file >>>> type? >>>> >>>> >>>&

Re: High memory usage with highly concurrent committers

2021-12-06 Thread Piotr Findeisen
>> >> >> >> On Wed, Dec 1, 2021 at 3:20 PM Mayur Srivastava < >> mayur.srivast...@twosigma.com> wrote: >> >> That is correct Daniel. >> >> >> >> I’ve tried to explain our use of S3FileIO with GCS in the “Supporting >> gs:// pref

RE: High memory usage with highly concurrent committers

2021-12-03 Thread Mayur Srivastava
From: Igor Dvorzhak Sent: Thursday, December 2, 2021 8:09 PM To: dev@iceberg.apache.org Subject: Re: High memory usage with highly concurrent committers For each written object GCS connector allocates ~64MiB of memory by default to improve performance of large object writes. If you want to reduce

RE: High memory usage with highly concurrent committers

2021-12-01 Thread Mayur Srivastava
That is correct Daniel. I’ve tried to explain our use of S3FileIO with GCS in the “Supporting gs:// prefix …” thread. Thanks, Mayur From: Daniel Weeks Sent: Wednesday, December 1, 2021 11:46 AM To: Iceberg Dev List Subject: Re: High memory usage with highly concurrent committers I feel like

Re: High memory usage with highly concurrent committers

2021-12-01 Thread Daniel Weeks
gt;> So, the high memory problem manifests only when using the default >> HadoopFileSystem. >> >> >> >> Thanks, Mayur >> >> >> >> PS: I had to change S3FileIO locally to accept gs:// prefix so that it >> works with GCS. Is there

Re: High memory usage with highly concurrent committers

2021-12-01 Thread Jack Ye
problem manifests only when using the default > HadoopFileSystem. > > > > Thanks, Mayur > > > > PS: I had to change S3FileIO locally to accept gs:// prefix so that it > works with GCS. Is there a plan to support gs:// prefix in the S3URI? > > > > *From:*

RE: High memory usage with highly concurrent committers

2021-11-30 Thread Mayur Srivastava
accept gs:// prefix so that it works with GCS. Is there a plan to support gs:// prefix in the S3URI? From: Ryan Blue Sent: Tuesday, November 30, 2021 3:53 PM To: Iceberg Dev List Subject: Re: High memory usage with highly concurrent committers Mayur, Is it possible to connect to this proce

Re: High memory usage with highly concurrent committers

2021-11-30 Thread Ryan Blue
Mayur, Is it possible to connect to this process with a profiler and look at what's taking up all of the space? I suspect that what's happening here is that you're loading the list of snapshots for each version of metadata, so you're holding a lot of copies of the entire snapshot history and poss