瀏覽代碼

Merge branch 'trunk' into HDFS-6581

arp 10 年之前
父節點
當前提交
7f0422be76
共有 19 個文件被更改,包括 58 次插入48 次删除
  1. 4 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 4 21
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
  3. 4 0
      hadoop-common-project/hadoop-kms/src/main/conf/kms-env.sh
  4. 7 0
      hadoop-common-project/hadoop-kms/src/main/libexec/kms-config.sh
  5. 1 0
      hadoop-common-project/hadoop-kms/src/main/sbin/kms.sh
  6. 1 0
      hadoop-common-project/hadoop-kms/src/main/tomcat/server.xml
  7. 1 1
      hadoop-common-project/hadoop-kms/src/main/tomcat/ssl-server.xml
  8. 2 0
      hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm
  9. 7 8
      hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java
  10. 5 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  11. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  12. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  13. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  14. 1 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
  15. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  16. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto
  17. 3 0
      hadoop-yarn-project/CHANGES.txt
  18. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java
  19. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java

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

@@ -554,6 +554,8 @@ Release 2.6.0 - UNRELEASED
     HADOOP-10954. Adding site documents of hadoop-tools (Masatake Iwasaki 
     HADOOP-10954. Adding site documents of hadoop-tools (Masatake Iwasaki 
     via aw)
     via aw)
 
 
+    HADOOP-11153. Make number of KMS threads configurable. (wang)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HADOOP-10838. Byte array native checksumming. (James Thomas via todd)
     HADOOP-10838. Byte array native checksumming. (James Thomas via todd)
@@ -761,6 +763,8 @@ Release 2.6.0 - UNRELEASED
     HADOOP-1110. JavaKeystoreProvider should not report a key as created if it
     HADOOP-1110. JavaKeystoreProvider should not report a key as created if it
     was not flushed to the backing file.
     was not flushed to the backing file.
 
 
+    HADOOP-11130. NFS updateMaps OS check is reversed (brandonli)
+
     BREAKDOWN OF HDFS-6134 AND HADOOP-10150 SUBTASKS AND RELATED JIRAS
     BREAKDOWN OF HDFS-6134 AND HADOOP-10150 SUBTASKS AND RELATED JIRAS
   
   
       HADOOP-10734. Implement high-performance secure random number sources.
       HADOOP-10734. Implement high-performance secure random number sources.

+ 4 - 21
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java

@@ -673,27 +673,11 @@ public class NetworkTopology {
     return node1.getParent()==node2.getParent();
     return node1.getParent()==node2.getParent();
   }
   }
 
 
