Browse Source

Merge -r 1414728:1414729 from trunk to branch-2. Fixes: MAPREDUCE-4778. Fair scheduler event log is only written if directory exists on HDFS. Contributed by Sandy Ryza.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1414730 13f79535-47bb-0310-9956-ffa450edef68
Thomas White 12 years ago
parent
commit
3eedaed911

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

@@ -95,6 +95,9 @@ Release 2.0.3-alpha - Unreleased
     YARN-222. Fair scheduler should create queue for each user by default.
     (Sandy Ryza via tomwhite)
 
+    MAPREDUCE-4778. Fair scheduler event log is only written if directory
+    exists on HDFS. (Sandy Ryza via tomwhite)
+
 Release 2.0.2-alpha - 2012-09-07 
 
     YARN-9. Rename YARN_HOME to HADOOP_YARN_HOME. (vinodkv via acmurthy)

+ 8 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairSchedulerEventLog.java

@@ -77,12 +77,11 @@ class FairSchedulerEventLog {
   boolean init(FairSchedulerConfiguration conf) {
     try {
       logDir = conf.getEventlogDir();
-      Path logDirPath = new Path(logDir);
-      FileSystem fs = logDirPath.getFileSystem(conf);
-      if (!fs.exists(logDirPath)) {
-        if (!fs.mkdirs(logDirPath)) {
+      File logDirFile = new File(logDir);
+      if (!logDirFile.exists()) {
+        if (!logDirFile.mkdirs()) {
           throw new IOException(
-              "Mkdirs failed to create " + logDirPath.toString());
+              "Mkdirs failed to create " + logDirFile.toString());
         }
       }
       String username = System.getProperty("user.name");
@@ -142,4 +141,8 @@ class FairSchedulerEventLog {
   synchronized boolean isEnabled() {
     return !logDisabled;
   }
+  
+  public String getLogFile() {
+    return logFile;
+  }
 }

+ 78 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairSchedulerEventLog.java

@@ -0,0 +1,78 @@
+/**
+ * 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.yarn.server.resourcemanager.scheduler.fair;
+
+import java.io.File;
+import java.io.IOException;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFairSchedulerEventLog {
+  private File logFile;
+  private FairScheduler scheduler;
+  private ResourceManager resourceManager;
+  
+  @Before
+  public void setUp() throws IOException {
+    scheduler = new FairScheduler();
+    
+    Configuration conf = new YarnConfiguration();
+    conf.setClass(YarnConfiguration.RM_SCHEDULER, FairScheduler.class,
+        ResourceScheduler.class);
+    conf.set("mapred.fairscheduler.eventlog.enabled", "true");
+
+    // All tests assume only one assignment per node update
+    conf.set(FairSchedulerConfiguration.ASSIGN_MULTIPLE, "false");
+    Store store = StoreFactory.getStore(conf);
+    resourceManager = new ResourceManager(store);
+    resourceManager.init(conf);
+    ((AsyncDispatcher)resourceManager.getRMContext().getDispatcher()).start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+  }
+
+  /**
+   * Make sure the scheduler creates the event log.
+   */
+  @Test
+  public void testCreateEventLog() throws IOException {
+    FairSchedulerEventLog eventLog = scheduler.getEventLog();
+    
+    logFile = new File(eventLog.getLogFile());
+    Assert.assertTrue(logFile.exists());
+  }
+  
+  @After
+  public void tearDown() {
+    logFile.delete();
+    logFile.getParentFile().delete(); // fairscheduler/
+    scheduler = null;
+    resourceManager = null;
+  }
+}