You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by om...@apache.org on 2011/03/04 05:14:55 UTC

svn commit: r1077437 [1/5] - in /hadoop/common/branches/branch-0.20-security-patches: ./ src/test/aop/build/ src/test/org/apache/hadoop/mapred/ src/test/system/aop/org/apache/hadoop/hdfs/ src/test/system/aop/org/apache/hadoop/hdfs/server/ src/test/syst...

Author: omalley
Date: Fri Mar  4 04:14:53 2011
New Revision: 1077437

URL: http://svn.apache.org/viewvc?rev=1077437&view=rev
Log:
commit 67e4039329625e1af13bcdad5407039e45759207
Author: Konstantin Boudnik <co...@goodenter-lm.local>
Date:   Mon May 3 18:58:34 2010 -0700

    HADOOP-6332 from https://issues.apache.org/jira/secure/attachment/12443539/6332-phase2.patch
    
    +++ b/YAHOO-CHANGES.txt
    +    HADOOP-6332. Large-scale Automated Test Framework (sharad, Sreekanth
    +    Ramakrishnan, at all via cos)
    +

Added:
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/HDFSPolicyProviderAspect.aj
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/server/
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/server/datanode/
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/server/datanode/DataNodeAspect.aj
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/server/namenode/
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/server/namenode/NameNodeAspect.aj
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/MapReducePolicyProviderAspect.aj
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/c++/
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/c++/runAs/
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/c++/runAs/Makefile.in
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/c++/runAs/configure
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/c++/runAs/configure.ac
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/c++/runAs/main.c
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/c++/runAs/runAs.c
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/c++/runAs/runAs.h.in
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/conf/
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/conf/hadoop-policy-system-test.xml
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/conf/system-test.xml
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/hdfs/
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/hdfs/TestHL040.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/hdfs/test/
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/hdfs/test/system/
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/hdfs/test/system/DNClient.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/hdfs/test/system/DNProtocol.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/hdfs/test/system/HDFSCluster.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/hdfs/test/system/HDFSDaemonClient.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/hdfs/test/system/NNClient.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/hdfs/test/system/NNProtocol.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestDistributedCacheModifiedFile.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestDistributedCachePrivateFile.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestDistributedCacheUnModifiedFile.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestFileOwner.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestJobKill.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestPushConfig.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestTaskKilling.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/test/system/process/MultiUserHadoopDaemonRemoteCluster.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/scripts/
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/scripts/pushConfig.sh
    hadoop/common/branches/branch-0.20-security-patches/src/test/testjar/JobKillCommitter.java
Modified:
    hadoop/common/branches/branch-0.20-security-patches/build.xml
    hadoop/common/branches/branch-0.20-security-patches/src/test/aop/build/aop.xml
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/UtilsForTests.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/JobTrackerAspect.aj
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/TaskAspect.aj
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/TaskTrackerAspect.aj
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/test/system/DaemonProtocolAspect.aj
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TTTaskInfoImpl.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestCluster.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestControlledJob.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapred/TestTaskOwner.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapreduce/test/system/JTProtocol.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapreduce/test/system/MRCluster.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapreduce/test/system/TTProtocol.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/mapreduce/test/system/TTTaskInfo.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/test/system/AbstractDaemonClient.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/test/system/AbstractDaemonCluster.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/test/system/DaemonProtocol.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/test/system/process/ClusterProcessManager.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/test/system/process/HadoopDaemonRemoteCluster.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/system/java/org/apache/hadoop/test/system/process/RemoteProcess.java

Modified: hadoop/common/branches/branch-0.20-security-patches/build.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/build.xml?rev=1077437&r1=1077436&r2=1077437&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/build.xml (original)
+++ hadoop/common/branches/branch-0.20-security-patches/build.xml Fri Mar  4 04:14:53 2011
@@ -864,8 +864,6 @@
     <attribute name="classpath" />
     <attribute name="test.dir" />
     <attribute name="fileset.dir" />
-    <attribute name="hadoop.home" default="" />
-    <attribute name="hadoop.conf.dir" default="" />
     <attribute name="hadoop.conf.dir.deployed" default="" />
     <attribute name="test.krb5.conf" default="" />
     <attribute name="test.krb5.conf.filename" default="" />