-  private static final ThreadLocal<Random> r = new ThreadLocal<Random>();
-
-  /**
-   * Getter for thread-local Random, which provides better performance than
-   * a shared Random (even though Random is thread-safe).
-   *
-   * @return Thread-local Random.
-   */
-  protected Random getRandom() {
-    Random rand = r.get();
-    if (rand == null) {
-      rand = new Random();
-      r.set(rand);
-    }
-    return rand;
-  }
+  private static final Random r = new Random();
 
 
   @VisibleForTesting
   @VisibleForTesting
   void setRandomSeed(long seed) {
   void setRandomSeed(long seed) {
-    Random rand = getRandom();
-    rand.setSeed(seed);
+    r.setSeed(seed);
   }
   }
 
 
   /** randomly choose one node from <i>scope</i>
   /** randomly choose one node from <i>scope</i>
@@ -745,7 +729,7 @@ public class NetworkTopology {
           "Failed to find datanode (scope=\"" + String.valueOf(scope) +
           "Failed to find datanode (scope=\"" + String.valueOf(scope) +
           "\" excludedScope=\"" + String.valueOf(excludedScope) + "\").");
           "\" excludedScope=\"" + String.valueOf(excludedScope) + "\").");
     }
     }
-    int leaveIndex = getRandom().nextInt(numOfDatanodes);
+    int leaveIndex = r.nextInt(numOfDatanodes);
     return innerNode.getLeaf(leaveIndex, node);
     return innerNode.getLeaf(leaveIndex, node);
   }
   }
 
 
@@ -918,11 +902,10 @@ public class NetworkTopology {
       list.add(node);
       list.add(node);
     }
     }
 
 
-    Random rand = getRandom();
     int idx = 0;
     int idx = 0;
     for (List<Node> list: tree.values()) {
     for (List<Node> list: tree.values()) {
       if (list != null) {
       if (list != null) {
-        Collections.shuffle(list, rand);
+        Collections.shuffle(list, r);
         for (Node n: list) {
         for (Node n: list) {
           nodes[idx] = n;
           nodes[idx] = n;
           idx++;
           idx++;

+ 4 - 0
hadoop-common-project/hadoop-kms/src/main/conf/kms-env.sh

@@ -36,6 +36,10 @@
 #
 #
 # export KMS_ADMIN_PORT=`expr ${KMS_HTTP_PORT} + 1`
 # export KMS_ADMIN_PORT=`expr ${KMS_HTTP_PORT} + 1`
 
 
+# The maximum number of Tomcat handler threads
+#
+# export KMS_MAX_THREADS=1000
+
 # The location of the SSL keystore if using SSL
 # The location of the SSL keystore if using SSL
 #
 #
 # export KMS_SSL_KEYSTORE_FILE=${HOME}/.keystore
 # export KMS_SSL_KEYSTORE_FILE=${HOME}/.keystore

+ 7 - 0
hadoop-common-project/hadoop-kms/src/main/libexec/kms-config.sh

@@ -136,6 +136,13 @@ else
   print "Using   KMS_ADMIN_PORT:     ${KMS_ADMIN_PORT}"
   print "Using   KMS_ADMIN_PORT:     ${KMS_ADMIN_PORT}"
 fi
 fi
 
 
+if [ "${KMS_MAX_THREADS}" = "" ]; then
+  export KMS_MAX_THREADS=1000
+  print "Setting KMS_MAX_THREADS:     ${KMS_MAX_THREADS}"
+else
+  print "Using   KMS_MAX_THREADS:     ${KMS_MAX_THREADS}"
+fi
+
 if [ "${KMS_SSL_KEYSTORE_FILE}" = "" ]; then
 if [ "${KMS_SSL_KEYSTORE_FILE}" = "" ]; then
   export KMS_SSL_KEYSTORE_FILE=${HOME}/.keystore
   export KMS_SSL_KEYSTORE_FILE=${HOME}/.keystore
   print "Setting KMS_SSL_KEYSTORE_FILE:     ${KMS_SSL_KEYSTORE_FILE}"
   print "Setting KMS_SSL_KEYSTORE_FILE:     ${KMS_SSL_KEYSTORE_FILE}"

+ 1 - 0
hadoop-common-project/hadoop-kms/src/main/sbin/kms.sh

@@ -44,6 +44,7 @@ catalina_opts="${catalina_opts} -Dkms.log.dir=${KMS_LOG}";
 catalina_opts="${catalina_opts} -Dkms.temp.dir=${KMS_TEMP}";
 catalina_opts="${catalina_opts} -Dkms.temp.dir=${KMS_TEMP}";
 catalina_opts="${catalina_opts} -Dkms.admin.port=${KMS_ADMIN_PORT}";
 catalina_opts="${catalina_opts} -Dkms.admin.port=${KMS_ADMIN_PORT}";
 catalina_opts="${catalina_opts} -Dkms.http.port=${KMS_HTTP_PORT}";
 catalina_opts="${catalina_opts} -Dkms.http.port=${KMS_HTTP_PORT}";
+catalina_opts="${catalina_opts} -Dkms.max.threads=${KMS_MAX_THREADS}";
 catalina_opts="${catalina_opts} -Dkms.ssl.keystore.file=${KMS_SSL_KEYSTORE_FILE}";
 catalina_opts="${catalina_opts} -Dkms.ssl.keystore.file=${KMS_SSL_KEYSTORE_FILE}";
 catalina_opts="${catalina_opts} -Dkms.ssl.keystore.pass=${KMS_SSL_KEYSTORE_PASS}";
 catalina_opts="${catalina_opts} -Dkms.ssl.keystore.pass=${KMS_SSL_KEYSTORE_PASS}";
 
 

+ 1 - 0
hadoop-common-project/hadoop-kms/src/main/tomcat/server.xml

@@ -73,6 +73,7 @@
          Define a non-SSL HTTP/1.1 Connector on port ${kms.http.port}
          Define a non-SSL HTTP/1.1 Connector on port ${kms.http.port}
     -->
     -->
     <Connector port="${kms.http.port}" protocol="HTTP/1.1"
     <Connector port="${kms.http.port}" protocol="HTTP/1.1"
+               maxThreads="${kms.max.threads}"
                connectionTimeout="20000"
                connectionTimeout="20000"
                redirectPort="8443"/>
                redirectPort="8443"/>
     <!-- A "Connector" using the shared thread pool-->
     <!-- A "Connector" using the shared thread pool-->

+ 1 - 1
hadoop-common-project/hadoop-kms/src/main/tomcat/ssl-server.xml

@@ -69,7 +69,7 @@
          connector should be using the OpenSSL style configuration
          connector should be using the OpenSSL style configuration
          described in the APR documentation -->
          described in the APR documentation -->
     <Connector port="${kms.http.port}" protocol="HTTP/1.1" SSLEnabled="true"
     <Connector port="${kms.http.port}" protocol="HTTP/1.1" SSLEnabled="true"
-               maxThreads="150" scheme="https" secure="true"
+               maxThreads="${kms.max.threads}" scheme="https" secure="true"
                clientAuth="false" sslProtocol="TLS"
                clientAuth="false" sslProtocol="TLS"
                keystoreFile="${kms.ssl.keystore.file}"
                keystoreFile="${kms.ssl.keystore.file}"
                keystorePass="${kms.ssl.keystore.pass}"/>
                keystorePass="${kms.ssl.keystore.pass}"/>

+ 2 - 0
hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm

@@ -152,6 +152,8 @@ hadoop-${project.version} $ sbin/kms.sh start
 
 
   * KMS_ADMIN_PORT
   * KMS_ADMIN_PORT
 
 
+  * KMS_MAX_THREADS
+
   * KMS_LOG
   * KMS_LOG
 
 
   NOTE: You need to restart the KMS for the configuration changes to take
   NOTE: You need to restart the KMS for the configuration changes to take

+ 7 - 8
hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/IdUserGroup.java

@@ -45,8 +45,8 @@ public class IdUserGroup {
   private final static String OS = System.getProperty("os.name");
   private final static String OS = System.getProperty("os.name");
 
 
   /** Shell commands to get users and groups */
   /** Shell commands to get users and groups */
