SQOOP-3243: Importing BLOB data causes 'Stream closed' error on encrypted HDFS

(Boglarka Egyed)
diff --git a/src/java/org/apache/sqoop/io/LobFile.java b/src/java/org/apache/sqoop/io/LobFile.java
index 4e1bf2c..4edde12 100644
--- a/src/java/org/apache/sqoop/io/LobFile.java
+++ b/src/java/org/apache/sqoop/io/LobFile.java
@@ -1665,10 +1665,7 @@
         this.out = null;
       }
 
-      if (this.countingOut != null) {
-        this.countingOut.close();
-        this.countingOut = null;
-      }
+      this.countingOut = null;
     }
 
     @Override
diff --git a/src/test/com/cloudera/sqoop/io/TestLobFile.java b/src/test/com/cloudera/sqoop/io/TestLobFile.java
index 029758c..ac461fd 100644
--- a/src/test/com/cloudera/sqoop/io/TestLobFile.java
+++ b/src/test/com/cloudera/sqoop/io/TestLobFile.java
@@ -25,11 +25,15 @@
 import java.io.OutputStream;
 import java.io.Reader;
 import java.io.Writer;
+import java.net.URI;
 import java.nio.CharBuffer;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CryptoOutputStream;
+import org.apache.hadoop.crypto.JceAesCtrCryptoCodec;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.Before;
@@ -39,6 +43,12 @@
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Answers.CALLS_REAL_METHODS;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.withSettings;
 
 import org.junit.Rule;
 import org.junit.rules.ExpectedException;
@@ -595,5 +605,38 @@
     runCompressedTest(CodecMap.LZO);
   }
 
+  @Test
+  public void testCryptoOutputStreamClosingDoesNotThrowExceptionAndClosedProperly() throws Exception {
+    // Tests that closing CryptoOutputStream doesn't throw exception neither with Java 7 nor with Java 8
+    // For a detailed explanation see SQOOP-3243
+    CryptoOutputStream cryptoOutputStream = createCryptoOutputStream();
+    FSDataOutputStream wrappedCryptoOutputStream = new FSDataOutputStream(cryptoOutputStream, null);
+
+    Path mockPath = spy(new Path("file://" + TEMP_BASE_DIR, "binary.lob"));
+    FileSystem mockFileSystem = mock(FileSystem.class, withSettings().defaultAnswer(CALLS_REAL_METHODS.get()));
+
+    doReturn(mockFileSystem).when(mockPath).getFileSystem(conf);
+    doReturn(null).when(mockFileSystem).getWorkingDirectory();
+    doReturn(wrappedCryptoOutputStream).when(mockFileSystem).create(mockPath);
+    doReturn(new URI("file:///")).when(mockFileSystem).getUri();
+
+    LobFile.Writer writer = LobFile.create(mockPath, conf);
+
+    writer.close();
+
+    verify(cryptoOutputStream).close();
+  }
+
+
+  public CryptoOutputStream createCryptoOutputStream() throws Exception {
+    final byte[] BYTES = {1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16};
+    Path p = new Path(TEMP_BASE_DIR, "binary.lob");
+
+    FSDataOutputStream fsDataOutputStream = fs.create(p);
+    CryptoOutputStream cryptoOutputStream = spy(new CryptoOutputStream(fsDataOutputStream, new JceAesCtrCryptoCodec(), 512, BYTES, BYTES));
+
+    return cryptoOutputStream;
+  }
+
 }