Browse Source

Backport MAPREDUCE-2202, MAPREDUCE-2228, MAPREDUCE-2723, MAPREDUCE-2282 + changes to fix ant tar.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/MR-279@1152932 13f79535-47bb-0310-9956-ffa450edef68
Mahadev Konar 14 years ago
parent
commit
324955c701

+ 4 - 0
mapreduce/CHANGES.txt

@@ -4,6 +4,10 @@ Trunk (unreleased changes)
 
 
     MAPREDUCE-279
+ 
+    Backport MAPREDUCE-2202, MAPREDUCE-2228, MAPREDUCE-2723, MAPREDUCE-2282 +  changes to fix 
+    ant tar.
+
     Fix the findbugs version to make it work with maven 3. (mahadev)
 
     Fixes for making MR-279 work with trunk common/hdfs. (mahadev)

+ 1 - 5
mapreduce/build-utils.xml

@@ -26,11 +26,7 @@ Contains utilities that are common for the main and contrib builds.
   <dirname property="build-utils.basedir" file="${ant.file.build-utils}"/>
   <property file="${build-utils.basedir}/build.properties"/>
 
-  <target name="java5.check" unless="java5.home">
-    <fail message="'java5.home' is not defined.  Forrest requires Java 5.  Please pass -Djava5.home=&lt;base of Java 5 distribution&gt; to Ant on the command-line, or set java5.home in build properties file." />
-  </target>
-	
-  <target name="forrest.check" unless="forrest.home" depends="java5.check">
+  <target name="forrest.check" unless="forrest.home">
     <fail message="'forrest.home' is not defined. Please pass -Dforrest.home=&lt;base of Apache Forrest installation&gt; to Ant on the command-line, or set forest.home in build properties file." />
   </target>
   

+ 1 - 4
mapreduce/build.xml

@@ -31,7 +31,7 @@
   <property name="Name" value="Hadoop-Mapred"/>
   <property name="name" value="hadoop-mapred"/>
   <!-- Need to change aop.xml project.version prop. synchronously -->
-  <property name="version" value="0.22.0-SNAPSHOT"/>
+  <property name="version" value="0.23.0-SNAPSHOT"/>
   <property name="final.name" value="${name}-${version}"/>
   <property name="test.final.name" value="${name}-test-${version}"/>
   <property name="examples.final.name" value="${name}-examples-${version}"/>
@@ -916,7 +916,6 @@
   	      tofile="${build.docs}/mapred-queues.xml" failonerror="false"/>
     <exec dir="${docs.src}" executable="${forrest.home}/bin/forrest"
 	  failonerror="true">
-      <env key="JAVA_HOME" value="${java5.home}"/>
     </exec>
     <copy todir="${build.docs}">
       <fileset dir="${docs.src}/build/site/" />
@@ -1734,7 +1733,6 @@
     <arg value="${findbugs.home}"/>
     <arg value="${forrest.home}"/>
     <arg value="${basedir}"/>
-    <arg value="${java5.home}"/>
   </exec>
 </target>
 
@@ -1757,7 +1755,6 @@
     <arg value="${basedir}"/>
     <arg value="${trigger.url}"/>
     <arg value="${jira.passwd}"/>
-    <arg value="${java5.home}"/>
     <arg value="${curl.cmd}"/>
     <arg value="${defect}"/>
   </exec>

+ 2 - 2
mapreduce/ivy/libraries.properties

@@ -22,8 +22,8 @@ aspectj.version=1.6.5
 
 checkstyle.version=4.2
 
-hadoop-common.version=0.22.0-SNAPSHOT
-hadoop-hdfs.version=0.22.0-SNAPSHOT
+hadoop-common.version=0.23.0-SNAPSHOT
+hadoop-hdfs.version=0.23.0-SNAPSHOT
 
 ivy.version=2.2.0
 

