소스 검색

HDFS-1107. Turn on append by default. Contributed by Konstantin Shvachko.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/trunk@939851 13f79535-47bb-0310-9956-ffa450edef68
Konstantin Shvachko 15 년 전
부모
커밋
db67875170

+ 3 - 1
CHANGES.txt

@@ -307,7 +307,7 @@ Trunk (unreleased changes)
     HDFS-1101. TestDiskError.testLocalDirs() fails. (cdouglas via jghoman)
 
     HDFS-1031. Enhance the webUi to list a few of the corrupted files in HDFS.
-    (André Orian via dhruba)
+    (Andre Orian via dhruba)
 
     HDFS-1078. Create static and dynamic versions of libhdfs.
     (Sam Rash via dhruba)
@@ -651,6 +651,8 @@ Release 0.21.0 - Unreleased
 
     HDFS-1067. Create block recovery tests that handle errors. (hairong)
 
+    HDFS-1107. Turn on append by default. (shv)
+
   BUG FIXES
 
     HDFS-76. Better error message to users when commands fail because of 

+ 0 - 9
src/java/hdfs-default.xml

@@ -416,15 +416,6 @@ creations/deletions), or "all".</description>
   </description>
 </property>
 
-<property>
-  <name>dfs.support.append</name>
-  <value>false</value>
-  <description>Does HDFS allow appends to files?
-               This is currently set to false because there are bugs in the
-               "append code" and is not supported in any prodction cluster.
-  </description>
-</property>
-
 <property>
   <name>dfs.datanode.plugins</name>
   <value></value>

+ 1 - 1
src/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -163,7 +163,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_HANDLER_COUNT_KEY = "dfs.namenode.handler.count";
   public static final int     DFS_NAMENODE_HANDLER_COUNT_DEFAULT = 10;
   public static final String  DFS_SUPPORT_APPEND_KEY = "dfs.support.append";
-  public static final boolean DFS_SUPPORT_APPEND_DEFAULT = false;
+  public static final boolean DFS_SUPPORT_APPEND_DEFAULT = true;
   public static final String  DFS_HTTPS_ENABLE_KEY = "dfs.https.enable";
   public static final boolean DFS_HTTPS_ENABLE_DEFAULT = false;
   public static final String  DFS_DEFAULT_CHUNK_VIEW_SIZE_KEY = "dfs.default.chunk.view.size";

+ 4 - 2
src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.fs.DU;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -839,14 +840,15 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
   // Used for synchronizing access to usage stats
   private Object statsLock = new Object();
 
-  boolean supportAppends = false;
+  boolean supportAppends = true;
 
   /**
    * An FSDataset has a directory where it loads its data files.
    */
   public FSDataset(DataStorage storage, Configuration conf) throws IOException {
     this.maxBlocksPerDir = conf.getInt("dfs.datanode.numblocks", 64);
-    this.supportAppends = conf.getBoolean("dfs.support.append", false);
+    this.supportAppends = conf.getBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY,
+                                      DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT);
     FSVolume[] volArray = new FSVolume[storage.getNumStorageDirs()];
     for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
       volArray[idx] = new FSVolume(storage.getStorageDir(idx).getCurrentDir(), conf);

+ 2 - 1
src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -450,7 +450,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     this.blockInvalidateLimit = Math.max(this.blockInvalidateLimit, 
                                          20*(int)(heartbeatInterval/1000));
     this.accessTimePrecision = conf.getLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 0);
-    this.supportAppends = conf.getBoolean("dfs.support.append", false);
+    this.supportAppends = conf.getBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY,
+                                      DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT);
     this.isAccessTokenEnabled = conf.getBoolean(
         DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, 
         DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_DEFAULT);