hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From e..@apache.org
Subject svn commit: r1124406 - in /hadoop/common/trunk: CHANGES.txt src/java/org/apache/hadoop/fs/FSDataInputStream.java src/test/core/org/apache/hadoop/fs/FSMainOperationsBaseTest.java
Date Wed, 18 May 2011 20:17:17 GMT
Author: eli
Date: Wed May 18 20:17:16 2011
New Revision: 1124406

URL: http://svn.apache.org/viewvc?rev=1124406&view=rev
Log:
HADOOP-7301. FSDataInputStream should expose a getWrappedStream method. Contributed by Jonathan
Hsieh

Modified:
    hadoop/common/trunk/CHANGES.txt
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java
    hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/FSMainOperationsBaseTest.java

Modified: hadoop/common/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/CHANGES.txt?rev=1124406&r1=1124405&r2=1124406&view=diff
==============================================================================
--- hadoop/common/trunk/CHANGES.txt (original)
+++ hadoop/common/trunk/CHANGES.txt Wed May 18 20:17:16 2011
@@ -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

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java?rev=1124406&r1=1124405&r2=1124406&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java Wed May 18 20:17:16
2011
@@ -64,4 +64,9 @@ public class FSDataInputStream extends D
   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;
+  }
 }

Modified: hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/FSMainOperationsBaseTest.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/FSMainOperationsBaseTest.java?rev=1124406&r1=1124405&r2=1124406&view=diff
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/FSMainOperationsBaseTest.java (original)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/FSMainOperationsBaseTest.java Wed
May 18 20:17:16 2011
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InputStream;
 
 
 
@@ -1044,6 +1045,16 @@ public abstract class FSMainOperationsBa
   }
 
   
+  @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);
   }



Mime
View raw message