-
Notifications
You must be signed in to change notification settings - Fork 28.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-13308] ManagedBuffers passed to OneToOneStreamManager need to be freed in non-error cases #11193
[SPARK-13308] ManagedBuffers passed to OneToOneStreamManager need to be freed in non-error cases #11193
Changes from 5 commits
ec8cc24
6134989
c9726c2
e5cf48d
2c00f29
cb99750
014ca9b
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -26,9 +26,13 @@ | |
import io.netty.channel.FileRegion; | ||
import io.netty.util.AbstractReferenceCounted; | ||
import org.junit.Test; | ||
import org.mockito.Mockito; | ||
|
||
import static org.junit.Assert.*; | ||
|
||
import org.apache.spark.network.TestManagedBuffer; | ||
import org.apache.spark.network.buffer.ManagedBuffer; | ||
import org.apache.spark.network.buffer.NettyManagedBuffer; | ||
import org.apache.spark.network.util.ByteArrayWritableChannel; | ||
|
||
public class MessageWithHeaderSuite { | ||
|
@@ -46,27 +50,51 @@ public void testShortWrite() throws Exception { | |
@Test | ||
public void testByteBufBody() throws Exception { | ||
ByteBuf header = Unpooled.copyLong(42); | ||
ByteBuf body = Unpooled.copyLong(84); | ||
MessageWithHeader msg = new MessageWithHeader(header, body, body.readableBytes()); | ||
ByteBuf bodyPassedToNettyManagedBuffer = Unpooled.copyLong(84); | ||
assertEquals(1, header.refCnt()); | ||
assertEquals(1, bodyPassedToNettyManagedBuffer.refCnt()); | ||
ManagedBuffer managedBuf = new NettyManagedBuffer(bodyPassedToNettyManagedBuffer); | ||
|
||
Object body = managedBuf.convertToNetty(); | ||
assertEquals(2, bodyPassedToNettyManagedBuffer.refCnt()); | ||
assertEquals(1, header.refCnt()); | ||
|
||
MessageWithHeader msg = new MessageWithHeader(managedBuf, header, body, managedBuf.size()); | ||
ByteBuf result = doWrite(msg, 1); | ||
assertEquals(msg.count(), result.readableBytes()); | ||
assertEquals(42, result.readLong()); | ||
assertEquals(84, result.readLong()); | ||
|
||
msg.deallocate(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: use |
||
assertEquals(0, bodyPassedToNettyManagedBuffer.refCnt()); | ||
assertEquals(0, header.refCnt()); | ||
} | ||
|
||
@Test | ||
public void testDeallocateReleasesManagedBuffer() throws Exception { | ||
ByteBuf header = Unpooled.copyLong(42); | ||
ManagedBuffer managedBuf = Mockito.spy(new TestManagedBuffer(84)); | ||
ByteBuf body = (ByteBuf) managedBuf.convertToNetty(); | ||
assertEquals(2, body.refCnt()); | ||
MessageWithHeader msg = new MessageWithHeader(managedBuf, header, body, body.readableBytes()); | ||
msg.deallocate(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: use There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done. |
||
Mockito.verify(managedBuf, Mockito.times(1)).release(); | ||
assertEquals(0, body.refCnt()); | ||
} | ||
|
||
private void testFileRegionBody(int totalWrites, int writesPerCall) throws Exception { | ||
ByteBuf header = Unpooled.copyLong(42); | ||
int headerLength = header.readableBytes(); | ||
TestFileRegion region = new TestFileRegion(totalWrites, writesPerCall); | ||
MessageWithHeader msg = new MessageWithHeader(header, region, region.count()); | ||
MessageWithHeader msg = new MessageWithHeader(null, header, region, region.count()); | ||
|
||
ByteBuf result = doWrite(msg, totalWrites / writesPerCall); | ||
assertEquals(headerLength + region.count(), result.readableBytes()); | ||
assertEquals(42, result.readLong()); | ||
for (long i = 0; i < 8; i++) { | ||
assertEquals(i, result.readLong()); | ||
} | ||
msg.deallocate(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: use |
||
} | ||
|
||
private ByteBuf doWrite(MessageWithHeader msg, int minExpectedWrites) throws Exception { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,50 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.network.server; | ||
|
||
import java.util.ArrayList; | ||
import java.util.List; | ||
|
||
import io.netty.channel.Channel; | ||
import org.junit.Test; | ||
import org.mockito.Mockito; | ||
|
||
import org.apache.spark.network.TestManagedBuffer; | ||
import org.apache.spark.network.buffer.ManagedBuffer; | ||
|
||
public class OneForOneStreamManagerSuite { | ||
|
||
@Test | ||
public void managedBuffersAreFeedWhenConnectionIsClosed() throws Exception { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The logic that this tests hasn't changed at all, but we didn't have unit tests for this case previously. |
||
OneForOneStreamManager manager = new OneForOneStreamManager(); | ||
List<ManagedBuffer> buffers = new ArrayList<>(); | ||
TestManagedBuffer buffer1 = Mockito.spy(new TestManagedBuffer(10)); | ||
TestManagedBuffer buffer2 = Mockito.spy(new TestManagedBuffer(20)); | ||
buffers.add(buffer1); | ||
buffers.add(buffer2); | ||
long streamId = manager.registerStream("appId", buffers.iterator()); | ||
|
||
Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS); | ||
manager.registerChannel(dummyChannel, streamId); | ||
|
||
manager.connectionTerminated(dummyChannel); | ||
|
||
Mockito.verify(buffer1, Mockito.times(1)).release(); | ||
Mockito.verify(buffer2, Mockito.times(1)).release(); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In order for this code to be correct / free from leaks, we need to ensure that
deallocate()
is always eventually called once this message is constructed. Instances of this class are constructed at a single call-site inMessageEncoder
. Once we construct the message, we immediately pass it to Netty code, which I assume will free it at the proper time.