+ 1 - 1
mapreduce/src/contrib/mumak/src/test/org/apache/hadoop/mapred/MockSimulatorJobTracker.java

@@ -466,7 +466,7 @@ public class MockSimulatorJobTracker implements InterTrackerProtocol,
   @Override
   public ProtocolSignature getProtocolSignature(String protocol,
       long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignature.getProtocolSigature(
+    return ProtocolSignature.getProtocolSignature(
         this, protocol, clientVersion, clientMethodsHash);
   }
 }

+ 1 - 1
mapreduce/src/contrib/raid/src/java/org/apache/hadoop/raid/RaidNode.java

@@ -197,7 +197,7 @@ public class RaidNode implements RaidProtocol {
   @Override
   public ProtocolSignature getProtocolSignature(String protocol,
       long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignature.getProtocolSigature(
+    return ProtocolSignature.getProtocolSignature(
         this, protocol, clientVersion, clientMethodsHash);
   }
 

+ 6 - 3
mapreduce/src/docs/forrest.properties

@@ -71,10 +71,13 @@
 #forrest.validate=true
 #forrest.validate.xdocs=${forrest.validate}
 #forrest.validate.skinconf=${forrest.validate}
-#forrest.validate.sitemap=${forrest.validate}
-#forrest.validate.stylesheets=${forrest.validate}
+# Workaround (HADOOP-7072) for http://issues.apache.org/jira/browse/FOR-984
+# Remove when forrest-0.9 is available
+forrest.validate.sitemap=false
+forrest.validate.stylesheets=false
+# End of forrest-0.8 + JDK6 workaround
 #forrest.validate.skins=${forrest.validate}
-#forrest.validate.skins.stylesheets=${forrest.validate.skins}
+forrest.validate.skins.stylesheets=false
 
 # *.failonerror=(true|false) - stop when an XML file is invalid
 #forrest.validate.failonerror=true

+ 1 - 1
mapreduce/src/java/org/apache/hadoop/mapred/IsolationRunner.java

@@ -61,7 +61,7 @@ public class IsolationRunner {
     @Override
     public ProtocolSignature getProtocolSignature(String protocol,
         long clientVersion, int clientMethodsHash) throws IOException {
-      return ProtocolSignature.getProtocolSigature(
+      return ProtocolSignature.getProtocolSignature(
           this, protocol, clientVersion, clientMethodsHash);
     }
 

+ 1 - 1
mapreduce/src/java/org/apache/hadoop/mapred/JobTracker.java

@@ -326,7 +326,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
   @Override
   public ProtocolSignature getProtocolSignature(String protocol,
       long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignature.getProtocolSigature(
+    return ProtocolSignature.getProtocolSignature(
         this, protocol, clientVersion, clientMethodsHash);
   }
 

+ 2 - 2
mapreduce/src/java/org/apache/hadoop/mapred/LocalJobRunner.java

@@ -96,7 +96,7 @@ public class LocalJobRunner implements ClientProtocol {
   @Override
   public ProtocolSignature getProtocolSignature(String protocol,
       long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignature.getProtocolSigature(
+    return ProtocolSignature.getProtocolSignature(
         this, protocol, clientVersion, clientMethodsHash);
   }
 
@@ -136,7 +136,7 @@ public class LocalJobRunner implements ClientProtocol {
     @Override
     public ProtocolSignature getProtocolSignature(String protocol,
         long clientVersion, int clientMethodsHash) throws IOException {
-      return ProtocolSignature.getProtocolSigature(
+      return ProtocolSignature.getProtocolSignature(
           this, protocol, clientVersion, clientMethodsHash);
     }
 

+ 1 - 1
mapreduce/src/java/org/apache/hadoop/mapred/TaskTracker.java

@@ -380,7 +380,7 @@ public class TaskTracker
   @Override
   public ProtocolSignature getProtocolSignature(String protocol,
       long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignature.getProtocolSigature(
+    return ProtocolSignature.getProtocolSignature(
         this, protocol, clientVersion, clientMethodsHash);
   }
 

+ 1 - 1
mapreduce/src/java/org/apache/hadoop/mapreduce/filecache/TrackerDistributedCacheManager.java

@@ -169,7 +169,7 @@ public class TrackerDistributedCacheManager {
         String cachePath = new Path (subDir, 
           new Path(uniqueString, makeRelative(cache, conf))).toString();
         localPath = lDirAllocator.getLocalPathForWrite(cachePath,
-          fileStatus.getLen(), trackerConf, isPublic);
+          fileStatus.getLen(), trackerConf);
         lcacheStatus = 
           new CacheStatus(new Path(localPath.toString().replace(cachePath, "")),
                           localPath, new Path(subDir), uniqueString, 

+ 41 - 0
mapreduce/src/test/mapred/org/apache/hadoop/cli/CLITestCmdMR.java

@@ -0,0 +1,41 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.cli;
+
+import org.apache.hadoop.cli.util.CLICommandTypes;
+import org.apache.hadoop.cli.util.CLITestCmd;
+import org.apache.hadoop.cli.util.CommandExecutor;
+
+public class CLITestCmdMR extends CLITestCmd {
+  public CLITestCmdMR(String str, CLICommandTypes type) {
+    super(str, type);
+  }
+
+  /**
+   * This is not implemented because HadoopArchive constructor requires JobConf
+   * to create an archive object. Because TestMRCLI uses setup method from
+   * TestHDFSCLI the initialization of executor objects happens before a config
+   * is created and updated. Thus, actual calls to executors happen in the body
+   * of the test method.
+   */
+  @Override
+  public CommandExecutor getExecutor(String tag)
+      throws IllegalArgumentException {
+    throw new IllegalArgumentException("Method isn't supported");
+  }
+}

+ 64 - 23
mapreduce/src/test/mapred/org/apache/hadoop/cli/TestMRCLI.java

@@ -18,8 +18,7 @@
 
 package org.apache.hadoop.cli;
 
-import org.apache.hadoop.cli.util.CommandExecutor;
-import org.apache.hadoop.cli.util.CLITestData.TestCmd;
+import org.apache.hadoop.cli.util.*;
 import org.apache.hadoop.cli.util.CommandExecutor.Result;
 import org.apache.hadoop.tools.HadoopArchives;
 import org.apache.hadoop.mapred.JobConf;
@@ -29,51 +28,59 @@ import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
 import org.apache.hadoop.security.authorize.HadoopPolicyProvider;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.util.ToolRunner;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.xml.sax.SAXException;
 
-public class TestMRCLI extends TestHDFSCLI{
+public class TestMRCLI extends TestHDFSCLI {
 
   protected MiniMRCluster mrCluster = null;
   protected String jobtracker = null;
-  protected MRCmdExecutor cmdExecutor = null;
-  protected ArchiveCmdExecutor archiveCmdExecutor = null;
-  
+  private JobConf mrConf;
+
+  @Before
   public void setUp() throws Exception {
     super.setUp();
     conf.setClass(PolicyProvider.POLICY_PROVIDER_CONFIG,
         HadoopPolicyProvider.class, PolicyProvider.class);
-    JobConf mrConf = new JobConf(conf);
+    mrConf = new JobConf(conf);
     mrCluster = new MiniMRCluster(1, dfsCluster.getFileSystem().getUri().toString(), 1, 
                            null, null, mrConf);
     jobtracker = mrCluster.createJobConf().get(JTConfig.JT_IPC_ADDRESS, "local");
-    cmdExecutor = new MRCmdExecutor(jobtracker);
-    archiveCmdExecutor = new ArchiveCmdExecutor(namenode, mrConf);
   }
 
-  
+  @After
   public void tearDown() throws Exception {
     mrCluster.shutdown();
     super.tearDown();
   }
-  
-  protected String getTestFile() {
+
+    @Override
+    protected TestConfigFileParser getConfigParser() {
+        return new TestConfigFileParserMR();
+    }
+
+    protected String getTestFile() {
     return "testMRConf.xml";
   }
-  
+
+  @Override
   protected String expandCommand(final String cmd) {
     String expCmd = cmd;
     expCmd = expCmd.replaceAll("JOBTRACKER", jobtracker);
-    expCmd = super.expandCommand(cmd);
+    expCmd = super.expandCommand(expCmd);
     return expCmd;
   }
-  
-  protected Result execute(TestCmd cmd) throws Exception {
-    if(cmd.getType() == TestCmd.CommandType.MRADMIN) {
-      return cmdExecutor.executeCommand(cmd.getCmd());
-    } else if(cmd.getType() == TestCmd.CommandType.ARCHIVE) {
-      return archiveCmdExecutor.executeCommand(cmd.getCmd());
-    } else {
+
+  @Override
+  protected Result execute(CLICommand cmd) throws Exception {
+    if (cmd.getType() instanceof CLICommandMRAdmin)
+      return new TestMRCLI.MRCmdExecutor(jobtracker).executeCommand(cmd.getCmd());
+    else if (cmd.getType() instanceof CLICommandArchive)
+      return new TestMRCLI.ArchiveCmdExecutor(namenode, mrConf).executeCommand(cmd.getCmd());
+    else
       return super.execute(cmd);
-    }
   }
   
   public static class MRCmdExecutor extends CommandExecutor {
@@ -81,11 +88,13 @@ public class TestMRCLI extends TestHDFSCLI{
     public MRCmdExecutor(String jobtracker) {
       this.jobtracker = jobtracker;
     }
+    @Override
     protected void execute(final String cmd) throws Exception{
       MRAdmin mradmin = new MRAdmin();
       String[] args = getCommandAsArgs(cmd, "JOBTRACKER", jobtracker);
       ToolRunner.run(mradmin, args);
     }
+
   }
   
   public static class ArchiveCmdExecutor extends CommandExecutor {
@@ -95,11 +104,43 @@ public class TestMRCLI extends TestHDFSCLI{
       this.namenode = namenode;
       this.jobConf = jobConf;
     }
+    @Override
     protected void execute(final String cmd) throws Exception {
-//      JobConf job=new JobConf(conf);
       HadoopArchives archive = new HadoopArchives(jobConf);
       String[] args = getCommandAsArgs(cmd, "NAMENODE", namenode);
       ToolRunner.run(archive, args);
     }
   }
+
+  @Test
+  @Override
+  public void testAll () {
+    super.testAll();
+  }
+
+  class TestConfigFileParserMR extends CLITestHelper.TestConfigFileParser {
+    @Override
+    public void endElement(String uri, String localName, String qName)
+        throws SAXException {
+      if (qName.equals("mr-admin-command")) {
+        if (testCommands != null) {
+          testCommands.add(new CLITestCmdMR(charString,
+              new CLICommandMRAdmin()));
+        } else if (cleanupCommands != null) {
+          cleanupCommands.add(new CLITestCmdMR(charString,
+              new CLICommandMRAdmin()));
+        }
+      } else if (qName.equals("archive-command")) {
+        if (testCommands != null) {
+          testCommands.add(new CLITestCmdMR(charString,
+              new CLICommandArchive()));
+        } else if (cleanupCommands != null) {
+          cleanupCommands.add(new CLITestCmdMR(charString,
+              new CLICommandArchive()));
+        }
+      } else {
+        super.endElement(uri, localName, qName);
+      }
+    }
+  }
 }

+ 21 - 0
mapreduce/src/test/mapred/org/apache/hadoop/cli/util/CLICommandArchive.java

@@ -0,0 +1,21 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.cli.util;
+
+public class CLICommandArchive implements CLICommandTypes {
+}

+ 21 - 0
mapreduce/src/test/mapred/org/apache/hadoop/cli/util/CLICommandMRAdmin.java

@@ -0,0 +1,21 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.cli.util;
+
+public class CLICommandMRAdmin implements CLICommandTypes {
+}

+ 12 - 21
mapreduce/src/test/mapred/org/apache/hadoop/mapred/TestMRServerPorts.java

@@ -44,7 +44,7 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 public class TestMRServerPorts extends TestCase {
 
   static { DefaultMetricsSystem.setMiniClusterMode(true); }
-
+  static final String THIS_HOST = TestHDFSServerPorts.getFullHostName() + ":0";
   TestHDFSServerPorts hdfs = new TestHDFSServerPorts();
 
   // Runs the JT in a separate thread
@@ -84,12 +84,9 @@ public class TestMRServerPorts extends TestCase {
   }
   
   private void setDataNodePorts(Configuration conf) {
-    conf.set("dfs.datanode.address", 
-        TestHDFSServerPorts.NAME_NODE_HOST + "0");
-    conf.set("dfs.datanode.http.address", 
-        TestHDFSServerPorts.NAME_NODE_HTTP_HOST + "0");
-    conf.set("dfs.datanode.ipc.address", 
-        TestHDFSServerPorts.NAME_NODE_HOST + "0");
+    conf.set("dfs.datanode.address", THIS_HOST);
+    conf.set("dfs.datanode.http.address", THIS_HOST);
+    conf.set("dfs.datanode.ipc.address", THIS_HOST);
   }
 
   /**
@@ -142,22 +139,20 @@ public class TestMRServerPorts extends TestCase {
       JobConf conf2 = new JobConf(hdfs.getConfig());
       conf2.set(JTConfig.JT_IPC_ADDRESS,
                 FileSystem.getDefaultUri(hdfs.getConfig()).toString());
-      conf2.set(JTConfig.JT_HTTP_ADDRESS,
-        TestHDFSServerPorts.NAME_NODE_HTTP_HOST + 0);
+      conf2.set(JTConfig.JT_HTTP_ADDRESS, THIS_HOST);
       boolean started = canStartJobTracker(conf2);
       assertFalse(started); // should fail
 
       // bind http server to the same port as name-node
-      conf2.set(JTConfig.JT_IPC_ADDRESS, TestHDFSServerPorts.NAME_NODE_HOST + 0);
+      conf2.set(JTConfig.JT_IPC_ADDRESS, THIS_HOST);
       conf2.set(JTConfig.JT_HTTP_ADDRESS,
         hdfs.getConfig().get("dfs.http.address"));
       started = canStartJobTracker(conf2);
       assertFalse(started); // should fail again
 
       // both ports are different from the name-node ones
-      conf2.set(JTConfig.JT_IPC_ADDRESS, TestHDFSServerPorts.NAME_NODE_HOST + 0);
-      conf2.set(JTConfig.JT_HTTP_ADDRESS,
-        TestHDFSServerPorts.NAME_NODE_HTTP_HOST + 0);
+      conf2.set(JTConfig.JT_IPC_ADDRESS, THIS_HOST);
+      conf2.set(JTConfig.JT_HTTP_ADDRESS, THIS_HOST);
       started = canStartJobTracker(conf2);
       assertTrue(started); // should start now
 
@@ -187,24 +182,20 @@ public class TestMRServerPorts extends TestCase {
       // start job tracker on the same port as name-node
       conf2.set(TTConfig.TT_REPORT_ADDRESS,
                 FileSystem.getDefaultUri(hdfs.getConfig()).toString());
-      conf2.set(TTConfig.TT_HTTP_ADDRESS,
-        TestHDFSServerPorts.NAME_NODE_HTTP_HOST + 0);
+      conf2.set(TTConfig.TT_HTTP_ADDRESS, THIS_HOST);
       boolean started = canStartTaskTracker(conf2);
       assertFalse(started); // should fail
 
       // bind http server to the same port as name-node
-      conf2.set(TTConfig.TT_REPORT_ADDRESS,
-        TestHDFSServerPorts.NAME_NODE_HOST + 0);
+      conf2.set(TTConfig.TT_REPORT_ADDRESS, THIS_HOST);
       conf2.set(TTConfig.TT_HTTP_ADDRESS,
         hdfs.getConfig().get("dfs.http.address"));
       started = canStartTaskTracker(conf2);
       assertFalse(started); // should fail again
 
       // both ports are different from the name-node ones
-      conf2.set(TTConfig.TT_REPORT_ADDRESS,
-        TestHDFSServerPorts.NAME_NODE_HOST + 0);
-      conf2.set(TTConfig.TT_HTTP_ADDRESS,
-        TestHDFSServerPorts.NAME_NODE_HTTP_HOST + 0);
+      conf2.set(TTConfig.TT_REPORT_ADDRESS, THIS_HOST);
+      conf2.set(TTConfig.TT_HTTP_ADDRESS, THIS_HOST);
       started = canStartTaskTracker(conf2);
       assertTrue(started); // should start now
     } finally {

+ 1 - 1
mapreduce/src/test/mapred/org/apache/hadoop/mapred/TestTaskCommit.java

@@ -173,7 +173,7 @@ public class TestTaskCommit extends HadoopTestCase {
     @Override
     public ProtocolSignature getProtocolSignature(String protocol,
         long clientVersion, int clientMethodsHash) throws IOException {
-      return ProtocolSignature.getProtocolSigature(
+      return ProtocolSignature.getProtocolSignature(
                   this, protocol, clientVersion, clientMethodsHash);
     }
   }

+ 1 - 1
mapreduce/src/test/mapred/org/apache/hadoop/tools/TestCopyFiles.java

@@ -62,7 +62,7 @@ public class TestCopyFiles extends TestCase {
     ((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hdfs.StateChange")
         ).getLogger().setLevel(Level.OFF);
     ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.OFF);
-    ((Log4JLogger)FSNamesystem.LOG).getLogger().setLevel(Level.OFF);
+    ((Log4JLogger)LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.OFF);
     ((Log4JLogger)DistCp.LOG).getLogger().setLevel(Level.ALL);
   }
   

+ 1 - 1
mapreduce/src/test/mapred/org/apache/hadoop/tools/TestDistCh.java

@@ -48,7 +48,7 @@ public class TestDistCh extends junit.framework.TestCase {
     ((Log4JLogger)LogFactory.getLog("org.apache.hadoop.hdfs.StateChange")
         ).getLogger().setLevel(Level.OFF);
     ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.OFF);
-    ((Log4JLogger)FSNamesystem.LOG).getLogger().setLevel(Level.OFF);
+    ((Log4JLogger)LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.OFF);
     ((Log4JLogger)TaskTracker.LOG).getLogger().setLevel(Level.OFF);
   }
 

+ 20 - 9
mapreduce/yarn/yarn-common/pom.xml

@@ -15,15 +15,26 @@
   </properties>
 
   <dependencies>
-    <dependency>
-      <groupId>log4j</groupId>
-      <artifactId>log4j</artifactId>
-      <version>1.2.12</version>
-    </dependency> 
-   <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>yarn-api</artifactId>
-    </dependency>
+  <dependency>
+    <groupId>log4j</groupId>
+    <artifactId>log4j</artifactId>
+    <version>1.2.12</version>
+    <exclusions>
+        <exclusion>
+          <groupId>com.sun.jdmk</groupId>
+          <artifactId>jmxtools</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.sun.jmx</groupId>
+          <artifactId>jmxri</artifactId>
+        </exclusion>
+    </exclusions>
+   </dependency>  
+   
+  <dependency>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>yarn-api</artifactId>
+  </dependency>
   </dependencies>
 
   <build>