You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by tu...@apache.org on 2012/03/14 22:14:14 UTC

svn commit: r1300721 - in /incubator/oozie/trunk: ./ core/src/test/java/org/apache/oozie/command/coord/ core/src/test/java/org/apache/oozie/coord/ core/src/test/java/org/apache/oozie/test/ examples/src/test/java/org/apache/oozie/example/

Author: tucu
Date: Wed Mar 14 21:14:13 2012
New Revision: 1300721

URL: http://svn.apache.org/viewvc?rev=1300721&view=rev
Log:
OOZIE-749 oozie tests doesn't delete the files in tmp directories (virag via tucu)

Modified:
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionStartXCommand.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordRerunCommand.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordRerunXCommand.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/coord/TestCoordELEvaluator.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/test/XTestCase.java
    incubator/oozie/trunk/examples/src/test/java/org/apache/oozie/example/TestLocalOozieExample.java
    incubator/oozie/trunk/pom.xml
    incubator/oozie/trunk/release-log.txt

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionStartXCommand.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionStartXCommand.java?rev=1300721&r1=1300720&r2=1300721&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionStartXCommand.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionStartXCommand.java Wed Mar 14 21:14:13 2012
@@ -6,9 +6,9 @@
  * 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.
@@ -182,7 +182,7 @@ public class TestCoordActionStartXComman
         action.setActionNumber(actionNum);
         action.setNominalTime(new Date());
         action.setStatus(Status.SUBMITTED);
-        String appPath = "/tmp/coord/no-op/";
+        String appPath = getTestCaseDir()+"/coord/no-op/";
         String actionXml = "<coordinator-app xmlns='uri:oozie:coordinator:0.2' xmlns:sla='uri:oozie:sla:0.1' name='NAME' " +
         		"frequency=\"1\" start='2009-02-01T01:00Z' end='2009-02-03T23:59Z' timezone='UTC' freq_timeunit='DAY' " +
         		"end_of_duration='NONE'  instance-number=\"1\" action-nominal-time=\"2009-02-01T01:00Z\">";

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordRerunCommand.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordRerunCommand.java?rev=1300721&r1=1300720&r2=1300721&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordRerunCommand.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordRerunCommand.java Wed Mar 14 21:14:13 2012
@@ -850,24 +850,6 @@ public class TestCoordRerunCommand exten
         return null;
     }
 
-    @Override
-    protected String getTestCaseDir() {
-        String testCaseDir = "/tmp/TestCoordRerunCommand/";
-        System.out.println(XLog.format("Setting testcase work dir[{0}]", testCaseDir));
-        try {
-            delete(new File(testCaseDir));
-        }
-        catch (IOException ioe) {
-            ioe.printStackTrace();
-            throw new RuntimeException(XLog.format("Could not delete testcase dir[{0}]", testCaseDir));
-        }
-        File dir = new File(testCaseDir);
-        if (!dir.mkdirs()) {
-            throw new RuntimeException(XLog.format("Could not create testcase dir[{0}]", testCaseDir));
-        }
-        return testCaseDir;
-    }
-
     private String getCoordJobXml(Path appPath) {
         String inputTemplate = appPath + "/coord-input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
         inputTemplate = Matcher.quoteReplacement(inputTemplate);

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordRerunXCommand.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordRerunXCommand.java?rev=1300721&r1=1300720&r2=1300721&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordRerunXCommand.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/command/coord/TestCoordRerunXCommand.java Wed Mar 14 21:14:13 2012
@@ -1071,24 +1071,6 @@ public class TestCoordRerunXCommand exte
     }
 
     @Override
