You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2020/06/24 11:41:29 UTC

[GitHub] [cassandra] jasonstack opened a new pull request #651: Fix entire sstable streaming leak and reduce buffer allocation

jasonstack opened a new pull request #651:
URL: https://github.com/apache/cassandra/pull/651


   


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #651: Fix entire sstable streaming leak and reduce buffer allocation

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #651:
URL: https://github.com/apache/cassandra/pull/651#discussion_r445170145



##########
File path: src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
##########
@@ -170,36 +170,50 @@ public int writeToChannel(Write write, StreamRateLimiter limiter) throws IOExcep
      */
     public long writeFileToChannel(FileChannel file, StreamRateLimiter limiter) throws IOException
     {
-        // write files in 1MiB chunks, since there may be blocking work performed to fetch it from disk,
-        // the data is never brought in process and is gated by the wire anyway
         if (channel.pipeline().get(SslHandler.class) != null)
-            return writeFileToChannel(file, limiter, 1 << 20, 1 << 20, 2 << 20);
+            // each batch is loaded into ByteBuffer, 64kb is more BufferPool friendly.
+            return writeFileToChannel(file, limiter, 1 << 16);
         else
+            // write files in 1MiB chunks, since there may be blocking work performed to fetch it from disk,
+            // the data is never brought in process and is gated by the wire anyway
             return writeFileToChannelZeroCopy(file, limiter, 1 << 20, 1 << 20, 2 << 20);
     }
 
-    public long writeFileToChannel(FileChannel fc, StreamRateLimiter limiter, int batchSize, int lowWaterMark, int highWaterMark) throws IOException
+    public long writeFileToChannel(FileChannel fc, StreamRateLimiter limiter, int batchSize) throws IOException
     {
         final long length = fc.size();
         long bytesTransferred = 0;
-        while (bytesTransferred < length)
+
+        try

Review comment:
       @jasonstack AFAICT here, the file channel should just always be closed when this method exits, unlike `writeFileToChannelZeroCopy` which doesn't buffer the contents of the file. In other words, if we just made this `try (fc)`, then removed the `catch` block and the `close()` inside our `Write`  implementation, wouldn't things behave exactly the same (and not have to worry about the position and length math to figure out whether we were on the last 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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #651: Fix entire sstable streaming leak and reduce buffer allocation

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #651:
URL: https://github.com/apache/cassandra/pull/651#discussion_r445731630



##########
File path: src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
##########
@@ -218,6 +220,7 @@ public long writeFileToChannel(FileChannel fc, StreamRateLimiter limiter, int ba
         return bytesTransferred;
     }
 
+    @VisibleForTesting

Review comment:
       +1




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #651: Fix entire sstable streaming leak and reduce buffer allocation

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #651:
URL: https://github.com/apache/cassandra/pull/651#discussion_r445673371



##########
File path: src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
##########
@@ -170,36 +170,50 @@ public int writeToChannel(Write write, StreamRateLimiter limiter) throws IOExcep
      */
     public long writeFileToChannel(FileChannel file, StreamRateLimiter limiter) throws IOException
     {
-        // write files in 1MiB chunks, since there may be blocking work performed to fetch it from disk,
-        // the data is never brought in process and is gated by the wire anyway
         if (channel.pipeline().get(SslHandler.class) != null)
-            return writeFileToChannel(file, limiter, 1 << 20, 1 << 20, 2 << 20);
+            // each batch is loaded into ByteBuffer, 64kb is more BufferPool friendly.
+            return writeFileToChannel(file, limiter, 1 << 16);

Review comment:
       Fair enough. I wonder if we could just make it `defaultHighWaterMark` then? Just want there to be some traceability for our decision expressed as code :)
   
   Either way 64kb seems fine.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #651: Fix entire sstable streaming leak and reduce buffer allocation

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #651:
URL: https://github.com/apache/cassandra/pull/651#discussion_r445735388



##########
File path: test/unit/org/apache/cassandra/net/AsyncStreamingOutputPlusTest.java
##########
@@ -108,7 +117,70 @@ public void testSuccess() throws IOException
             assertEquals(1, read.getLong(0));
             assertEquals(2, read.getLong(8));
         }
+    }
+
+    @Test
+    public void testWriteFileToChannelZeroCopy() throws IOException
+    {
+        testWriteFileToChannel(true);
+    }
+
+    @Test
+    public void testWriteFileToChannelSSL() throws IOException
+    {
+        testWriteFileToChannel(false);
+    }
+
+    private void testWriteFileToChannel(boolean zeroCopy) throws IOException
+    {
+        File file = populateTempData("zero_copy_" + zeroCopy);
+        int length = (int) file.length();
+
+        EmbeddedChannel channel = new TestChannel(4);
+        StreamManager.StreamRateLimiter limiter = new StreamManager.StreamRateLimiter(FBUtilities.getBroadcastAddressAndPort());
+
+        try (RandomAccessFile raf = new RandomAccessFile(file.getPath(), "r");
+             FileChannel fileChannel = raf.getChannel();
+             AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel))
+        {
+            assertFalse(isClosed(fileChannel));
+
+            if (zeroCopy)
+                out.writeFileToChannelZeroCopy(fileChannel, limiter, length, length, length * 2);
+            else
+                out.writeFileToChannel(fileChannel, limiter, length);
+
+            assertEquals(length, out.flushed());
+            assertEquals(length, out.flushedToNetwork());
+            assertEquals(length, out.position());
+
+            assertTrue(isClosed(fileChannel));
+        }
+    }
+
+    private File populateTempData(String name) throws IOException
+    {
+        File file = Files.createTempFile(name, ".txt").toFile();
+        file.deleteOnExit();
 
+        Random r = new Random();
+        byte [] content = new byte[16];
+        r.nextBytes(content);
+        Files.write(file.toPath(), content);
+
+        return file;
     }
 
