You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by "CamilAbraham (via GitHub)" <gi...@apache.org> on 2023/08/14 11:23:31 UTC

[GitHub] [avro] CamilAbraham opened a new pull request, #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

CamilAbraham opened a new pull request, #2439:
URL: https://github.com/apache/avro/pull/2439

   <!--
   
   *Thank you very much for contributing to Apache Avro - we are happy that you want to help us improve Avro. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Avro a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/AVRO/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "AVRO-XXXX: [component] Title of the pull request", where *AVRO-XXXX* should be replaced by the actual issue number. 
       The *component* is optional, but can help identify the correct reviewers faster: either the language ("java", "python") or subsystem such as "build" or "doc" are good candidates.  
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests. You can [build the entire project](https://github.com/apache/avro/blob/master/BUILD.md) or just the [language-specific SDK](https://avro.apache.org/project/how-to-contribute/#unit-tests).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Every commit message references Jira issues in their subject lines. In addition, commits follow the guidelines from [How to write a good git commit message](https://chris.beams.io/posts/git-commit/)
       1. Subject is separated from body by a blank line
       1. Subject is limited to 50 characters (not including Jira issue reference)
       1. Subject does not end with a period
       1. Subject uses the imperative mood ("add", not "adding")
       1. Body wraps at 72 characters
       1. Body explains "what" and "why", not "how"
   
   -->
   
   ## What is the purpose of the change
   
   The aim of this pull request is to improve memory usage when reading Avro encoded files. Fixing AVRO-3802.
   
   This issue was detected when using Apache.Avro C# library for compression and decompression of large Avro files. Each file is 2-3MB and decompression of 20 files is using 2.1GB memory and memory keep increasing around 200-300MB for follow up calls. 
   
   Attached are images taken of memory profiler of three calls before fix. Each call decompress the same 20 files. First call uses 2.1GB, increase to 2.37GB and then 2.68GB. 
   
   Call tree is showing most of the memory are used by `DeflatCodec.CopyTo` method. 
   
   - `CopyTo` method is not using Array pool. Default Microsoft's stream `copyTo` method is much more efficient. 
   - `inStream` and `outStream` memory streams are not in using statement.
   - On `outstream` object creation, object can be initialized with defined length/stream capacity. 
   
   ![image](https://github.com/apache/avro/assets/72411444/9078647b-5a30-45f0-a9c4-1314cc1548f8)
   ![image](https://github.com/apache/avro/assets/72411444/bf2d0ece-f193-4240-a70b-297bae852b36)
   
   ## Verifying this change
   
   Added test that validates that memory usage after reading records from file reader returns to its initial value. Proving that the reader object is disposed correctly.
   
   
   ## Documentation
   
   No new features added.
   


-- 
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: dev-unsubscribe@avro.apache.org

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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "KalleOlaviNiemitalo (via GitHub)" <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1295875980


##########
lang/csharp/src/apache/main/IO/BinaryDecoder.cs:
##########
@@ -26,14 +26,25 @@ namespace Avro.IO
     public partial class BinaryDecoder : Decoder, IDisposable
     {
         private readonly Stream stream;
+        private readonly bool ownStream;
 
         /// <summary>
         /// Initializes a new instance of the <see cref="BinaryDecoder"/> class.
         /// </summary>
         /// <param name="stream">Stream to decode.</param>
-        public BinaryDecoder(Stream stream)
+        public BinaryDecoder(Stream stream) : this(stream, false)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="BinaryDecoder"/> class.
+        /// </summary>
+        /// <param name="stream">Stream to decode.</param>
+        /// <param name="ownStream">Leave stream open after disposing the object.</param>

Review Comment:
   The [StreamReader constructor](https://learn.microsoft.com/dotnet/api/system.io.streamreader.-ctor?view=netstandard-2.0#system-io-streamreader-ctor(system-io-stream-system-text-encoding-system-boolean-system-int32-system-boolean)) has a `bool leaveOpen` parameter for a similar purpose.  The default value is `false`.
   
   The [HttpClient constructor](https://learn.microsoft.com/dotnet/api/system.net.http.httpclient.-ctor?view=netstandard-2.0#system-net-http-httpclient-ctor(system-net-http-httpmessagehandler-system-boolean)) instead has a `bool disposeHandler` parameter.  The default value is `true`.
   
   To preserve compatibility with earlier Avro versions while being easy to understand, I think this should be `bool disposeStream` and default to `false`.  The caller tells BinaryDecoder whether BinaryDecoder should dispose the stream.



-- 
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@avro.apache.org

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


[GitHub] [avro] KalleOlaviNiemitalo commented on pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "KalleOlaviNiemitalo (via GitHub)" <gi...@apache.org>.
KalleOlaviNiemitalo commented on PR #2439:
URL: https://github.com/apache/avro/pull/2439#issuecomment-1680627634

   @zcsizmadia, I agree the IDispose implementation on BinaryEncoder and BinaryDecoder can be omitted from this PR.  Especially as this PR does not make DataFileReader\<T>.Dispose() call BinaryDecoder.Dispose on its `_decoder` or `_datumDecoder`.
   
   If DataFileReader\<T> is changed to dispose of streams other than its main `_stream`, and it is convenient to do that as part of BinaryDecoder disposal, then one can also define `private class DisposingBinaryDecoder : BinaryDecoder, IDisposable` within DataFileReader\<T> and it won't be a change in public 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: issues-unsubscribe@avro.apache.org

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


[GitHub] [avro] zcsizmadia commented on pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "zcsizmadia (via GitHub)" <gi...@apache.org>.
zcsizmadia commented on PR #2439:
URL: https://github.com/apache/avro/pull/2439#issuecomment-1680651039

   @CamilAbraham Just a little bit reasoning behind this is if the user has CA2000 (or IDE0067) warning enabled  in their code, after making BinaryEncoder or Decoder IDisposable they will get compiler/analyzer warning(s). Just fyi the avro code analyzer config we use, disables CA2000 (https://github.com/apache/avro/blob/master/lang/csharp/CodeAnalysis.src.globalconfig#L413-L414)
   
   https://learn.microsoft.com/en-us/dotnet/fundamentals/code-analysis/quality-rules/ca2000
   


-- 
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@avro.apache.org

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


[GitHub] [avro] zcsizmadia commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "zcsizmadia (via GitHub)" <gi...@apache.org>.
zcsizmadia commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1295849834


##########
lang/csharp/src/apache/main/IO/BinaryDecoder.cs:
##########
@@ -298,6 +309,10 @@ private void Skip(long p)
         }
 
         /// <inheritdoc />
-        public void Dispose() => stream?.Dispose();
+        public void Dispose()
+        {
+            if(!ownStream)

Review Comment:
   This condition IMO should be `if (ownStream) { stream?.Dispose(); }`



##########
lang/csharp/src/apache/main/IO/BinaryDecoder.cs:
##########
@@ -26,14 +26,25 @@ namespace Avro.IO
     public partial class BinaryDecoder : Decoder, IDisposable
     {
         private readonly Stream stream;
+        private readonly bool ownStream;
 
         /// <summary>
         /// Initializes a new instance of the <see cref="BinaryDecoder"/> class.
         /// </summary>
         /// <param name="stream">Stream to decode.</param>
-        public BinaryDecoder(Stream stream)
+        public BinaryDecoder(Stream stream) : this(stream, false)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="BinaryDecoder"/> class.
+        /// </summary>
+        /// <param name="stream">Stream to decode.</param>
+        /// <param name="ownStream">Leave stream open after disposing the object.</param>

Review Comment:
   For me the ownStream name indicates that this object (BinaryDecoder) will now takes ownership of the stream. So ownStream means "Stream is disposed when this object is disposed".



-- 
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@avro.apache.org

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


[GitHub] [avro] zcsizmadia commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "zcsizmadia (via GitHub)" <gi...@apache.org>.
zcsizmadia commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1293496616


##########
lang/csharp/src/apache/main/IO/BinaryDecoder.cs:
##########
@@ -296,5 +296,8 @@ private void Skip(long p)
         {
             stream.Seek(p, SeekOrigin.Current);
         }
+
+        /// <inheritdoc />
+        public void Dispose() => stream?.Dispose();

Review Comment:
   If the stream should be disposed when The BinaryDecoder is disposed, a new constructor with `bool ownStream` can be added, which marks the stream for disposal in the Decoder's Dispose. Similiarly what https://github.com/apache/avro/blob/master/lang/csharp/src/apache/main/File/DataFileReader.cs#L125-L137. Of course that feature is not really in the scope of 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.

To unsubscribe, e-mail: issues-unsubscribe@avro.apache.org

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


[GitHub] [avro] github-code-scanning[bot] commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1293668358


##########
lang/csharp/src/apache/main/Specific/SpecificReader.cs:
##########
@@ -130,20 +130,22 @@
                 }
             }
 
-            var defaultStream = new MemoryStream();
-            var defaultEncoder = new BinaryEncoder(defaultStream);
-            var defaultDecoder = new BinaryDecoder(defaultStream);
-            foreach (Field rf in rs)
+            using (var defaultStream = new MemoryStream())
             {
-                if (writerSchema.Contains(rf.Name)) continue;
+                var defaultEncoder = new BinaryEncoder(defaultStream);
+                var defaultDecoder = new BinaryDecoder(defaultStream);
+                foreach (Field rf in rs)
+                {
+                    if (writerSchema.Contains(rf.Name)) continue;
 
-                defaultStream.Position = 0; // reset for writing
-                Resolver.EncodeDefaultValue(defaultEncoder, rf.Schema, rf.DefaultValue);
-                defaultStream.Flush();
-                defaultStream.Position = 0; // reset for reading
+                    defaultStream.Position = 0; // reset for writing
+                    Resolver.EncodeDefaultValue(defaultEncoder, rf.Schema, rf.DefaultValue);
+                    defaultStream.Flush();
+                    defaultStream.Position = 0; // reset for reading
 
-                obj = rec.Get(rf.Pos);
-                rec.Put(rf.Pos, Read(obj, rf.Schema, rf.Schema, defaultDecoder));
+                    obj = rec.Get(rf.Pos);
+                    rec.Put(rf.Pos, Read(obj, rf.Schema, rf.Schema, defaultDecoder));
+                }

Review Comment:
   ## Missed opportunity to use Where
   
   This foreach loop [implicitly filters its target sequence](1) - consider filtering the sequence explicitly using '.Where(...)'.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/3105)



##########
lang/csharp/src/apache/main/Specific/SpecificReader.cs:
##########
@@ -130,20 +130,22 @@
                 }
             }
 
-            var defaultStream = new MemoryStream();
-            var defaultEncoder = new BinaryEncoder(defaultStream);
-            var defaultDecoder = new BinaryDecoder(defaultStream);
-            foreach (Field rf in rs)
+            using (var defaultStream = new MemoryStream())
             {
-                if (writerSchema.Contains(rf.Name)) continue;
+                var defaultEncoder = new BinaryEncoder(defaultStream);
+                var defaultDecoder = new BinaryDecoder(defaultStream);
+                foreach (Field rf in rs)
+                {
+                    if (writerSchema.Contains(rf.Name)) continue;
 
-                defaultStream.Position = 0; // reset for writing
-                Resolver.EncodeDefaultValue(defaultEncoder, rf.Schema, rf.DefaultValue);
-                defaultStream.Flush();
-                defaultStream.Position = 0; // reset for reading
+                    defaultStream.Position = 0; // reset for writing
+                    Resolver.EncodeDefaultValue(defaultEncoder, rf.Schema, rf.DefaultValue);
+                    defaultStream.Flush();
+                    defaultStream.Position = 0; // reset for reading
 
-                obj = rec.Get(rf.Pos);
-                rec.Put(rf.Pos, Read(obj, rf.Schema, rf.Schema, defaultDecoder));
+                    obj = rec.Get(rf.Pos);

Review Comment:
   ## Dereferenced variable may be null
   
   Variable [rec](1) may be null at this access because of [this](2) assignment.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/3104)



##########
lang/csharp/src/apache/test/File/FileTests.cs:
##########
@@ -668,6 +668,55 @@
             }
         }
 
+        [Test]
+        public void TestDeflateReadMemoryUsage([Values(specificSchema)] string schemaStr)
+        {
+            // create and write out
+            IList<Foo> records = MakeRecords(GetTestFooObject());
+
+            Process currentProcess = Process.GetCurrentProcess();
+
+            MemoryStream dataFileOutputStream = new MemoryStream();
+
+            Schema schema = Schema.Parse(schemaStr);
+            DatumWriter<Foo> writer = new SpecificWriter<Foo>(schema);
+            using (IFileWriter<Foo> dataFileWriter = DataFileWriter<Foo>.OpenWriter(writer, dataFileOutputStream, Codec.CreateCodec(Codec.Type.Deflate)))
+            {
+                for (int i = 0; i < 10; ++i)
+                {
+                    foreach (Foo foo in records)
+                    {
+                        dataFileWriter.Append(foo);
+                    }
+
+                    // write out block
+                    if (i == 1 || i == 4)
+                    {
+                        dataFileWriter.Sync();
+                    }
+                }
+            }
+
+            long startMemoryUsedBytes = currentProcess.WorkingSet64;
+
+            MemoryStream dataFileInputStream = new MemoryStream(dataFileOutputStream.ToArray());
+            dataFileInputStream.Position = 0;
+
+            // read back
+            IList<Foo> readRecords = new List<Foo>();

Review Comment:
   ## Container contents are never accessed
   
   The contents of this container are never accessed.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/3107)



-- 
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@avro.apache.org

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


[GitHub] [avro] CamilAbraham commented on pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "CamilAbraham (via GitHub)" <gi...@apache.org>.
CamilAbraham commented on PR #2439:
URL: https://github.com/apache/avro/pull/2439#issuecomment-1685918922

   It's my first time contributing to this project, what is the process to get this change merged?


-- 
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@avro.apache.org

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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "KalleOlaviNiemitalo (via GitHub)" <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1295882191


##########
lang/csharp/src/apache/main/IO/BinaryDecoder.cs:
##########
@@ -26,14 +26,25 @@ namespace Avro.IO
     public partial class BinaryDecoder : Decoder, IDisposable
     {
         private readonly Stream stream;
+        private readonly bool ownStream;
 
         /// <summary>
         /// Initializes a new instance of the <see cref="BinaryDecoder"/> class.
         /// </summary>
         /// <param name="stream">Stream to decode.</param>
-        public BinaryDecoder(Stream stream)
+        public BinaryDecoder(Stream stream) : this(stream, false)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="BinaryDecoder"/> class.
+        /// </summary>
+        /// <param name="stream">Stream to decode.</param>
+        /// <param name="ownStream">Leave stream open after disposing the object.</param>

Review Comment:
   Alternatively, it could be `bool leaveOpen` and default to `true`; but I feel `false` is generally nicer than `true` as a default value of a `bool` parameter.



-- 
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@avro.apache.org

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


[GitHub] [avro] zcsizmadia commented on pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "zcsizmadia (via GitHub)" <gi...@apache.org>.
zcsizmadia commented on PR #2439:
URL: https://github.com/apache/avro/pull/2439#issuecomment-1677403975

   > > * `inStream` and `outStream` memory streams are not in using statement.
   > 
   > While `using` is a good practice, I don't believe it will have any measurable effect on memory consumption or execution time when used on a `MemoryStream`. Those streams don't hold any unmanaged resources, and `Dispose` won't speed up the garbage collection of the underlying `byte[]`, especially if the `MemoryStream` itself becomes garbage immediately afterwards.
   
   In the case of MemoryStream, the using statement is probably is just a good practice, however when disposable object are used in C#, IMO it should be clear how an object is disposed, without knowing hwo the object is implemented underneath the hood or it uses unmanaged objects or not. E.g. it is very trivial to convert a memorystream in->out code to FileStream. And in that case  the disposed is mandatory to avoid issues, e.g. files being kept open.


-- 
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@avro.apache.org

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


[GitHub] [avro] zcsizmadia merged pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "zcsizmadia (via GitHub)" <gi...@apache.org>.
zcsizmadia merged PR #2439:
URL: https://github.com/apache/avro/pull/2439


-- 
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: dev-unsubscribe@avro.apache.org

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


[GitHub] [avro] zcsizmadia commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "zcsizmadia (via GitHub)" <gi...@apache.org>.
zcsizmadia commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1293479830


##########
lang/csharp/src/apache/main/IO/BinaryEncoder.cs:
##########
@@ -228,5 +228,8 @@ public void Flush()
         {
             Stream.Flush();
         }
+
+        /// <inheritdoc />
+        public void Dispose() => Stream?.Dispose();

Review Comment:
   Same issue as BinaryDecoder.Dispose



##########
lang/csharp/src/apache/test/File/FileTests.cs:
##########
@@ -668,6 +668,55 @@ public void TestPartialReadAll([Values(specificSchema)] string schemaStr, [Value
             }
         }
 
+        [Test]
+        public void TestDeflateReadMemoryUsage([Values(specificSchema)] string schemaStr)
+        {
+            // create and write out
+            IList<Foo> records = MakeRecords(GetTestFooObject());
+
+            Process currentProcess = Process.GetCurrentProcess();
+
+            MemoryStream dataFileOutputStream = new MemoryStream();
+
+            Schema schema = Schema.Parse(schemaStr);
+            DatumWriter<Foo> writer = new SpecificWriter<Foo>(schema);
+            using (IFileWriter<Foo> dataFileWriter = DataFileWriter<Foo>.OpenWriter(writer, dataFileOutputStream, Codec.CreateCodec(Codec.Type.Deflate)))
+            {
+                for (int i = 0; i < 10; ++i)
+                {
+                    foreach (Foo foo in records)
+                    {
+                        dataFileWriter.Append(foo);
+                    }
+
+                    // write out block
+                    if (i == 1 || i == 4)
+                    {
+                        dataFileWriter.Sync();
+                    }
+                }
+            }
+
+            long startMemoryUsedBytes = currentProcess.WorkingSet64;
+
+            MemoryStream dataFileInputStream = new MemoryStream(dataFileOutputStream.ToArray());
+            dataFileInputStream.Position = 0;
+
+            // read back
+            IList<Foo> readRecords = new List<Foo>();
+            using (IFileReader<Foo> reader = DataFileReader<Foo>.OpenReader(dataFileInputStream, schema))
+            {
+                // read records from synced position
+                foreach (Foo rec in reader.NextEntries)
+                    readRecords.Add(rec);
+            }
+
+            long totalMemoryUsedBytes = currentProcess.WorkingSet64 - startMemoryUsedBytes;
+
+            Assert.IsTrue(totalMemoryUsedBytes  == 0, "Total memory usage in working set");

Review Comment:
   Agreed with @KalleOlaviNiemitalo. This test is unreliable and IMO not needed.



##########
lang/csharp/src/apache/main/File/DeflateCodec.cs:
##########
@@ -58,18 +58,19 @@ public override void Compress(MemoryStream inputStream, MemoryStream outputStrea
         /// <inheritdoc/>
         public override byte[] Decompress(byte[] compressedData, int length)
         {
-
-            MemoryStream inStream = new MemoryStream(compressedData);
-            MemoryStream outStream = new MemoryStream();
-
-            using (DeflateStream Decompress =
-                        new DeflateStream(inStream,
-                        CompressionMode.Decompress))
+            using (MemoryStream inStream = new MemoryStream(compressedData))
             {
-                CopyTo(Decompress, outStream);
+                using (MemoryStream outStream = new MemoryStream(inStream.Capacity))

Review Comment:
   Personally I like to stack the using in the beginning in cases like this. e.g. https://github.com/apache/avro/blob/master/lang/csharp/src/apache/codec/Avro.File.BZip2/BZip2.cs#L78-L79
   
   Addistionaly use the MemoryStream constructor to limit the inSTream access only to length, mentioned by @KalleOlaviNiemitalo 



##########
lang/csharp/src/apache/main/IO/BinaryDecoder.cs:
##########
@@ -296,5 +296,8 @@ private void Skip(long p)
         {
             stream.Seek(p, SeekOrigin.Current);
         }
+
+        /// <inheritdoc />
+        public void Dispose() => stream?.Dispose();

Review Comment:
   This is a breaking change. This assumes, that the BinaryDecoder object takes ownership if the stream and once the BinaryDecoder is disposed, the stream is disposed as well and cannot be used any more. Most of the nurmal use cases the stream can be disposed here without any harm, however the calling code might need to keep trhe stream alive in special cases, e.g. there is still some additional data in the stream for other purposes.
   



-- 
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@avro.apache.org

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


[GitHub] [avro] zcsizmadia commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "zcsizmadia (via GitHub)" <gi...@apache.org>.
zcsizmadia commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1293513432


##########
lang/csharp/src/apache/main/Specific/SpecificReader.cs:
##########
@@ -130,20 +130,22 @@ protected override object ReadRecord(object reuse, RecordSchema writerSchema, Sc
                 }
             }
 
-            var defaultStream = new MemoryStream();

Review Comment:
   Similar code like this is in GenericReader and PreresolvingDatumReader. If SpecificReader fixes the MemoryStream leak, thos others should be fixed as well.



-- 
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@avro.apache.org

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


[GitHub] [avro] zcsizmadia commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "zcsizmadia (via GitHub)" <gi...@apache.org>.
zcsizmadia commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1295964336


##########
lang/csharp/src/apache/main/Generic/GenericReader.cs:
##########
@@ -297,8 +297,6 @@ protected virtual object ReadRecord(object reuse, RecordSchema writerSchema, Sch
                 var defaultDecoder = new BinaryDecoder(defaultStream);
                 foreach (Field rf in rs.Fields.Where(rf => !writerSchema.Contains(rf.Name)))

Review Comment:
   CodeQL warnings like this, I ussuallyh just ignore, since it is really not a concern in a legacy code like avro. However thanks for getting rid of it ;)



-- 
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@avro.apache.org

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


[GitHub] [avro] zcsizmadia commented on pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "zcsizmadia (via GitHub)" <gi...@apache.org>.
zcsizmadia commented on PR #2439:
URL: https://github.com/apache/avro/pull/2439#issuecomment-1680601602

   @CamilAbraham @KalleOlaviNiemitalo After thinking more about the ownstream/leaveOpen issue. Adding that featurte requires to convert the BinaryEncoder and Decoder to become IDisposable. I dont think that should be in the scope of this ticket. My vote is not to add the Dispose feature to the Encoder and Decoder
   


-- 
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@avro.apache.org

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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "KalleOlaviNiemitalo (via GitHub)" <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1306652260


##########
lang/csharp/src/apache/main/Generic/GenericReader.cs:
##########
@@ -297,8 +297,6 @@ protected virtual object ReadRecord(object reuse, RecordSchema writerSchema, Sch
                 var defaultDecoder = new BinaryDecoder(defaultStream);
                 foreach (Field rf in rs.Fields.Where(rf => !writerSchema.Contains(rf.Name)))

Review Comment:
   I expect this change makes the code slower, as it now has to allocate a closure and a delegate, and the calls become indirect too.  Roslyn contribution guidelines advise "Avoid LINQ" for that reason.  So I'd prefer reverting this and disabling the CodeQL warning.
   
   * <https://github.com/dotnet/roslyn/blob/34268d1bb9370c7b01c742303a895a99daf10d6a/CONTRIBUTING.md?plain=1#L83>
   * <https://stackoverflow.com/questions/22894877/avoid-allocations-in-compiler-hot-paths-roslyn-coding-conventions>



##########
lang/csharp/src/apache/main/File/DeflateCodec.cs:
##########
@@ -58,32 +58,14 @@ public override void Compress(MemoryStream inputStream, MemoryStream outputStrea
         /// <inheritdoc/>
         public override byte[] Decompress(byte[] compressedData, int length)
         {
-
-            MemoryStream inStream = new MemoryStream(compressedData);
-            MemoryStream outStream = new MemoryStream();
-
-            using (DeflateStream Decompress =
-                        new DeflateStream(inStream,
-                        CompressionMode.Decompress))
-            {
-                CopyTo(Decompress, outStream);
-            }
-
-            return outStream.ToArray();
-        }
-
-        /// <summary>
-        /// Copies to stream.
-        /// </summary>
-        /// <param name="from">stream you are copying from</param>
-        /// <param name="to">stream you are copying to</param>
-        private static void CopyTo(Stream from, Stream to)
-        {
-            byte[] buffer = new byte[4096];
-            int read;
-            while ((read = from.Read(buffer, 0, buffer.Length)) != 0)
+            using (MemoryStream inStream = new MemoryStream(compressedData, 0, length))

Review Comment:
   Could make `MemoryStream inStream` read-only by [constructing it](https://learn.microsoft.com/dotnet/api/system.io.memorystream.-ctor#system-io-memorystream-ctor(system-byte()-system-int32-system-int32-system-boolean)) with `writable: false` (the default is `true`).  Not important though, as DeflateStream with CompressionMode.Decompress won't try to write to it anyway.



-- 
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@avro.apache.org

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


[GitHub] [avro] CamilAbraham commented on pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "CamilAbraham (via GitHub)" <gi...@apache.org>.
CamilAbraham commented on PR #2439:
URL: https://github.com/apache/avro/pull/2439#issuecomment-1680632383

   > @CamilAbraham @KalleOlaviNiemitalo After thinking more about the ownstream/leaveOpen issue. Adding that featurte requires to convert the BinaryEncoder and Decoder to become IDisposable. I dont think that should be in the scope of this ticket. My vote is not to add the Dispose feature to the Encoder and Decoder
   
   The reason I introduced this change since the beginning was because I believe that disposing the stream will reduce memory leakage. Is there a reason why making the object IDisposable is a bad idea? 


-- 
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@avro.apache.org

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


[GitHub] [avro] CamilAbraham commented on pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "CamilAbraham (via GitHub)" <gi...@apache.org>.
CamilAbraham commented on PR #2439:
URL: https://github.com/apache/avro/pull/2439#issuecomment-1680768109

   Thanks! That's really helpful


-- 
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@avro.apache.org

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


[GitHub] [avro] zcsizmadia commented on pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "zcsizmadia (via GitHub)" <gi...@apache.org>.
zcsizmadia commented on PR #2439:
URL: https://github.com/apache/avro/pull/2439#issuecomment-1686329212

   I will merge it. I was not in a rush to do so, wanted to make sure that @KalleOlaviNiemitalo has time to address any issues he might have.


-- 
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@avro.apache.org

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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "KalleOlaviNiemitalo (via GitHub)" <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1293370557


##########
lang/csharp/src/apache/main/File/DeflateCodec.cs:
##########
@@ -58,18 +58,19 @@ public override void Compress(MemoryStream inputStream, MemoryStream outputStrea
         /// <inheritdoc/>
         public override byte[] Decompress(byte[] compressedData, int length)
         {
-
-            MemoryStream inStream = new MemoryStream(compressedData);
-            MemoryStream outStream = new MemoryStream();
-
-            using (DeflateStream Decompress =
-                        new DeflateStream(inStream,
-                        CompressionMode.Decompress))
+            using (MemoryStream inStream = new MemoryStream(compressedData))
             {
-                CopyTo(Decompress, outStream);
+                using (MemoryStream outStream = new MemoryStream(inStream.Capacity))
+                {
+                    using (DeflateStream decompress =
+                           new DeflateStream(inStream,
+                               CompressionMode.Decompress))
+                    {
+                        decompress.CopyTo(outStream, length);

Review Comment:
   Instead, I think you could use the `MemoryStream(byte[] buffer, int index, int count)` constructor for `inStream`, to prevent the `DeflateStream` from reading more than `length` bytes of compressed data.



-- 
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@avro.apache.org

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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "KalleOlaviNiemitalo (via GitHub)" <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1293354478


##########
lang/csharp/src/apache/main/File/DeflateCodec.cs:
##########
@@ -58,18 +58,19 @@ public override void Compress(MemoryStream inputStream, MemoryStream outputStrea
         /// <inheritdoc/>
         public override byte[] Decompress(byte[] compressedData, int length)
         {
-
-            MemoryStream inStream = new MemoryStream(compressedData);
-            MemoryStream outStream = new MemoryStream();
-
-            using (DeflateStream Decompress =
-                        new DeflateStream(inStream,
-                        CompressionMode.Decompress))
+            using (MemoryStream inStream = new MemoryStream(compressedData))
             {
-                CopyTo(Decompress, outStream);
+                using (MemoryStream outStream = new MemoryStream(inStream.Capacity))

Review Comment:
   Is `inStream.Capacity` worthwhile here?  It's the compressed size, and the uncompressed data that is written to `outStream` could be much larger.  I guess it won't hurt.



-- 
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@avro.apache.org

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


[GitHub] [avro] KalleOlaviNiemitalo commented on pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "KalleOlaviNiemitalo (via GitHub)" <gi...@apache.org>.
KalleOlaviNiemitalo commented on PR #2439:
URL: https://github.com/apache/avro/pull/2439#issuecomment-1677374079

   > * `inStream` and `outStream` memory streams are not in using statement.
   
   While `using` is a good practice, I don't believe it will have any measurable effect on memory consumption or execution time when used on a `MemoryStream`.  Those streams don't hold any unmanaged resources, and `Dispose` won't speed up the garbage collection of the underlying `byte[]`, especially if the `MemoryStream` itself becomes garbage immediately afterwards.


-- 
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@avro.apache.org

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


[GitHub] [avro] zcsizmadia commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "zcsizmadia (via GitHub)" <gi...@apache.org>.
zcsizmadia commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1293513432


##########
lang/csharp/src/apache/main/Specific/SpecificReader.cs:
##########
@@ -130,20 +130,22 @@ protected override object ReadRecord(object reuse, RecordSchema writerSchema, Sc
                 }
             }
 
-            var defaultStream = new MemoryStream();

Review Comment:
   Similar code like this is in GenericReader and PreresolvingDatumReader. If SpecificReader fixes the MemoryStream leak, those others should be fixed as well.



-- 
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@avro.apache.org

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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "KalleOlaviNiemitalo (via GitHub)" <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1293368983


##########
lang/csharp/src/apache/main/File/DeflateCodec.cs:
##########
@@ -58,18 +58,19 @@ public override void Compress(MemoryStream inputStream, MemoryStream outputStrea
         /// <inheritdoc/>
         public override byte[] Decompress(byte[] compressedData, int length)
         {
-
-            MemoryStream inStream = new MemoryStream(compressedData);
-            MemoryStream outStream = new MemoryStream();
-
-            using (DeflateStream Decompress =
-                        new DeflateStream(inStream,
-                        CompressionMode.Decompress))
+            using (MemoryStream inStream = new MemoryStream(compressedData))
             {
-                CopyTo(Decompress, outStream);
+                using (MemoryStream outStream = new MemoryStream(inStream.Capacity))
+                {
+                    using (DeflateStream decompress =
+                           new DeflateStream(inStream,
+                               CompressionMode.Decompress))
+                    {
+                        decompress.CopyTo(outStream, length);

Review Comment:
   With this change, `private static void CopyTo(Stream from, Stream to)` is no longer used; please delete it then.



-- 
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@avro.apache.org

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


[GitHub] [avro] KalleOlaviNiemitalo commented on pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "KalleOlaviNiemitalo (via GitHub)" <gi...@apache.org>.
KalleOlaviNiemitalo commented on PR #2439:
URL: https://github.com/apache/avro/pull/2439#issuecomment-1677424213

   I meant, if DeflateStream.Read does more work per call than MemoryStream.Read.


-- 
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@avro.apache.org

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


[GitHub] [avro] zcsizmadia commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "zcsizmadia (via GitHub)" <gi...@apache.org>.
zcsizmadia commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1295858752


##########
lang/csharp/src/apache/main/IO/BinaryDecoder.cs:
##########
@@ -298,6 +309,10 @@ private void Skip(long p)
         }
 
         /// <inheritdoc />
-        public void Dispose() => stream?.Dispose();
+        public void Dispose()
+        {
+            if(!ownStream)

Review Comment:
   Since the ownStream = false in the default constructor, the stream would be disposed using the default constructor, which woulr make it a breaking change.



##########
lang/csharp/src/apache/main/IO/BinaryDecoder.cs:
##########
@@ -298,6 +309,10 @@ private void Skip(long p)
         }
 
         /// <inheritdoc />
-        public void Dispose() => stream?.Dispose();
+        public void Dispose()
+        {
+            if(!ownStream)

Review Comment:
   Since the ownStream = false in the default constructor, the stream would be disposed using the default constructor, which would make it a breaking change.



-- 
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@avro.apache.org

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


[GitHub] [avro] CamilAbraham commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "CamilAbraham (via GitHub)" <gi...@apache.org>.
CamilAbraham commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1295886622


##########
lang/csharp/src/apache/main/IO/BinaryDecoder.cs:
##########
@@ -26,14 +26,25 @@ namespace Avro.IO
     public partial class BinaryDecoder : Decoder, IDisposable
     {
         private readonly Stream stream;
+        private readonly bool ownStream;
 
         /// <summary>
         /// Initializes a new instance of the <see cref="BinaryDecoder"/> class.
         /// </summary>
         /// <param name="stream">Stream to decode.</param>
-        public BinaryDecoder(Stream stream)
+        public BinaryDecoder(Stream stream) : this(stream, false)
+        {
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="BinaryDecoder"/> class.
+        /// </summary>
+        /// <param name="stream">Stream to decode.</param>
+        /// <param name="ownStream">Leave stream open after disposing the object.</param>

Review Comment:
   This makes a lot of sense. Will change the descriptions and Dispose logic of the encoder and decoder



-- 
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@avro.apache.org

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


[GitHub] [avro] zcsizmadia commented on pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "zcsizmadia (via GitHub)" <gi...@apache.org>.
zcsizmadia commented on PR #2439:
URL: https://github.com/apache/avro/pull/2439#issuecomment-1677449522

   Agreed on addig new APIs to the Codecs base class/interface, which are more Stream based and not byte[]. There was this PR https://github.com/apache/avro/pull/1358 which added the Deflate(start, length) api, however I dont really remeber what the real reason was and I was too lazy to read the whole ticket :)


-- 
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@avro.apache.org

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


[GitHub] [avro] github-code-scanning[bot] commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1295870592


##########
lang/csharp/src/apache/main/Generic/GenericReader.cs:
##########
@@ -290,21 +290,23 @@
                 }
             }
 
-            var defaultStream = new MemoryStream();
-            var defaultEncoder = new BinaryEncoder(defaultStream);
-            var defaultDecoder = new BinaryDecoder(defaultStream);
-            foreach (Field rf in rs)
+            using (var defaultStream = new MemoryStream())
             {
-                if (writerSchema.Contains(rf.Name)) continue;
+                var defaultEncoder = new BinaryEncoder(defaultStream);
+                var defaultDecoder = new BinaryDecoder(defaultStream);
+                foreach (Field rf in rs)
+                {
+                    if (writerSchema.Contains(rf.Name)) continue;
 
-                defaultStream.Position = 0; // reset for writing
-                Resolver.EncodeDefaultValue(defaultEncoder, rf.Schema, rf.DefaultValue);
-                defaultStream.Flush();
-                defaultStream.Position = 0; // reset for reading
+                    defaultStream.Position = 0; // reset for writing
+                    Resolver.EncodeDefaultValue(defaultEncoder, rf.Schema, rf.DefaultValue);
+                    defaultStream.Flush();
+                    defaultStream.Position = 0; // reset for reading
 
-                object obj = null;
-                TryGetField(rec, rf.Name, rf.Pos, out obj);
-                AddField(rec, rf.Name, rf.Pos, Read(obj, rf.Schema, rf.Schema, defaultDecoder));
+                    object obj = null;
+                    TryGetField(rec, rf.Name, rf.Pos, out obj);
+                    AddField(rec, rf.Name, rf.Pos, Read(obj, rf.Schema, rf.Schema, defaultDecoder));
+                }

Review Comment:
   ## Missed opportunity to use Where
   
   This foreach loop [implicitly filters its target sequence](1) - consider filtering the sequence explicitly using '.Where(...)'.
   
   [Show more details](https://github.com/apache/avro/security/code-scanning/3115)



-- 
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@avro.apache.org

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


[GitHub] [avro] CamilAbraham commented on pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "CamilAbraham (via GitHub)" <gi...@apache.org>.
CamilAbraham commented on PR #2439:
URL: https://github.com/apache/avro/pull/2439#issuecomment-1680619890

   > @CamilAbraham @KalleOlaviNiemitalo After thinking more about the ownstream/leaveOpen issue. Adding that featurte requires to convert the BinaryEncoder and Decoder to become IDisposable. I dont think that should be in the scope of this ticket. My vote is not to add the Dispose feature to the Encoder and Decoder
   
   This was done since the beginning of the PR. The idea was that those 2 objects would dispose the Stream, reducing memory leakage (IMO making it part of the scope of the change). After @KalleOlaviNiemitalo pointed out that this would break some specific usages, the `ownStream` flag was introduced.  


-- 
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@avro.apache.org

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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "KalleOlaviNiemitalo (via GitHub)" <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1293368344


##########
lang/csharp/src/apache/main/File/DeflateCodec.cs:
##########
@@ -58,18 +58,19 @@ public override void Compress(MemoryStream inputStream, MemoryStream outputStrea
         /// <inheritdoc/>
         public override byte[] Decompress(byte[] compressedData, int length)
         {
-
-            MemoryStream inStream = new MemoryStream(compressedData);
-            MemoryStream outStream = new MemoryStream();
-
-            using (DeflateStream Decompress =
-                        new DeflateStream(inStream,
-                        CompressionMode.Decompress))
+            using (MemoryStream inStream = new MemoryStream(compressedData))
             {
-                CopyTo(Decompress, outStream);
+                using (MemoryStream outStream = new MemoryStream(inStream.Capacity))
+                {
+                    using (DeflateStream decompress =
+                           new DeflateStream(inStream,
+                               CompressionMode.Decompress))
+                    {
+                        decompress.CopyTo(outStream, length);

Review Comment:
   `length` seems to be the compressed size, not the uncompressed size, so it should not be used in this `CopyTo`.



-- 
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@avro.apache.org

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


[GitHub] [avro] KalleOlaviNiemitalo commented on a diff in pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "KalleOlaviNiemitalo (via GitHub)" <gi...@apache.org>.
KalleOlaviNiemitalo commented on code in PR #2439:
URL: https://github.com/apache/avro/pull/2439#discussion_r1293381830


##########
lang/csharp/src/apache/test/File/FileTests.cs:
##########
@@ -668,6 +668,55 @@ public void TestPartialReadAll([Values(specificSchema)] string schemaStr, [Value
             }
         }
 
+        [Test]
+        public void TestDeflateReadMemoryUsage([Values(specificSchema)] string schemaStr)
+        {
+            // create and write out
+            IList<Foo> records = MakeRecords(GetTestFooObject());
+
+            Process currentProcess = Process.GetCurrentProcess();
+
+            MemoryStream dataFileOutputStream = new MemoryStream();
+
+            Schema schema = Schema.Parse(schemaStr);
+            DatumWriter<Foo> writer = new SpecificWriter<Foo>(schema);
+            using (IFileWriter<Foo> dataFileWriter = DataFileWriter<Foo>.OpenWriter(writer, dataFileOutputStream, Codec.CreateCodec(Codec.Type.Deflate)))
+            {
+                for (int i = 0; i < 10; ++i)
+                {
+                    foreach (Foo foo in records)
+                    {
+                        dataFileWriter.Append(foo);
+                    }
+
+                    // write out block
+                    if (i == 1 || i == 4)
+                    {
+                        dataFileWriter.Sync();
+                    }
+                }
+            }
+
+            long startMemoryUsedBytes = currentProcess.WorkingSet64;
+
+            MemoryStream dataFileInputStream = new MemoryStream(dataFileOutputStream.ToArray());
+            dataFileInputStream.Position = 0;
+
+            // read back
+            IList<Foo> readRecords = new List<Foo>();
+            using (IFileReader<Foo> reader = DataFileReader<Foo>.OpenReader(dataFileInputStream, schema))
+            {
+                // read records from synced position
+                foreach (Foo rec in reader.NextEntries)
+                    readRecords.Add(rec);
+            }
+
+            long totalMemoryUsedBytes = currentProcess.WorkingSet64 - startMemoryUsedBytes;
+
+            Assert.IsTrue(totalMemoryUsedBytes  == 0, "Total memory usage in working set");

Review Comment:
   This test looks unreliable:
   
   * WorkingSet64 is the amount of physical memory reserved for the process.  If the process allocates some memory and doesn't use it afterwards, and the operating system pages it out, then it won't be included in WorkingSet64.
   * Process.WorkingSet64 updates only when you call Process.Refresh().
   * Can other tests be run in parallel with this one?  If they can, the memory allocated by them could cause this test to fail.
   * The test code between the `currentProcess.WorkingSet64` evaluations allocates new objects whose memory might not be garbage-collected and released to the OS soon enough.  The runtime can also allocate memory for JIT-compiled methods.
   
   The test might become more reliable if it were changed to:
   
   * Run the decompression a few times before the measurement, to get to a stable state.
   * Measure AppDomain.MonitoringTotalAllocatedMemorySize or GC.GetAllocatedBytesForCurrentThread(), rather than Process.WorkingSet64.
   * Allow some amount of allocations but not too much.  This might require a larger data file, in order to distinguish the important allocations from noise.



-- 
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@avro.apache.org

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


[GitHub] [avro] KalleOlaviNiemitalo commented on pull request #2439: AVRO-3802: [Csharp] Fix memory leak on deflate codec decompression

Posted by "KalleOlaviNiemitalo (via GitHub)" <gi...@apache.org>.
KalleOlaviNiemitalo commented on PR #2439:
URL: https://github.com/apache/avro/pull/2439#issuecomment-1677409788

   It might be possible to reduce memory allocations further, by changing the Codec API so that the caller of Codec.Decompress has to create the MemoryStream to which Codec.Decompress writes the data.  This would:
   
   - Allow the same MemoryStream and its buffer to be used for multiple data blocks.
   - Avoid copying the data from the MemoryStream to an exact-size `byte[]` and from there to another MemoryStream.
   
   Alternatively, with a different API change, it might be possible to make BinaryDecoder read directly from the DeflateStream, avoiding the output MemoryStream altogether.  However, that change might hurt performance if BinaryDecoder.Read does more work per call than MemoryStream.Read.


-- 
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@avro.apache.org

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