From 752f6d821358346c13c048a911542dc3443c782a Mon Sep 17 00:00:00 2001 From: gardenia Date: Tue, 7 Feb 2023 12:01:57 +0000 Subject: [PATCH] HADOOP-18621. Resource leak in CryptoOutputStream.close() (#5347) When closing we need to wrap the flush() in a try .. finally, otherwise when flush throws it will stop completion of the remainder of the close activities and in particular the close of the underlying wrapped stream object resulting in a resource leak. Contributed by Colm Dougan --- .../hadoop/crypto/CryptoOutputStream.java | 13 +++++++----- .../crypto/TestCryptoOutputStreamClosing.java | 20 +++++++++++++++++++ 2 files changed, 28 insertions(+), 5 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java index fb5ee21190..df36bd6fe6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java @@ -242,12 +242,15 @@ public synchronized void close() throws IOException { return; } try { - flush(); - if (closeOutputStream) { - super.close(); - codec.close(); + try { + flush(); + } finally { + if (closeOutputStream) { + super.close(); + codec.close(); + } + freeBuffers(); } - freeBuffers(); } finally { closed = true; } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoOutputStreamClosing.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoOutputStreamClosing.java index 39e4bb8588..04cdb962ac 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoOutputStreamClosing.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoOutputStreamClosing.java @@ -17,12 +17,14 @@ */ package org.apache.hadoop.crypto; +import java.io.IOException; import java.io.OutputStream; import org.apache.hadoop.conf.Configuration; import org.junit.BeforeClass; import org.junit.Test; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.mockito.Mockito.*; /** @@ -54,4 +56,22 @@ public void testOutputStreamNotClosing() throws Exception { verify(outputStream, never()).close(); } + @Test + public void testUnderlyingOutputStreamClosedWhenExceptionClosing() throws Exception { + OutputStream outputStream = mock(OutputStream.class); + CryptoOutputStream cos = spy(new CryptoOutputStream(outputStream, codec, + new byte[16], new byte[16], 0L, true)); + + // exception while flushing during close + doThrow(new IOException("problem flushing wrapped stream")) + .when(cos).flush(); + + intercept(IOException.class, + () -> cos.close()); + + // We expect that the close of the CryptoOutputStream closes the + // wrapped OutputStream even though we got an exception + // during CryptoOutputStream::close (in the flush method) + verify(outputStream).close(); + } }