-  static final String LINUX_GET_ALL_USERS_CMD = "getent passwd | cut -d: -f1,3";
-  static final String LINUX_GET_ALL_GROUPS_CMD = "getent group | cut -d: -f1,3";
+  static final String GET_ALL_USERS_CMD = "getent passwd | cut -d: -f1,3";
+  static final String GET_ALL_GROUPS_CMD = "getent group | cut -d: -f1,3";
   static final String MAC_GET_ALL_USERS_CMD = "dscl . -list /Users UniqueID";
   static final String MAC_GET_ALL_USERS_CMD = "dscl . -list /Users UniqueID";
   static final String MAC_GET_ALL_GROUPS_CMD = "dscl . -list /Groups PrimaryGroupID";
   static final String MAC_GET_ALL_GROUPS_CMD = "dscl . -list /Groups PrimaryGroupID";
 
 
@@ -223,16 +223,15 @@ public class IdUserGroup {
           + "' does not exist.");
           + "' does not exist.");
     }
     }
 
 
-    if (OS.startsWith("Linux")) {
-      updateMapInternal(uMap, "user", LINUX_GET_ALL_USERS_CMD, ":",
+    if (OS.startsWith("Mac")) {
+      updateMapInternal(uMap, "user", MAC_GET_ALL_USERS_CMD, "\\s+",
           staticMapping.uidMapping);
           staticMapping.uidMapping);
-      updateMapInternal(gMap, "group", LINUX_GET_ALL_GROUPS_CMD, ":",
+      updateMapInternal(gMap, "group", MAC_GET_ALL_GROUPS_CMD, "\\s+",
           staticMapping.gidMapping);
           staticMapping.gidMapping);
     } else {
     } else {
-      // Mac
-      updateMapInternal(uMap, "user", MAC_GET_ALL_USERS_CMD, "\\s+",
+      updateMapInternal(uMap, "user", GET_ALL_USERS_CMD, ":",
           staticMapping.uidMapping);
           staticMapping.uidMapping);
-      updateMapInternal(gMap, "group", MAC_GET_ALL_GROUPS_CMD, "\\s+",
+      updateMapInternal(gMap, "group", GET_ALL_GROUPS_CMD, ":",
           staticMapping.gidMapping);
           staticMapping.gidMapping);
     }
     }
 
 

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -520,6 +520,8 @@ Release 2.6.0 - UNRELEASED
 
 
     HDFS-7104. Fix and clarify INodeInPath getter functions. (Zhe Zhang via wang)
     HDFS-7104. Fix and clarify INodeInPath getter functions. (Zhe Zhang via wang)
 
 
