Forráskód Böngészése

HADOOP-7754. Expose file descriptors from Hadoop-wrapped local FileSystems (todd and ahmed via tucu)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1373236 13f79535-47bb-0310-9956-ffa450edef68
Alejandro Abdelnur 12 éve
szülő
commit
b3121a4944

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -24,6 +24,9 @@ Release 2.0.1-alpha - UNRELEASED
 
     HADOOP-8581. add support for HTTPS to the web UIs. (tucu)
 
+    HADOOP-7754. Expose file descriptors from Hadoop-wrapped local 
+    FileSystems (todd and ahmed via tucu)
+
   IMPROVEMENTS
 
     HADOOP-8340. SNAPSHOT build versions should compare as less than their eventual

+ 12 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java

@@ -18,6 +18,8 @@
 package org.apache.hadoop.fs;
 
 import java.io.BufferedInputStream;
+import java.io.FileDescriptor;
+import java.io.FileInputStream;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -31,7 +33,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class BufferedFSInputStream extends BufferedInputStream
-implements Seekable, PositionedReadable {
+implements Seekable, PositionedReadable, HasFileDescriptor {
   /**
    * Creates a <code>BufferedFSInputStream</code>
    * with the specified buffer size,
@@ -97,4 +99,13 @@ implements Seekable, PositionedReadable {
   public void readFully(long position, byte[] buffer) throws IOException {
     ((FSInputStream)in).readFully(position, buffer);
   }
+
+  @Override
+  public FileDescriptor getFileDescriptor() throws IOException {
+    if (in instanceof HasFileDescriptor) {
+      return ((HasFileDescriptor) in).getFileDescriptor();
+    } else {
+      return null;
+    }
+  }
 }

+ 12 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java

@@ -28,7 +28,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceAudience.Public
 @InterfaceStability.Stable
 public class FSDataInputStream extends DataInputStream
-    implements Seekable, PositionedReadable, Closeable, ByteBufferReadable {
+    implements Seekable, PositionedReadable, Closeable, ByteBufferReadable, HasFileDescriptor {
 
   public FSDataInputStream(InputStream in)
     throws IOException {
@@ -125,4 +125,15 @@ public class FSDataInputStream extends DataInputStream
 
     throw new UnsupportedOperationException("Byte-buffer read unsupported by input stream");
   }
+
+  @Override
+  public FileDescriptor getFileDescriptor() throws IOException {
+    if (in instanceof HasFileDescriptor) {
+      return ((HasFileDescriptor) in).getFileDescriptor();
+    } else if (in instanceof FileInputStream) {
+      return ((FileInputStream) in).getFD();
+    } else {
+      return null;
+    }
+  }
 }

+ 40 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HasFileDescriptor.java

@@ -0,0 +1,40 @@
+/**
+ * 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.hadoop.fs;
+
+import java.io.FileDescriptor;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Having a FileDescriptor
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface HasFileDescriptor {
+
+  /**
+   * @return the FileDescriptor
+   * @throws IOException
+   */
+  public FileDescriptor getFileDescriptor() throws IOException;
+
+}

+ 7 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -26,6 +26,7 @@ import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
+import java.io.FileDescriptor;
 import java.net.URI;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
@@ -111,7 +112,7 @@ public class RawLocalFileSystem extends FileSystem {
   /*******************************************************
    * For open()'s FSInputStream.
    *******************************************************/
-  class LocalFSFileInputStream extends FSInputStream {
+  class LocalFSFileInputStream extends FSInputStream implements HasFileDescriptor {
     private FileInputStream fis;
     private long position;
 
@@ -181,6 +182,11 @@ public class RawLocalFileSystem extends FileSystem {
       }
       return value;
     }
+
+    @Override
+    public FileDescriptor getFileDescriptor() throws IOException {
+      return fis.getFD();
+    }
   }
   
   public FSDataInputStream open(Path f, int bufferSize) throws IOException {

+ 10 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java

@@ -248,4 +248,14 @@ public class TestLocalFileSystem {
     }
     assertEquals(1, fileSchemeCount);
   }
+
+  public void testHasFileDescriptor() throws IOException {
+    Configuration conf = new Configuration();
+    LocalFileSystem fs = FileSystem.getLocal(conf);
+    Path path = new Path(TEST_ROOT_DIR, "test-file");
+    writeFile(fs, path, 1);
+    BufferedFSInputStream bis = new BufferedFSInputStream(
+        new RawLocalFileSystem().new LocalFSFileInputStream(path), 1024);
+    assertNotNull(bis.getFileDescriptor());
+  }
 }