-    protected String getTestCaseDir() {
-        String testCaseDir = "/tmp/TestCoordRerunXCommand/";
-        System.out.println(XLog.format("Setting testcase work dir[{0}]", testCaseDir));
-        try {
-            delete(new File(testCaseDir));
-        }
-        catch (IOException ioe) {
-            ioe.printStackTrace();
-            throw new RuntimeException(XLog.format("Could not delete testcase dir[{0}]", testCaseDir));
-        }
-        File dir = new File(testCaseDir);
-        if (!dir.mkdirs()) {
-            throw new RuntimeException(XLog.format("Could not create testcase dir[{0}]", testCaseDir));
-        }
-        return testCaseDir;
-    }
-
-    @Override
     protected String getCoordJobXml(Path appPath) {
         String inputTemplate = appPath + "/coord-input/${YEAR}/${MONTH}/${DAY}/${HOUR}/${MINUTE}";
         inputTemplate = Matcher.quoteReplacement(inputTemplate);

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/coord/TestCoordELEvaluator.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/coord/TestCoordELEvaluator.java?rev=1300721&r1=1300720&r2=1300721&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/coord/TestCoordELEvaluator.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/coord/TestCoordELEvaluator.java Wed Mar 14 21:14:13 2012
@@ -6,9 +6,9 @@
  * 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.
@@ -176,24 +176,25 @@ public class TestCoordELEvaluator extend
         // Configuration conf = new
         // XConfiguration(IOUtils.getResourceAsReader("org/apache/oozie/coord/conf.xml",
         // -1));
-        Configuration conf = new XConfiguration(new StringReader(getConfString()));
+    	String testCaseDir = getTestCaseDir();
+    	Configuration conf = new XConfiguration(new StringReader(getConfString()));
 
         Date actualTime = DateUtils.parseDateUTC("2009-09-01T01:00Z");
         Date nominalTime = DateUtils.parseDateUTC("2009-09-01T00:00Z");
-        String dataEvntXML = "<data-in name=\"A\" dataset=\"a\"><uris>file:///tmp/coord/US/2009/1/30|file:///tmp/coord/US/2009/1/31</uris>";
+        String dataEvntXML = "<data-in name=\"A\" dataset=\"a\"><uris>file:///"+testCaseDir+"/US/2009/1/30|file:///tmp/coord/US/2009/1/31</uris>";
         dataEvntXML += "<dataset name=\"a\" frequency=\"1440\" initial-instance=\"2009-01-01T00:00Z\"  freq_timeunit=\"MINUTE\" timezone=\"UTC\" end_of_duration=\"NONE\">";
-        dataEvntXML += "<uri-template>file:///tmp/coord/${YEAR}/${MONTH}/${DAY}</uri-template></dataset></data-in>";
+        dataEvntXML += "<uri-template>file:///"+testCaseDir+"/${YEAR}/${MONTH}/${DAY}</uri-template></dataset></data-in>";
         Element dEvent = XmlUtils.parseXml(dataEvntXML);
         ELEvaluator eval = CoordELEvaluator.createLazyEvaluator(actualTime, nominalTime, dEvent, conf);
-        createDir("/tmp/coord/2009/01/02");
+        createDir(testCaseDir+"/2009/01/02");
         String expr = "${coord:latest(0)} ${coord:latest(-1)}";
         // Dependent on the directory structure
         // TODO: Create the directory
         assertEquals("2009-01-02T00:00Z ${coord:latest(-1)}", eval.evaluate(expr, String.class));
 
         // future
-        createDir("/tmp/coord/2009/09/04");
-        createDir("/tmp/coord/2009/09/05");
+        createDir(testCaseDir+"/2009/09/04");
+        createDir(testCaseDir+"/2009/09/05");
         expr = "${coord:future(1, 30)}";
         assertEquals("2009-09-05T00:00Z", eval.evaluate(expr, String.class));
 

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/test/XTestCase.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/test/XTestCase.java?rev=1300721&r1=1300720&r2=1300721&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/test/XTestCase.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/test/XTestCase.java Wed Mar 14 21:14:13 2012
@@ -245,7 +245,7 @@ public abstract class XTestCase extends 
         source = source.getAbsoluteFile();
         if (!source.exists()) {
             System.err.println();
-            System.err.println(XLog.format("Custom configuration file for testing does no exist [{0}]", 
+            System.err.println(XLog.format("Custom configuration file for testing does no exist [{0}]",
                                            source.getAbsolutePath()));
             System.err.println();
             System.exit(-1);
@@ -266,7 +266,7 @@ public abstract class XTestCase extends 
             System.setProperty("oozie.service.HadoopAccessorService.kerberos.enabled", "true");
         }
         if (System.getProperty("oozie.test.hadoop.minicluster", "true").equals("true")) {
-            setUpEmbeddedHadoop();
+            setUpEmbeddedHadoop(getTestCaseDir());
         }
 
         if (System.getProperty("oozie.test.db.host") == null) {
@@ -633,11 +633,11 @@ public abstract class XTestCase extends 
     private static MiniDFSCluster dfsCluster = null;
     private static MiniMRCluster mrCluster = null;
 
-    private static void setUpEmbeddedHadoop() throws Exception {
+    private void setUpEmbeddedHadoop(String testCaseDir) throws Exception {
         if (dfsCluster == null && mrCluster == null) {
-            if (System.getProperty("hadoop.log.dir") == null) {
-                System.setProperty("hadoop.log.dir", "/tmp");
-            }
+			if (System.getProperty("hadoop.log.dir") == null) {
+				System.setProperty("hadoop.log.dir", testCaseDir);
+			}
             int taskTrackers = 2;
             int dataNodes = 2;
             String oozieUser = getOozieUser();
@@ -665,18 +665,17 @@ public abstract class XTestCase extends 
             UserGroupInformation.createUserForTesting(getTestUser(), userGroups);
             UserGroupInformation.createUserForTesting(getTestUser2(), userGroups);
             UserGroupInformation.createUserForTesting(getTestUser3(), new String[] { "users" } );
-
-            conf.set("hadoop.tmp.dir", "/tmp/minicluster");
+            conf.set("hadoop.tmp.dir", "target/test-data"+"/minicluster");
 
             dfsCluster = new MiniDFSCluster(conf, dataNodes, true, null);
             FileSystem fileSystem = dfsCluster.getFileSystem();
-            fileSystem.mkdirs(new Path("/tmp"));
-            fileSystem.mkdirs(new Path("/tmp/minicluster/mapred"));
+            fileSystem.mkdirs(new Path("target/test-data"));
+            fileSystem.mkdirs(new Path("target/test-data"+"/minicluster/mapred"));
             fileSystem.mkdirs(new Path("/user"));
             fileSystem.mkdirs(new Path("/hadoop/mapred/system"));
-            fileSystem.setPermission(new Path("/tmp"), FsPermission.valueOf("-rwxrwxrwx"));
-            fileSystem.setPermission(new Path("/tmp/minicluster"), FsPermission.valueOf("-rwxrwxrwx"));
-            fileSystem.setPermission(new Path("/tmp/minicluster/mapred"), FsPermission.valueOf("-rwxrwxrwx"));
+            fileSystem.setPermission(new Path("target/test-data"), FsPermission.valueOf("-rwxrwxrwx"));
+            fileSystem.setPermission(new Path("target/test-data"+"/minicluster"), FsPermission.valueOf("-rwxrwxrwx"));
+            fileSystem.setPermission(new Path("target/test-data"+"/minicluster/mapred"), FsPermission.valueOf("-rwxrwxrwx"));
             fileSystem.setPermission(new Path("/user"), FsPermission.valueOf("-rwxrwxrwx"));
             fileSystem.setPermission(new Path("/hadoop/mapred/system"), FsPermission.valueOf("-rwx------"));
             String nnURI = fileSystem.getUri().toString();

Modified: incubator/oozie/trunk/examples/src/test/java/org/apache/oozie/example/TestLocalOozieExample.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/examples/src/test/java/org/apache/oozie/example/TestLocalOozieExample.java?rev=1300721&r1=1300720&r2=1300721&view=diff
==============================================================================
--- incubator/oozie/trunk/examples/src/test/java/org/apache/oozie/example/TestLocalOozieExample.java (original)
+++ incubator/oozie/trunk/examples/src/test/java/org/apache/oozie/example/TestLocalOozieExample.java Wed Mar 14 21:14:13 2012
@@ -32,44 +32,12 @@ import java.io.IOException;
 
 public class TestLocalOozieExample extends XFsTestCase {
     private String oozieLocalLog;
-    private String testDir;
-    private FileSystem fileSystem;
-    private Path fsTestDir;
 
     @Override
     protected void setUp() throws Exception {
         super.setUp();
-        File dir = new File(System.getProperty("oozie.test.dir", "/tmp"));
-        dir = new File(dir, "oozietests");
-        dir = new File(dir, getClass().getName());
-        dir = new File(dir, getName());
-        dir.mkdirs();
-        testDir = dir.getAbsolutePath();
-
-        fileSystem = getFileSystem();
-
-        Path path = new Path(fileSystem.getWorkingDirectory(), "oozietests/" + getClass().getName() + "/" + getName());
-        fsTestDir = fileSystem.makeQualified(path);
-        System.out.println(XLog.format("Setting FS testcase work dir[{0}]", fsTestDir));
-        fileSystem.delete(fsTestDir, true);
-        if (!fileSystem.mkdirs(path)) {
-            throw new IOException(XLog.format("Could not create FS testcase dir [{0}]", fsTestDir));
-        }
         oozieLocalLog = System.getProperty("oozielocal.log");
-        System.setProperty("oozielocal.log", "/tmp/oozielocal.log");
-
-        String testCaseDir = getTestCaseDirInternal(this);
-        File file = new File(testCaseDir);
-        delete(file);
-        if (!file.mkdir()) {
-            throw new RuntimeException(XLog.format("could not create path [{0}]", file.getAbsolutePath()));
-        }
-        file = new File(file, "conf");
-        if (!file.mkdir()) {
-            throw new RuntimeException(XLog.format("could not create path [{0}]", file.getAbsolutePath()));
-        }
-        //setting up Oozie HOME and an empty conf directory
-        //System.setProperty(Services.OOZIE_HOME_DIR, testCaseDir);
+        System.setProperty("oozielocal.log", getTestCaseDir()+"/oozielocal.log");
     }
 
     @Override
@@ -93,19 +61,8 @@ public class TestLocalOozieExample exten
         }
     }
 
-    private String getTestCaseDirInternal(TestCase testCase) {
-        ParamChecker.notNull(testCase, "testCase");
-        File dir = new File(System.getProperty("oozie.test.dir", "/tmp"));
-        dir = new File(dir, "oozietests");
-        dir = new File(dir, testCase.getClass().getName());
-        dir = new File(dir, testCase.getName());
-        return dir.getAbsolutePath();
-    }
-
     @Override
     protected void tearDown() throws Exception {
-        fileSystem = null;
-        fsTestDir = null;
         if (oozieLocalLog != null) {
             System.setProperty("oozielocal.log", oozieLocalLog);
         }
@@ -117,20 +74,20 @@ public class TestLocalOozieExample exten
     }
 
     public void testLocalOozieExampleEnd() throws IOException {
-        Path app = new Path(fsTestDir, "app");
-        File props = new File(testDir, "job.properties");
+        Path app = new Path(getFsTestCaseDir(), "app");
+        File props = new File(getTestCaseDir(), "job.properties");
         IOUtils.copyStream(IOUtils.getResourceAsStream("localoozieexample-wf.xml", -1),
-                           fileSystem.create(new Path(app, "workflow.xml")));
+                           getFileSystem().create(new Path(app, "workflow.xml")));
         IOUtils.copyStream(IOUtils.getResourceAsStream("localoozieexample-end.properties", -1),
                            new FileOutputStream(props));
         assertEquals(0, LocalOozieExample.execute(app.toString(), props.toString()));
     }
 
     public void testLocalOozieExampleKill() throws IOException {
-        Path app = new Path(fsTestDir, "app");
-        File props = new File(testDir, "job.properties");
+        Path app = new Path(getFsTestCaseDir(), "app");
+        File props = new File(getTestCaseDir(), "job.properties");
         IOUtils.copyStream(IOUtils.getResourceAsStream("localoozieexample-wf.xml", -1),
-                           fileSystem.create(new Path(app, "workflow.xml")));
+                           getFileSystem().create(new Path(app, "workflow.xml")));
         IOUtils.copyStream(IOUtils.getResourceAsStream("localoozieexample-kill.properties", -1),
                            new FileOutputStream(props));
         assertEquals(-1, LocalOozieExample.execute(app.toString(), props.toString()));

Modified: incubator/oozie/trunk/pom.xml
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/pom.xml?rev=1300721&r1=1300720&r2=1300721&view=diff
==============================================================================
--- incubator/oozie/trunk/pom.xml (original)
+++ incubator/oozie/trunk/pom.xml Wed Mar 14 21:14:13 2012
@@ -628,8 +628,8 @@
 
     <build>
         <pluginManagement>
-            <plugins>
-                <plugin>
+            	<plugins>
+	            <plugin>
                     <groupId>org.apache.maven.plugins</groupId>
                     <artifactId>maven-enforcer-plugin</artifactId>
                     <version>1.0</version>
@@ -710,7 +710,7 @@
         </pluginManagement>
 
         <plugins>
-            <plugin>
+			<plugin>
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-enforcer-plugin</artifactId>
                 <inherited>false</inherited>
@@ -767,7 +767,6 @@
                     <forkedProcessTimeoutInSeconds>${test.timeout}</forkedProcessTimeoutInSeconds>
                     <argLine>-Xmx1024m -da -XX:MaxPermSize=512m</argLine>
                     <systemPropertyVariables>
-                        <hadoop.log.dir>/tmp</hadoop.log.dir>
                         <oozie.test.db.host>${oozie.test.db.host}</oozie.test.db.host>
                         <oozie.test.config.file>${oozie.test.config.file}</oozie.test.config.file>
                         <oozie.data.dir>${oozie.data.dir}</oozie.data.dir>

Modified: incubator/oozie/trunk/release-log.txt
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/release-log.txt?rev=1300721&r1=1300720&r2=1300721&view=diff
==============================================================================
--- incubator/oozie/trunk/release-log.txt (original)
+++ incubator/oozie/trunk/release-log.txt Wed Mar 14 21:14:13 2012
@@ -1,5 +1,6 @@
 -- Oozie 3.2.0 release
 
+OOZIE-749 oozie tests doesn't delete the files in tmp directories (virag via tucu)
 OOZIE-761 Fixes in CoordELFunctions and testcases for Hadoop 0.23 (tucu)
 OOZIE-760 oozied.sh needs to be able to pick up CATALINA_HOME (rvs via tucu)
 OOZIE-757 hive-contrib version should use ${hive.version} (abayer via tucu)