@@ -904,8 +902,14 @@
                      value="@{test.krb5.conf.filename}"/>
         <sysproperty key="hadoop.policy.file" value="hadoop-policy.xml" />
         <sysproperty key="java.library.path"
-          value="${build.native}/lib:${lib.dir}/native/${build.platform}"/>
-        <sysproperty key="install.c++.examples" value="${install.c++.examples}"/>
+             value="${build.native}/lib:${lib.dir}/native/${build.platform}"/>
+        <sysproperty key="install.c++.examples"
+                     value="${install.c++.examples}" />
+        <sysproperty key="testjar"
+                     value="@{test.dir}/testjar" />
+        <!-- System properties that are specifically set for system tests -->
+        <sysproperty key="test.system.hdrc.deployed.hadoopconfdir"
+                     value="@{hadoop.conf.dir.deployed}" />
         <!-- set io.compression.codec.lzo.class in the child jvm only if it is set -->
         <syspropertyset dynamic="no">
           <propertyref name="io.compression.codec.lzo.class"/>

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/aop/build/aop.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/aop/build/aop.xml?rev=1077437&r1=1077436&r2=1077437&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/aop/build/aop.xml (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/aop/build/aop.xml Fri Mar  4 04:14:53 2011
@@ -88,7 +88,6 @@
   <!-- Classpath for running system tests -->
   <path id="test.system.classpath">
         <pathelement location="${hadoop.conf.dir.deployed}" />
-        <pathelement location="${hadoop.conf.dir}" />
         <pathelement location="${system-test-build-dir}/test/extraconf" />
         <pathelement location="${system-test-build-dir}/test/classes" />
         <pathelement location="${system-test-build-dir}/classes" />
@@ -114,16 +113,11 @@
   <!-- ================ -->
   <!-- run system tests -->
   <!-- ================ -->
-  <target name="test-system" depends="-test-system-deployed, -test-system-local"
+  <target name="test-system" depends="ivy-retrieve-common"
     description="Run system tests">
-  </target>
-
-  <target name="-test-system-local"
-    depends="ivy-retrieve-common, prepare-test-system" 
-    unless="hadoop.conf.dir.deployed">
-    <macro-jar-examples
-      build.dir="${system-test-build-dir}"
-      basedir="${system-test-build-dir}/examples">
+    <subant buildpath="build.xml" target="jar-test-system"/>
+    <macro-jar-examples build.dir="${system-test-build-dir}"
+                        basedir="${system-test-build-dir}/examples">
     </macro-jar-examples>
     <macro-test-runner test.file="${test.all.tests.file}"
                        classpath="test.system.classpath"
@@ -149,12 +143,6 @@
     </macro-test-runner>
   </target>
 
-  <target name="prepare-test-system" depends="jar-test-system">
-    <subant buildpath="build.xml" target="inject-system-faults">
-      <property name="build.dir" value="${system-test-build-dir}" />
-    </subant>
-  </target>
-
   <target name="injectfaults"
           description="Instrument classes with faults and other AOP advices">
     <mkdir dir="${build-fi.dir}"/>
@@ -259,4 +247,43 @@
   </macrodef>
 
   <!--End of Fault Injection (FI) related session-->
+
+  <!-- Start of cluster controller binary target -->
+  <property name="runAs.src" 
+    value ="${test.src.dir}/system/c++/runAs"/>
+  <property name="runAs.build.dir" 
+    value="${system-test-build-dir}/c++-build"/>
+  <property name="runAs.configure.script" 
+    value="${runAs.build.dir}/configure"/>
+  <target name="init-runAs-build">
+    <condition property="runAs.parameters.passed">
+      <not>
+        <equals arg1="${run-as.hadoop.home.dir}" 
+          arg2="$${run-as.hadoop.home.dir}"/>
+      </not>
+    </condition>
+    <fail unless="runAs.parameters.passed" 
+          message="Required parameters run-as.hadoop.home.dir not passed to the build"/>
+    <mkdir dir="${runAs.build.dir}"/>
+    <copy todir="${runAs.build.dir}" overwrite="true">
+      <fileset dir="${runAs.src}" includes="**/*"/>
+    </copy>
+    <chmod perm="+x" file="${runAs.configure.script}">
+    </chmod>
+  </target>
+
+  <target name="configure-runAs" 
+    depends="init-runAs-build">
+    <exec executable="${runAs.configure.script}" 
+      dir="${runAs.build.dir}" failonerror="true">
+      <arg value="--with-home=${run-as.hadoop.home.dir}"/>
+    </exec>
+  </target>
+  <target name="run-as" depends="configure-runAs">
+    <exec executable="${make.cmd}" dir="${runAs.build.dir}" 
+        searchpath="yes" failonerror="yes">
+     <arg value="all" />
+    </exec>
+  </target>
+  <!-- End of cluster controller binary target -->
 </project>

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/UtilsForTests.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/UtilsForTests.java?rev=1077437&r1=1077436&r2=1077437&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/UtilsForTests.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/mapred/UtilsForTests.java Fri Mar  4 04:14:53 2011
@@ -34,6 +34,7 @@ import org.apache.commons.logging.Log;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -815,5 +816,39 @@ public class UtilsForTests {
       throw new RuntimeException("Could not start jt", e);
     }
   }