+    private static boolean isClosed(FileChannel channel)

Review comment:
       @jasonstack Could we remove this method if we just use `FileChannel#isOpen()` in the test?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #651: Fix entire sstable streaming leak and reduce buffer allocation

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #651:
URL: https://github.com/apache/cassandra/pull/651#discussion_r445673371



##########
File path: src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
##########
@@ -170,36 +170,50 @@ public int writeToChannel(Write write, StreamRateLimiter limiter) throws IOExcep
      */
     public long writeFileToChannel(FileChannel file, StreamRateLimiter limiter) throws IOException
     {
-        // write files in 1MiB chunks, since there may be blocking work performed to fetch it from disk,
-        // the data is never brought in process and is gated by the wire anyway
         if (channel.pipeline().get(SslHandler.class) != null)
-            return writeFileToChannel(file, limiter, 1 << 20, 1 << 20, 2 << 20);
+            // each batch is loaded into ByteBuffer, 64kb is more BufferPool friendly.
+            return writeFileToChannel(file, limiter, 1 << 16);

Review comment:
       Fair enough. I wonder if we could just make it `defaultHighWaterMark` then? Just want there to be some traceability for our decision expressed as code :)




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #651: Fix entire sstable streaming leak and reduce buffer allocation

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #651:
URL: https://github.com/apache/cassandra/pull/651#discussion_r445149854



##########
File path: src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
##########
@@ -170,36 +170,50 @@ public int writeToChannel(Write write, StreamRateLimiter limiter) throws IOExcep
      */
     public long writeFileToChannel(FileChannel file, StreamRateLimiter limiter) throws IOException
     {
-        // write files in 1MiB chunks, since there may be blocking work performed to fetch it from disk,
-        // the data is never brought in process and is gated by the wire anyway
         if (channel.pipeline().get(SslHandler.class) != null)
-            return writeFileToChannel(file, limiter, 1 << 20, 1 << 20, 2 << 20);
+            // each batch is loaded into ByteBuffer, 64kb is more BufferPool friendly.
+            return writeFileToChannel(file, limiter, 1 << 16);

Review comment:
       @jasonstack Would it be less brittle here if we just made the batch size `BufferPool.NORMAL_CHUNK_SIZE`? That seems like it would still avoid passing the threshold where we have to allocate new buffers, and would adjust automatically if we ever had to change the normal chunk size. The thing I don't know is what the optimal buffer size actually is. If we want to stay as close as possible to the existing behavior, using `BufferPool.NORMAL_CHUNK_SIZE` obviously makes sense.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #651: Fix entire sstable streaming leak and reduce buffer allocation

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #651:
URL: https://github.com/apache/cassandra/pull/651#discussion_r445752959



##########
File path: test/unit/org/apache/cassandra/net/AsyncStreamingOutputPlusTest.java
##########
@@ -108,7 +117,70 @@ public void testSuccess() throws IOException
             assertEquals(1, read.getLong(0));
             assertEquals(2, read.getLong(8));
         }
