소스 검색

HADOOP-6235. Adds new method to FileSystem for clients to get server defaults. Contributed by Kan Zhang.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@812597 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 16 년 전
부모
커밋
3f384b42d2
3개의 변경된 파일115개의 추가작업 그리고 0개의 파일을 삭제
  1. 3 0
      CHANGES.txt
  2. 14 0
      src/java/org/apache/hadoop/fs/FileSystem.java
  3. 98 0
      src/java/org/apache/hadoop/fs/FsServerDefaults.java

+ 3 - 0
CHANGES.txt

@@ -178,6 +178,9 @@ Trunk (unreleased changes)
     HADOOP-6105. Adds support for automatically handling deprecation of
     configuration keys. (V.V.Chaitanya Krishna via yhemanth)
     
+    HADOOP-6235. Adds new method to FileSystem for clients to get server
+    defaults. (Kan Zhang via suresh)
+
   IMPROVEMENTS
 
     HADOOP-4565. Added CombineFileInputFormat to use data locality information

+ 14 - 0
src/java/org/apache/hadoop/fs/FileSystem.java

@@ -364,6 +364,20 @@ public abstract class FileSystem extends Configured implements Closeable {
     return new BlockLocation[] { new BlockLocation(name, host, 0, file.getLen()) };
   }
   
+  /**
+   * Return a set of server default configuration values
+   * @return server default configuration values
+   * @throws IOException
+   */
+  public FsServerDefaults getServerDefaults() throws IOException {
+    Configuration conf = getConf();
+    return new FsServerDefaults(getDefaultBlockSize(), 
+        conf.getInt("io.bytes.per.checksum", 512), 
+        64 * 1024, 
+        getDefaultReplication(), 
+        conf.getInt("io.file.buffer.size", 4096));
+  }
+
   /**
    * Opens an FSDataInputStream at the indicated Path.
    * @param f the file name to open

+ 98 - 0
src/java/org/apache/hadoop/fs/FsServerDefaults.java

@@ -0,0 +1,98 @@
+/**
+ * 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.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
+
+/****************************************************
+ * Provides server default configuration values to clients.
+ * 
+ ****************************************************/
+public class FsServerDefaults implements Writable {
+
+  static { // register a ctor
+    WritableFactories.setFactory(FsServerDefaults.class, new WritableFactory() {
+      public Writable newInstance() {
+        return new FsServerDefaults();
+      }
+    });
+  }
+
+  private long blockSize;
+  private int bytesPerChecksum;
+  private int writePacketSize;
+  private short replication;
+  private int fileBufferSize;
+
+  public FsServerDefaults() {
+  }
+
+  public FsServerDefaults(long blockSize, int bytesPerChecksum,
+      int writePacketSize, short replication, int fileBufferSize) {
+    this.blockSize = blockSize;
+    this.bytesPerChecksum = bytesPerChecksum;
+    this.writePacketSize = writePacketSize;
+    this.replication = replication;
+    this.fileBufferSize = fileBufferSize;
+  }
+
+  public long getBlockSize() {
+    return blockSize;
+  }
+
+  public int getBytesPerChecksum() {
+    return bytesPerChecksum;
+  }
+
+  public int getWritePacketSize() {
+    return writePacketSize;
+  }
+
+  public short getReplication() {
+    return replication;
+  }
+
+  public int getFileBufferSize() {
+    return fileBufferSize;
+  }
+
+  // /////////////////////////////////////////
+  // Writable
+  // /////////////////////////////////////////
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(blockSize);
+    out.writeInt(bytesPerChecksum);
+    out.writeInt(writePacketSize);
+    out.writeShort(replication);
+    out.writeInt(fileBufferSize);
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    blockSize = in.readLong();
+    bytesPerChecksum = in.readInt();
+    writePacketSize = in.readInt();
+    replication = in.readShort();
+    fileBufferSize = in.readInt();
+  }
+}