forked from apache/spark
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-29971][CORE] Fix buffer leaks in `TransportFrameDecoder/Transp…
…ortCipher` - Correctly release `ByteBuf` in `TransportCipher` in all cases - Move closing / releasing logic to `handlerRemoved(...)` so we are guaranteed that is always called. We need to carefully manage the ownership / lifecycle of `ByteBuf` instances so we don't leak any of these. We did not correctly do this in all cases: - when end up in invalid cipher state. - when partial data was received and the channel is closed before the full frame is decoded Fixes netty/netty#9784. No. Pass the newly added UTs. Closes apache#26609 from normanmaurer/leaks_2_4. Authored-by: Norman Maurer <norman_maurer@apple.com>
- Loading branch information
1 parent
6880ccd
commit e8e6e60
Showing
4 changed files
with
148 additions
and
35 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
91 changes: 91 additions & 0 deletions
91
...on/network-common/src/test/java/org/apache/spark/network/crypto/TransportCipherSuite.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,91 @@ | ||
/* | ||
* 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.crypto; | ||
|
||
import javax.crypto.spec.SecretKeySpec; | ||
import java.io.IOException; | ||
import java.nio.channels.ReadableByteChannel; | ||
import java.nio.channels.WritableByteChannel; | ||
|
||
import io.netty.buffer.ByteBuf; | ||
import io.netty.buffer.Unpooled; | ||
import io.netty.channel.embedded.EmbeddedChannel; | ||
import org.apache.commons.crypto.stream.CryptoInputStream; | ||
import org.apache.commons.crypto.stream.CryptoOutputStream; | ||
import org.apache.spark.network.util.MapConfigProvider; | ||
import org.apache.spark.network.util.TransportConf; | ||
import org.hamcrest.CoreMatchers; | ||
import org.junit.Test; | ||
|
||
import static org.junit.Assert.assertEquals; | ||
import static org.junit.Assert.assertFalse; | ||
import static org.junit.Assert.assertThat; | ||
import static org.junit.Assert.fail; | ||
import static org.mockito.Mockito.any; | ||
import static org.mockito.Mockito.anyInt; | ||
import static org.mockito.Mockito.mock; | ||
import static org.mockito.Mockito.when; | ||
|
||
public class TransportCipherSuite { | ||
|
||
@Test | ||
public void testBufferNotLeaksOnInternalError() throws IOException { | ||
String algorithm = "TestAlgorithm"; | ||
TransportConf conf = new TransportConf("Test", MapConfigProvider.EMPTY); | ||
TransportCipher cipher = new TransportCipher(conf.cryptoConf(), conf.cipherTransformation(), | ||
new SecretKeySpec(new byte[256], algorithm), new byte[0], new byte[0]) { | ||
|
||
@Override | ||
CryptoOutputStream createOutputStream(WritableByteChannel ch) { | ||
return null; | ||
} | ||
|
||
@Override | ||
CryptoInputStream createInputStream(ReadableByteChannel ch) throws IOException { | ||
CryptoInputStream mockInputStream = mock(CryptoInputStream.class); | ||
when(mockInputStream.read(any(byte[].class), anyInt(), anyInt())) | ||
.thenThrow(new InternalError()); | ||
return mockInputStream; | ||
} | ||
}; | ||
|
||
EmbeddedChannel channel = new EmbeddedChannel(); | ||
cipher.addToChannel(channel); | ||
|
||
ByteBuf buffer = Unpooled.wrappedBuffer(new byte[] { 1, 2 }); | ||
ByteBuf buffer2 = Unpooled.wrappedBuffer(new byte[] { 1, 2 }); | ||
|
||
try { | ||
channel.writeInbound(buffer); | ||
fail("Should have raised InternalError"); | ||
} catch (InternalError expected) { | ||
// expected | ||
assertEquals(0, buffer.refCnt()); | ||
} | ||
|
||
try { | ||
channel.writeInbound(buffer2); | ||
fail("Should have raised an exception"); | ||
} catch (Throwable expected) { | ||
assertThat(expected, CoreMatchers.instanceOf(IOException.class)); | ||
assertEquals(0, buffer2.refCnt()); | ||
} | ||
|
||
// Simulate closing the connection | ||
assertFalse(channel.finish()); | ||
} | ||
} |