+    }
+
+    @Test
+    public void testWriteFileToChannelZeroCopy() throws IOException
+    {
+        testWriteFileToChannel(true);
+    }
+
+    @Test
+    public void testWriteFileToChannelSSL() throws IOException
+    {
+        testWriteFileToChannel(false);
+    }
+
+    private void testWriteFileToChannel(boolean zeroCopy) throws IOException
+    {
+        File file = populateTempData("zero_copy_" + zeroCopy);
+        int length = (int) file.length();
+
+        EmbeddedChannel channel = new TestChannel(4);
+        StreamManager.StreamRateLimiter limiter = new StreamManager.StreamRateLimiter(FBUtilities.getBroadcastAddressAndPort());
+
+        try (RandomAccessFile raf = new RandomAccessFile(file.getPath(), "r");
+             FileChannel fileChannel = raf.getChannel();
+             AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel))
+        {
+            assertFalse(isClosed(fileChannel));
+
+            if (zeroCopy)
+                out.writeFileToChannelZeroCopy(fileChannel, limiter, length, length, length * 2);
+            else
+                out.writeFileToChannel(fileChannel, limiter, length);
+
+            assertEquals(length, out.flushed());
+            assertEquals(length, out.flushedToNetwork());
+            assertEquals(length, out.position());
+
+            assertTrue(isClosed(fileChannel));
+        }
+    }
+
+    private File populateTempData(String name) throws IOException
+    {
+        File file = Files.createTempFile(name, ".txt").toFile();
+        file.deleteOnExit();
 
+        Random r = new Random();
+        byte [] content = new byte[16];
+        r.nextBytes(content);
+        Files.write(file.toPath(), content);
+
+        return file;
     }
 
+    private static boolean isClosed(FileChannel channel)

Review comment:
       +1




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #651: Fix entire sstable streaming leak and reduce buffer allocation

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #651:
URL: https://github.com/apache/cassandra/pull/651#discussion_r445730975



##########
File path: src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
##########
@@ -218,6 +220,7 @@ public long writeFileToChannel(FileChannel fc, StreamRateLimiter limiter, int ba
         return bytesTransferred;
     }
 
+    @VisibleForTesting

Review comment:
       nit: alternatively could make them package-private




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #651: Fix entire sstable streaming leak and reduce buffer allocation

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #651:
URL: https://github.com/apache/cassandra/pull/651#discussion_r445381758



##########
File path: src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
##########
@@ -170,36 +170,50 @@ public int writeToChannel(Write write, StreamRateLimiter limiter) throws IOExcep
      */
     public long writeFileToChannel(FileChannel file, StreamRateLimiter limiter) throws IOException
     {
-        // write files in 1MiB chunks, since there may be blocking work performed to fetch it from disk,
-        // the data is never brought in process and is gated by the wire anyway
         if (channel.pipeline().get(SslHandler.class) != null)
-            return writeFileToChannel(file, limiter, 1 << 20, 1 << 20, 2 << 20);
+            // each batch is loaded into ByteBuffer, 64kb is more BufferPool friendly.
+            return writeFileToChannel(file, limiter, 1 << 16);
         else
+            // write files in 1MiB chunks, since there may be blocking work performed to fetch it from disk,
+            // the data is never brought in process and is gated by the wire anyway
             return writeFileToChannelZeroCopy(file, limiter, 1 << 20, 1 << 20, 2 << 20);
     }
 
-    public long writeFileToChannel(FileChannel fc, StreamRateLimiter limiter, int batchSize, int lowWaterMark, int highWaterMark) throws IOException
+    public long writeFileToChannel(FileChannel fc, StreamRateLimiter limiter, int batchSize) throws IOException
     {
         final long length = fc.size();
         long bytesTransferred = 0;
-        while (bytesTransferred < length)
+
+        try

Review comment:
       you are right.. we can move it into `finally` block




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #651: Fix entire sstable streaming leak and reduce buffer allocation

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #651:
URL: https://github.com/apache/cassandra/pull/651#discussion_r445735388



##########
File path: test/unit/org/apache/cassandra/net/AsyncStreamingOutputPlusTest.java
##########
@@ -108,7 +117,70 @@ public void testSuccess() throws IOException
             assertEquals(1, read.getLong(0));
             assertEquals(2, read.getLong(8));
         }