+
+  /**
+   * This creates a file in the dfs
+   * @param dfs FileSystem Local File System where file needs to be picked
+   * @param URIPATH Path dfs path where file needs to be copied
+   * @param permission FsPermission File permission
+   * @return returns the DataOutputStream
+   */
+  public static DataOutputStream
+      createTmpFileDFS(FileSystem dfs, Path URIPATH,
+      FsPermission permission, String input) throws Exception {
+    //Creating the path with the file
+    DataOutputStream file =
+      FileSystem.create(dfs, URIPATH, permission);
+    file.writeBytes(input);
+    file.close();
+    return file;
+  }
+
+  /**
+   * This formats the long tasktracker name to just the FQDN
+   * @param taskTrackerLong String The long format of the tasktracker string
+   * @return String The FQDN of the tasktracker
+   * @throws Exception
+   */
+  public static String getFQDNofTT (String taskTrackerLong) throws Exception {
+    //Getting the exact FQDN of the tasktracker from the tasktracker string.
+    String[] firstSplit = taskTrackerLong.split("_");
+    String tmpOutput = firstSplit[1];
+    String[] secondSplit = tmpOutput.split(":");
+    String tmpTaskTracker = secondSplit[0];
+    return tmpTaskTracker;
+  }
+
 }
 

Added: hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/HDFSPolicyProviderAspect.aj
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/HDFSPolicyProviderAspect.aj?rev=1077437&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/HDFSPolicyProviderAspect.aj (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/HDFSPolicyProviderAspect.aj Fri Mar  4 04:14:53 2011
@@ -0,0 +1,63 @@
+/**
+ * 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.hdfs;
+
+import java.util.ArrayList;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.test.system.DaemonProtocol;
+import org.apache.hadoop.hdfs.test.system.DNProtocol;
+import org.apache.hadoop.hdfs.test.system.NNProtocol;
+import org.apache.hadoop.security.authorize.Service;
+import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
+
+/**
+ * This aspect adds two HDFS Herriot specific protocols tp the list of 'authorized'
+ * Herriot protocols.
+ * Protocol descriptors i.e. 'security.nn.protocol.acl' have to be added to
+ * <code>hadoop-policy.xml</code> if present
+ */
+public privileged aspect HDFSPolicyProviderAspect {
+  private static final Log LOG = LogFactory
+      .getLog(HDFSPolicyProviderAspect.class);
+
+  ArrayList<Service> herriotHDFSServices = null;
+
+  pointcut updateHDFSServices() :
+    execution (public Service[] HDFSPolicyProvider.getServices());
+
+  Service[] around() : updateHDFSServices () {
+    herriotHDFSServices = new ArrayList<Service>();
+    for (Service s : HDFSPolicyProvider.hdfsServices) {
+      LOG.debug("Copying configured protocol to "
+          + s.getProtocol().getCanonicalName());
+      herriotHDFSServices.add(s);
+    }
+    herriotHDFSServices.add(new Service("security.daemon.protocol.acl",
+        DaemonProtocol.class));
+    herriotHDFSServices.add(new Service("security.nn.protocol.acl",
+        NNProtocol.class));
+    herriotHDFSServices.add(new Service("security.dn.protocol.acl",
+        DNProtocol.class));
+    final Service[] retArray = herriotHDFSServices
+        .toArray(new Service[herriotHDFSServices.size()]);
+    LOG.debug("Number of configured protocols to return: " + retArray.length);
+    return retArray;
+  }
+}

Added: hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/server/datanode/DataNodeAspect.aj
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/server/datanode/DataNodeAspect.aj?rev=1077437&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/server/datanode/DataNodeAspect.aj (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/server/datanode/DataNodeAspect.aj Fri Mar  4 04:14:53 2011
@@ -0,0 +1,67 @@
+/**
+ * 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.hdfs.server.datanode;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.AbstractList;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.test.system.DNProtocol;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.system.DaemonProtocol;
+
+public privileged aspect DataNodeAspect {
+  declare parents : DataNode implements DNProtocol;
+
+  public Configuration DataNode.getDaemonConf() {
+    return super.getConf();
+  }
+
+  pointcut dnConstructorPointcut(Configuration conf, AbstractList<File> dirs) :
+    call(DataNode.new(Configuration, AbstractList<File>))
+    && args(conf, dirs);
+
+  after(Configuration conf, AbstractList<File> dirs) returning (DataNode datanode):
+    dnConstructorPointcut(conf, dirs) {
+    try {
+      UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+      datanode.setUser(ugi.getShortUserName());
+    } catch (IOException e) {
+      datanode.LOG.warn("Unable to get the user information for the " +
+          "Jobtracker");
+    }
+    datanode.setReady(true);
+  }
+
+  pointcut getVersionAspect(String protocol, long clientVersion) :
+    execution(public long DataNode.getProtocolVersion(String ,
+      long) throws IOException) && args(protocol, clientVersion);
+
+  long around(String protocol, long clientVersion) :
+    getVersionAspect(protocol, clientVersion) {
+    if(protocol.equals(DaemonProtocol.class.getName())) {
+      return DaemonProtocol.versionID;
+    } else if(protocol.equals(DNProtocol.class.getName())) {
+      return DNProtocol.versionID;
+    } else {
+      return proceed(protocol, clientVersion);
+    }
+  }
+}
\ No newline at end of file

Added: hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/server/namenode/NameNodeAspect.aj
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/server/namenode/NameNodeAspect.aj?rev=1077437&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/server/namenode/NameNodeAspect.aj (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/hdfs/server/namenode/NameNodeAspect.aj Fri Mar  4 04:14:53 2011
@@ -0,0 +1,77 @@
+/**
+ * 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.hdfs.server.namenode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.test.system.NNProtocol;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.system.DaemonProtocol;
+
+public privileged aspect NameNodeAspect {
+  declare parents : NameNode implements NNProtocol;
+
+  // Namename doesn't store a copy of its configuration
+  // because it can be changed through the life cycle of the object
+  // So, the an exposed reference needs to be added and updated after
+  // new NameNode(Configuration conf) is complete
+  Configuration NameNode.configRef = null;
+
+  // Method simply assign a reference to the NameNode configuration object
+  void NameNode.setRef (Configuration conf) {
+    if (configRef == null)
+      configRef = conf;
+  }
+
+  public Configuration NameNode.getDaemonConf() {
+    return configRef;
+  }
+
+  pointcut nnConstructorPointcut(Configuration conf) :
+    call(NameNode.new(Configuration)) && args(conf);
+
+  after(Configuration conf) returning (NameNode namenode):
+    nnConstructorPointcut(conf) {
+    try {
+      UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+      namenode.setUser(ugi.getShortUserName());
+    } catch (IOException e) {
+      namenode.LOG.warn("Unable to get the user information for the " +
+          "Jobtracker");
+    }
+    namenode.setRef(conf);
+    namenode.setReady(true);
+  }
+
+  pointcut getVersionAspect(String protocol, long clientVersion) :
+    execution(public long NameNode.getProtocolVersion(String ,
+      long) throws IOException) && args(protocol, clientVersion);
+
+  long around(String protocol, long clientVersion) :
+    getVersionAspect(protocol, clientVersion) {
+    if(protocol.equals(DaemonProtocol.class.getName())) {
+      return DaemonProtocol.versionID;
+    } else if(protocol.equals(NNProtocol.class.getName())) {
+      return NNProtocol.versionID;
+    } else {
+      return proceed(protocol, clientVersion);
+    }
+  }
+}
\ No newline at end of file

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/JobTrackerAspect.aj
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/JobTrackerAspect.aj?rev=1077437&r1=1077436&r2=1077437&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/JobTrackerAspect.aj (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/JobTrackerAspect.aj Fri Mar  4 04:14:53 2011
@@ -33,6 +33,7 @@ import org.apache.hadoop.mapreduce.test.
 import org.apache.hadoop.mapreduce.test.system.JobInfo;
 import org.apache.hadoop.mapreduce.test.system.TTInfo;
 import org.apache.hadoop.mapreduce.test.system.TaskInfo;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.system.DaemonProtocol;
 
 /**
@@ -195,6 +196,13 @@ public privileged aspect JobTrackerAspec
   after(JobConf conf, String jobtrackerIndentifier) 
     returning (JobTracker tracker): jtConstructorPointCut(conf, 
         jobtrackerIndentifier) {
+    try {
+      UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+      tracker.setUser(ugi.getShortUserName());
+    } catch (IOException e) {
+      tracker.LOG.warn("Unable to get the user information for the " +
+      		"Jobtracker");
+    }
     tracker.setReady(true);
   }
   

Added: hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/MapReducePolicyProviderAspect.aj
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/MapReducePolicyProviderAspect.aj?rev=1077437&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/MapReducePolicyProviderAspect.aj (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/MapReducePolicyProviderAspect.aj Fri Mar  4 04:14:53 2011
@@ -0,0 +1,58 @@
+/**
+ * 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.mapred;
+
+import java.util.ArrayList;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapreduce.test.system.TTProtocol;
+import org.apache.hadoop.security.authorize.Service;
+import org.apache.hadoop.test.system.DaemonProtocol;
+
+/**
+ * This aspect adds two MR specific Herriot protocols tp the list of
+ * 'authorized' Herriot protocols. Protocol descriptors i.e.
+ * 'security.tt.protocol.acl' have to be added to <code>hadoop-policy.xml</code>
+ * if present
+ */
+public privileged aspect MapReducePolicyProviderAspect {
+  private static final Log LOG = LogFactory
+      .getLog(MapReducePolicyProviderAspect.class);
+  ArrayList<Service> herriotMRServices = null;
+
+  pointcut updateMRServices() :
+    execution (public Service[] MapReducePolicyProvider.getServices());
+
+  Service[] around() : updateMRServices () {
+    herriotMRServices = new ArrayList<Service>();
+    for (Service s : MapReducePolicyProvider.mapReduceServices) {
+      LOG.debug("Copying configured protocol to "
+          + s.getProtocol().getCanonicalName());
+      herriotMRServices.add(s);
+    }
+    herriotMRServices.add(new Service("security.daemon.protocol.acl",
+        DaemonProtocol.class));
+    herriotMRServices.add(new Service("security.tt.protocol.acl",
+        TTProtocol.class));
+    final Service[] retArray = herriotMRServices
+        .toArray(new Service[herriotMRServices.size()]);
+    LOG.debug("Number of configured protocols to return: " + retArray.length);
+    return retArray;
+  }
+}

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/TaskAspect.aj
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/TaskAspect.aj?rev=1077437&r1=1077436&r2=1077437&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/TaskAspect.aj (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/TaskAspect.aj Fri Mar  4 04:14:53 2011
@@ -29,6 +29,7 @@ import org.apache.hadoop.mapred.Task.Tas
 import org.apache.hadoop.mapreduce.test.system.FinishTaskControlAction;
 import org.apache.hadoop.test.system.ControlAction;
 import org.apache.hadoop.test.system.DaemonProtocol;
+import org.apache.hadoop.mapreduce.test.system.TTProtocol;
 
 public privileged aspect TaskAspect {
 
@@ -106,8 +107,8 @@ public privileged aspect TaskAspect {
   after(Class k, long version, InetSocketAddress addr, Configuration conf) 
     throws IOException : rpcInterceptor(k, version, addr, conf) {
     daemonProxy = 
-      (DaemonProtocol) RPC.getProxy(
-          DaemonProtocol.class, DaemonProtocol.versionID, addr, conf);
+      (TTProtocol) RPC.getProxy(
+          TTProtocol.class, TTProtocol.versionID, addr, conf);
   }
   
 }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/TaskTrackerAspect.aj
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/TaskTrackerAspect.aj?rev=1077437&r1=1077436&r2=1077437&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/TaskTrackerAspect.aj (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/mapred/TaskTrackerAspect.aj Fri Mar  4 04:14:53 2011
@@ -26,9 +26,12 @@ import org.apache.hadoop.mapreduce.test.
 import org.apache.hadoop.mapreduce.test.system.TTTaskInfo;
 import org.apache.hadoop.mapred.TTTaskInfoImpl.MapTTTaskInfo;
 import org.apache.hadoop.mapred.TTTaskInfoImpl.ReduceTTTaskInfo;
-import org.apache.hadoop.test.system.ControlAction;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.system.DaemonProtocol;
+import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.mapred.TaskTracker.TaskInProgress;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
 
 public privileged aspect TaskTrackerAspect {
 
@@ -36,6 +39,7 @@ public privileged aspect TaskTrackerAspe
 
   // Add a last sent status field to the Tasktracker class.
   TaskTrackerStatus TaskTracker.lastSentStatus = null;
+  static String TaskTracker.TASKJARDIR = TaskTracker.JARSDIR;
 
   public synchronized TaskTrackerStatus TaskTracker.getStatus()
       throws IOException {
@@ -75,11 +79,11 @@ public privileged aspect TaskTrackerAspe
     if (tip.task.isMapTask()) {
       info = new MapTTTaskInfo(tip.slotTaken, tip.wasKilled,
           (MapTaskStatus) tip.getStatus(), tip.getJobConf(), tip.getTask()
-              .getUser(), tip.getTask().isTaskCleanupTask());
+              .getUser(), tip.getTask().isTaskCleanupTask(), getPid(tip.getTask().getTaskID()));
     } else {
       info = new ReduceTTTaskInfo(tip.slotTaken, tip.wasKilled,
           (ReduceTaskStatus) tip.getStatus(), tip.getJobConf(), tip.getTask()
-              .getUser(), tip.getTask().isTaskCleanupTask());
+              .getUser(), tip.getTask().isTaskCleanupTask(),getPid(tip.getTask().getTaskID()));
     }
     return info;
   }
@@ -98,6 +102,13 @@ public privileged aspect TaskTrackerAspe
 
   after(JobConf conf) returning (TaskTracker tracker): 
     ttConstructorPointCut(conf) {
+    try {
+      UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+      tracker.setUser(ugi.getShortUserName());
+    } catch (IOException e) {
+      tracker.LOG.warn("Unable to get the user information for the " +
+          "Jobtracker");
+    }
     tracker.setReady(true);
   }
   
@@ -114,6 +125,31 @@ public privileged aspect TaskTrackerAspe
     } else {
       return proceed(protocol, clientVersion);
     }
-  }
+  }  
 
+  public boolean TaskTracker.isProcessTreeAlive(String pid) throws IOException {
+    // Command to be executed is as follows :
+    // ps -o pid,ppid,sid,command -e | grep -v ps | grep -v grep | grep
+    // "$pid"
+    String checkerCommand =
+        getDaemonConf().get(
+            "test.system.processgroup_checker_command",
+            "ps -o pid,ppid,sid,command -e "
+                + "| grep -v ps | grep -v grep | grep \"$");
+    String[] command =
+        new String[] { "bash", "-c", checkerCommand + pid + "\"" };
+    ShellCommandExecutor shexec = new ShellCommandExecutor(command);
+    try {
+      shexec.execute();
+    } catch (Shell.ExitCodeException e) {
+      TaskTracker.LOG
+          .info("The process tree grep threw a exitcode exception pointing "
+              + "to process tree not being alive.");
+      return false;
+    }
+    TaskTracker.LOG.info("The task grep command is : "
+        + shexec.toString() + " the output from command is : "
+        + shexec.getOutput());
+    return true;
+  }
 }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/test/system/DaemonProtocolAspect.aj
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/test/system/DaemonProtocolAspect.aj?rev=1077437&r1=1077436&r2=1077437&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/test/system/DaemonProtocolAspect.aj (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/aop/org/apache/hadoop/test/system/DaemonProtocolAspect.aj Fri Mar  4 04:14:53 2011
@@ -25,7 +25,8 @@ import java.util.List;
 import java.util.ArrayList;
 import java.util.Map;
 import java.util.Properties;
-
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.fs.FileStatus;
@@ -47,7 +48,8 @@ public aspect DaemonProtocolAspect {
   @SuppressWarnings("unchecked")
   private HashMap<Object, List<ControlAction>> DaemonProtocol.actions = 
     new HashMap<Object, List<ControlAction>>();
-  
+  private static final Log LOG = LogFactory.getLog(
+      DaemonProtocolAspect.class.getName());
   /**
    * Set if the daemon process is ready or not, concrete daemon protocol should
    * implement pointcuts to determine when the daemon is ready and use the
@@ -239,20 +241,36 @@ public aspect DaemonProtocolAspect {
     return  logDir+File.separator+daemonLogPattern+"*";
   }
 
-  public int DaemonProtocol.getNumberOfMatchesInLogFile(String pattern)
-      throws IOException {
-    String filePattern = getFilePattern();
+  public int DaemonProtocol.getNumberOfMatchesInLogFile(String pattern,
+      String[] list) throws IOException {
+    StringBuffer filePattern = new StringBuffer(getFilePattern());    
+    if(list != null){
+      for(int i =0; i < list.length; ++i)
+      {
+        filePattern.append(" | grep -v " + list[i] );
+      }
+    }  
     String[] cmd =
         new String[] {
             "bash",
             "-c",
             "grep -c "
                 + pattern + " " + filePattern
-                + " | awk -F: '{s+=$2} END {print s}'" };
+                + " | awk -F: '{s+=$2} END {print s}'" };    
     ShellCommandExecutor shexec = new ShellCommandExecutor(cmd);
     shexec.execute();
     String output = shexec.getOutput();
     return Integer.parseInt(output.replaceAll("\n", "").trim());
   }
+
+  private String DaemonProtocol.user = null;
+  
+  public String DaemonProtocol.getDaemonUser() {
+    return user;
+  }
+  
+  public void DaemonProtocol.setUser(String user) {
+    this.user = user;
+  }
 }
 

Added: hadoop/common/branches/branch-0.20-security-patches/src/test/system/c++/runAs/Makefile.in
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/system/c%2B%2B/runAs/Makefile.in?rev=1077437&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/system/c++/runAs/Makefile.in (added)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/system/c++/runAs/Makefile.in Fri Mar  4 04:14:53 2011
@@ -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
+#
+#     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.
+#
+OBJS=main.o runAs.o
+CC=@CC@
+CFLAGS = @CFLAGS@
+BINARY=runAs
+installdir = @prefix@
+
+all: $(OBJS)
+	$(CC) $(CFLAG) -o $(BINARY) $(OBJS)
+
+main.o: runAs.o main.c
+	$(CC) $(CFLAG) -o main.o -c main.c
+
+runAs.o: runAs.h runAs.c
+	$(CC) $(CFLAG) -o runAs.o -c runAs.c
+
+clean:
+	rm -rf $(BINARY) $(OBJS) $(TESTOBJS)
+
+install: all
+	cp $(BINARY) $(installdir)
+
+uninstall:
+	rm -rf $(installdir)/$(BINARY)
+	rm -rf $(BINARY)