From 14fd85cdc57686fb649730bbd4e98abd0f76e2bd Mon Sep 17 00:00:00 2001 From: Eli Collins Date: Wed, 18 May 2011 20:17:16 +0000 Subject: [PATCH] HADOOP-7301. FSDataInputStream should expose a getWrappedStream method. Contributed by Jonathan Hsieh git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1124406 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 +++ src/java/org/apache/hadoop/fs/FSDataInputStream.java | 5 +++++ .../apache/hadoop/fs/FSMainOperationsBaseTest.java | 11 +++++++++++ 3 files changed, 19 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 1615e25c0f..da887fd27a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -161,6 +161,9 @@ Trunk (unreleased changes) HADOOP-7286. Refactor the du/dus/df commands to conform to new FsCommand class. (Daryn Sharp via todd) + HADOOP-7301. FSDataInputStream should expose a getWrappedStream method. + (Jonathan Hsieh via eli) + OPTIMIZATIONS BUG FIXES diff --git a/src/java/org/apache/hadoop/fs/FSDataInputStream.java b/src/java/org/apache/hadoop/fs/FSDataInputStream.java index cb4dd61470..b594f4f5ea 100644 --- a/src/java/org/apache/hadoop/fs/FSDataInputStream.java +++ b/src/java/org/apache/hadoop/fs/FSDataInputStream.java @@ -64,4 +64,9 @@ public void readFully(long position, byte[] buffer) public boolean seekToNewSource(long targetPos) throws IOException { return ((Seekable)in).seekToNewSource(targetPos); } + + // Returns the underlying input stream. This is used by unit tests. + public InputStream getWrappedStream() { + return in; + } } diff --git a/src/test/core/org/apache/hadoop/fs/FSMainOperationsBaseTest.java b/src/test/core/org/apache/hadoop/fs/FSMainOperationsBaseTest.java index 147bc97366..caff77d933 100644 --- a/src/test/core/org/apache/hadoop/fs/FSMainOperationsBaseTest.java +++ b/src/test/core/org/apache/hadoop/fs/FSMainOperationsBaseTest.java @@ -20,6 +20,7 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.io.InputStream; @@ -1044,6 +1045,16 @@ public void testOutputStreamClosedTwice() throws IOException { } + @Test + public void testGetWrappedInputStream() throws IOException { + Path src = getTestRootPath(fSys, "test/hadoop/file"); + createFile(src); + FSDataInputStream in = fSys.open(src); + InputStream is = in.getWrappedStream(); + in.close(); + Assert.assertNotNull(is); + } + protected void createFile(Path path) throws IOException { FileSystemTestHelper.createFile(fSys, path); }