+    }
+
+    @Test
+    public void testWriteFileToChannelZeroCopy() throws IOException
+    {
+        testWriteFileToChannel(true);
+    }
+
+    @Test
+    public void testWriteFileToChannelSSL() throws IOException
+    {
+        testWriteFileToChannel(false);
+    }
+
+    private void testWriteFileToChannel(boolean zeroCopy) throws IOException
+    {
+        File file = populateTempData("zero_copy_" + zeroCopy);
+        int length = (int) file.length();
+
+        EmbeddedChannel channel = new TestChannel(4);
+        StreamManager.StreamRateLimiter limiter = new StreamManager.StreamRateLimiter(FBUtilities.getBroadcastAddressAndPort());
+
+        try (RandomAccessFile raf = new RandomAccessFile(file.getPath(), "r");
+             FileChannel fileChannel = raf.getChannel();
+             AsyncStreamingOutputPlus out = new AsyncStreamingOutputPlus(channel))
+        {
+            assertFalse(isClosed(fileChannel));
+
+            if (zeroCopy)
+                out.writeFileToChannelZeroCopy(fileChannel, limiter, length, length, length * 2);
+            else
+                out.writeFileToChannel(fileChannel, limiter, length);
+
+            assertEquals(length, out.flushed());
+            assertEquals(length, out.flushedToNetwork());
+            assertEquals(length, out.position());
+
+            assertTrue(isClosed(fileChannel));
+        }
+    }
+
+    private File populateTempData(String name) throws IOException
+    {
+        File file = Files.createTempFile(name, ".txt").toFile();
+        file.deleteOnExit();
 
+        Random r = new Random();
+        byte [] content = new byte[16];
+        r.nextBytes(content);
+        Files.write(file.toPath(), content);
+
+        return file;
     }
 
+    private static boolean isClosed(FileChannel channel)

Review comment:
       Could we remove this method if we just use `FileChannel#isOpen()` in the test?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #651: Fix entire sstable streaming leak and reduce buffer allocation

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #651:
URL: https://github.com/apache/cassandra/pull/651#discussion_r445380479



##########
File path: src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
##########
@@ -170,36 +170,50 @@ public int writeToChannel(Write write, StreamRateLimiter limiter) throws IOExcep
      */
     public long writeFileToChannel(FileChannel file, StreamRateLimiter limiter) throws IOException
     {
-        // write files in 1MiB chunks, since there may be blocking work performed to fetch it from disk,
-        // the data is never brought in process and is gated by the wire anyway
         if (channel.pipeline().get(SslHandler.class) != null)
-            return writeFileToChannel(file, limiter, 1 << 20, 1 << 20, 2 << 20);
+            // each batch is loaded into ByteBuffer, 64kb is more BufferPool friendly.
+            return writeFileToChannel(file, limiter, 1 << 16);

Review comment:
       Using 128kb (NORMAL_CHUNK_SIZE) won't cause out-of-pool allocation.
   
   But current default low/high water mark is 32kb/64kb, I think a lower batch size will allow the write thread to wake up earlier and perform the blocking disk read earlier, assuming netty worker thread is flushing at a constant rate..
   
   > The thing I don't know is what the optimal buffer size actually is.
   
   My gut feeling is somewhere between low water mark and high water mark.. Legacy stream writer uses 64kb size. Zero-copy has higher batch-size and low/high water mark because buffer won't be copied to user space..




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jasonstack commented on a change in pull request #651: Fix entire sstable streaming leak and reduce buffer allocation

Posted by GitBox <gi...@apache.org>.
jasonstack commented on a change in pull request #651:
URL: https://github.com/apache/cassandra/pull/651#discussion_r445386515



##########
File path: src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
##########
@@ -170,36 +170,50 @@ public int writeToChannel(Write write, StreamRateLimiter limiter) throws IOExcep
      */

Review comment:
       I will update the javadoc. `link` can point to the class but not method param..




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a change in pull request #651: Fix entire sstable streaming leak and reduce buffer allocation

Posted by GitBox <gi...@apache.org>.
maedhroz commented on a change in pull request #651:
URL: https://github.com/apache/cassandra/pull/651#discussion_r445179566



##########
File path: src/java/org/apache/cassandra/net/AsyncStreamingOutputPlus.java
##########
@@ -170,36 +170,50 @@ public int writeToChannel(Write write, StreamRateLimiter limiter) throws IOExcep
      */

Review comment:
       nit: If we fix up the JavaDoc at all, I would use `link` instead of `code`, given it makes in IDE navigation easier.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dineshjoshi closed pull request #651: Fix entire sstable streaming leak and reduce buffer allocation

Posted by GitBox <gi...@apache.org>.
dineshjoshi closed pull request #651:
URL: https://github.com/apache/cassandra/pull/651


   


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org