+    HDFS-7124. Remove EncryptionZoneManager.NULL_EZ. (clamb via wang)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-6690. Deduplicate xattr names in memory. (wang)
     HDFS-6690. Deduplicate xattr names in memory. (wang)
@@ -530,6 +532,9 @@ Release 2.6.0 - UNRELEASED
     HDFS-6865. Byte array native checksumming on client side
     HDFS-6865. Byte array native checksumming on client side
     (James Thomas via todd)
     (James Thomas via todd)
 
 
+    HDFS-7122. Use of ThreadLocal<Random> results in poor block placement.
+    (wang)
+
   BUG FIXES
   BUG FIXES
 
 
     HDFS-6823. dfs.web.authentication.kerberos.principal shows up in logs for 
     HDFS-6823. dfs.web.authentication.kerberos.principal shows up in logs for 

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -2940,8 +2940,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           throws IOException {
           throws IOException {
     checkOpen();
     checkOpen();
     try {
     try {
-      final EncryptionZone ez = namenode.getEZForPath(src);
-      return (ez.getId() < 0) ? null : ez;
+      return namenode.getEZForPath(src);
     } catch (RemoteException re) {
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
       throw re.unwrapRemoteException(AccessControlException.class,
                                      UnresolvedPathException.class);
                                      UnresolvedPathException.class);

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -1340,7 +1340,9 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       GetEZForPathResponseProto.Builder builder =
       GetEZForPathResponseProto.Builder builder =
           GetEZForPathResponseProto.newBuilder();
           GetEZForPathResponseProto.newBuilder();
       final EncryptionZone ret = server.getEZForPath(req.getSrc());
       final EncryptionZone ret = server.getEZForPath(req.getSrc());
-      builder.setZone(PBHelper.convert(ret));
+      if (ret != null) {
+        builder.setZone(PBHelper.convert(ret));
+      }
       return builder.build();
       return builder.build();
     } catch (IOException e) {
     } catch (IOException e) {
       throw new ServiceException(e);
       throw new ServiceException(e);

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -1347,7 +1347,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
     try {
       final EncryptionZonesProtos.GetEZForPathResponseProto response =
       final EncryptionZonesProtos.GetEZForPathResponseProto response =
           rpcProxy.getEZForPath(null, req);
           rpcProxy.getEZForPath(null, req);
-      return PBHelper.convert(response.getZone());
+      if (response.hasZone()) {
+        return PBHelper.convert(response.getZone());
+      } else {
+        return null;
+      }
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }

+ 1 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java

@@ -57,10 +57,6 @@ public class EncryptionZoneManager {
   public static Logger LOG = LoggerFactory.getLogger(EncryptionZoneManager
   public static Logger LOG = LoggerFactory.getLogger(EncryptionZoneManager
       .class);
       .class);
 
 
-  public static final EncryptionZone NULL_EZ =
-      new EncryptionZone(-1, "", CipherSuite.UNKNOWN,
-          CryptoProtocolVersion.UNKNOWN, "");
-
   /**
   /**
    * EncryptionZoneInt is the internal representation of an encryption zone. The
    * EncryptionZoneInt is the internal representation of an encryption zone. The
    * external representation of an EZ is embodied in an EncryptionZone and
    * external representation of an EZ is embodied in an EncryptionZone and
@@ -226,7 +222,7 @@ public class EncryptionZoneManager {
   EncryptionZone getEZINodeForPath(INodesInPath iip) {
   EncryptionZone getEZINodeForPath(INodesInPath iip) {
     final EncryptionZoneInt ezi = getEncryptionZoneForPath(iip);
     final EncryptionZoneInt ezi = getEncryptionZoneForPath(iip);
     if (ezi == null) {
     if (ezi == null) {
-      return NULL_EZ;
+      return null;
     } else {
     } else {
       return new EncryptionZone(ezi.getINodeId(), getFullPathName(ezi),
       return new EncryptionZone(ezi.getINodeId(), getFullPathName(ezi),
           ezi.getSuite(), ezi.getVersion(), ezi.getKeyName());
           ezi.getSuite(), ezi.getVersion(), ezi.getKeyName());

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -2863,8 +2863,7 @@ public class FSDirectory implements Closeable {
         iip = getINodesInPath(inode.getFullPathName(), true);
         iip = getINodesInPath(inode.getFullPathName(), true);
       }
       }
       EncryptionZone encryptionZone = getEZForPath(iip);
       EncryptionZone encryptionZone = getEZForPath(iip);
-      if (encryptionZone == null ||
-          encryptionZone.equals(EncryptionZoneManager.NULL_EZ)) {
+      if (encryptionZone == null) {
         // not an encrypted file
         // not an encrypted file
         return null;
         return null;
       } else if(encryptionZone.getPath() == null
       } else if(encryptionZone.getPath() == null

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto

@@ -63,5 +63,5 @@ message GetEZForPathRequestProto {
 }
 }
 
 
 message GetEZForPathResponseProto {
 message GetEZForPathResponseProto {
-    required EncryptionZoneProto zone = 1;
+    optional EncryptionZoneProto zone = 1;
 }
 }

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

@@ -465,6 +465,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2608. FairScheduler: Potential deadlocks in loading alloc files and 
     YARN-2608. FairScheduler: Potential deadlocks in loading alloc files and 
     clock access. (Wei Yan via kasha)
     clock access. (Wei Yan via kasha)
 
 
+    YARN-2606. Application History Server tries to access hdfs before doing
+    secure login (Mit Desai via jeagles)
+
 Release 2.5.1 - 2014-09-05
 Release 2.5.1 - 2014-09-05
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/FileSystemApplicationHistoryStore.java

@@ -115,7 +115,8 @@ public class FileSystemApplicationHistoryStore extends AbstractService
   }
   }
 
 
   @Override
   @Override
-  public void serviceInit(Configuration conf) throws Exception {
+  public void serviceStart() throws Exception {
+    Configuration conf = getConfig();
     Path fsWorkingPath =
     Path fsWorkingPath =
         new Path(conf.get(YarnConfiguration.FS_APPLICATION_HISTORY_STORE_URI,
         new Path(conf.get(YarnConfiguration.FS_APPLICATION_HISTORY_STORE_URI,
             conf.get("hadoop.tmp.dir") + "/yarn/timeline/generic-history"));
             conf.get("hadoop.tmp.dir") + "/yarn/timeline/generic-history"));
@@ -132,7 +133,7 @@ public class FileSystemApplicationHistoryStore extends AbstractService
       LOG.error("Error when initializing FileSystemHistoryStorage", e);
       LOG.error("Error when initializing FileSystemHistoryStorage", e);
       throw e;
       throw e;
     }
     }
-    super.serviceInit(conf);
+    super.serviceStart();
   }
   }
 
 
   @Override
   @Override

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java

@@ -61,10 +61,10 @@ public class TestFileSystemApplicationHistoryStore extends
   @Before
   @Before
   public void setup() throws Exception {
   public void setup() throws Exception {
     fs = new RawLocalFileSystem();
     fs = new RawLocalFileSystem();
-    initStore(fs);
+    initAndStartStore(fs);
   }
   }
 
 
-  private void initStore(final FileSystem fs) throws IOException,
+  private void initAndStartStore(final FileSystem fs) throws IOException,
       URISyntaxException {
       URISyntaxException {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     fs.initialize(new URI("/"), conf);
     fs.initialize(new URI("/"), conf);
@@ -272,7 +272,7 @@ public class TestFileSystemApplicationHistoryStore extends
     doReturn(true).when(fs).isDirectory(any(Path.class));
     doReturn(true).when(fs).isDirectory(any(Path.class));
 
 
     try {
     try {
-      initStore(fs);
+      initAndStartStore(fs);
     } catch (Exception e) {
     } catch (Exception e) {
       Assert.fail("Exception should not be thrown: " + e);
       Assert.fail("Exception should not be thrown: " + e);
     }
     }
@@ -293,7 +293,7 @@ public class TestFileSystemApplicationHistoryStore extends
     doThrow(new IOException()).when(fs).mkdirs(any(Path.class));
     doThrow(new IOException()).when(fs).mkdirs(any(Path.class));
 
 
     try {
     try {
-      initStore(fs);
+      initAndStartStore(fs);
       Assert.fail("Exception should have been thrown");
       Assert.fail("Exception should have been thrown");
     } catch (Exception e) {
     } catch (Exception e) {
       // Expected failure
       // Expected failure