浏览代码

HUE-753 [jb] Re-enable unit tests for job browser

- Remove jobsubd
- Added OozieServerProvider
- Fetching job tracker job IDs from jobs list
abec 13 年之前
父节点
当前提交
1bc3e17863

+ 189 - 147
apps/jobbrowser/src/jobbrowser/tests.py

@@ -15,19 +15,25 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+import json
+import logging
 import time
+import unittest
 
 from nose.tools import assert_true, assert_false, assert_equal
 from nose.plugins.skip import SkipTest
 
 from desktop.lib.django_test_util import make_logged_in_client
-from hadoop import mini_cluster
-from jobsub.models import JobDesign
+from desktop.lib.test_utils import grant_access
+from jobsub.models import OozieDesign
 from liboozie.oozie_api_test import OozieServerProvider
 
 from jobbrowser import models, views
 
 
+LOG = logging.getLogger(__name__)
+
+
 def test_dots_to_camel_case():
   assert_equal("fooBar", models.dots_to_camel_case("foo.bar"))
   assert_equal("fooBarBaz", models.dots_to_camel_case("foo.bar.baz"))
@@ -42,56 +48,66 @@ def test_format_counter_name():
   assert_equal("Foo Bar Baz", views.format_counter_name("fooBarBaz"))
   assert_equal("Foo", views.format_counter_name("foo"))
   assert_equal("Foo.", views.format_counter_name("foo."))
-  assert_equal("A Bbb Ccc", views.format_counter_name("A_BBB_CCC"))
-
-
-def get_hadoop_job_id(jobsubd, jobsub_id, timeout_sec=60):
-  handle = SubmissionHandle(id=jobsub_id)
-  job_data = jobsubd.client.get_job_data(handle)
+  assert_equal("A Bbb Ccc", views.format_counter_name("A_BBB_CCC"))\
 
+def get_hadoop_job_id(oozie_api, oozie_jobid, action_index=1, timeout=60, step=5):
+  hadoop_job_id = None
   start = time.time()
-  while len(job_data.hadoop_job_ids) == 0:
-    assert_true(time.time() - start < timeout_sec,
-        "Timed out waiting for job to start")
-    time.sleep(1)
-    job_data = jobsubd.client.get_job_data(handle)
-
-  return job_data.hadoop_job_ids[0]
-
-
-class TestJobBrowserWithHadoop(object):
+  while not hadoop_job_id and time.time() - start < timeout:
+    time.sleep(step)
+    hadoop_job_id = oozie_api.get_job(oozie_jobid).actions[action_index].externalId
+  if not hadoop_job_id:
+    logs = OozieServerProvider.oozie.get_job_log(oozie_jobid)
+    msg = "[%d] %s took more than %d to create a job: %s" % (time.time(), oozie_jobid, timeout, logs)
+    LOG.info(msg)
+    raise Exception(msg)
+  return hadoop_job_id
+
+class TestJobBrowserWithHadoop(unittest.TestCase, OozieServerProvider):
   """
   Tests for JobBrowser that requires Hadoop. Use the same mini_cluster and jobsubd.
   """
   requires_hadoop = True
+  user_count = 0
 
   @classmethod
   def setup_class(cls):
-    raise SkipTest
-    client = make_logged_in_client('test')
-    cluster = mini_cluster.shared_cluster(conf=True)
-    jobsubd = in_process_jobsubd(cluster.config_dir)
+    OozieServerProvider.setup_class()
+    if not cls.cluster.fs.exists("/tmp"):
+      cls.cluster.fs.do_as_superuser(cls.cluster.fs.mkdir, "/tmp")
+    cls.cluster.fs.do_as_superuser(cls.cluster.fs.chmod, "/tmp", 0777)
 
-    # Make home directory
-    cluster.fs.setuser(cluster.superuser)
-    if not cluster.fs.exists("/user/test"):
-      cluster.fs.mkdir("/user/test")
-    cluster.fs.chown("/user/test", "test", "test")
+    # Install examples
+    import jobsub.management.commands.jobsub_setup as jobsub_setup
+    if not jobsub_setup.Command().has_been_setup():
+      jobsub_setup.Command().handle()
 
-    if not cluster.fs.exists("/tmp"):
-      cluster.fs.mkdir("/tmp")
-    cluster.fs.chmod("/tmp", int('777', 8))
+    cls.sleep_design_id = OozieDesign.objects.get(name='sleep_job').id
 
-    cluster.fs.setuser("test")
+  def setUp(self):
+    TestJobBrowserWithHadoop.user_count += 1
+    self.username = 'test' + str(TestJobBrowserWithHadoop.user_count)
+    self.home_dir = '/user/%s' % self.username
 
-    cls.cluster = cluster
-    cls.client = client
-    cls.jobsubd = jobsubd
+    self.cluster.fs.do_as_user(self.username, self.cluster.fs.create_home_dir, self.home_dir)
+    self.cluster.fs.do_as_superuser(self.cluster.fs.chmod, self.home_dir, 0777, True)
+    self.cluster.fs.do_as_superuser(self.cluster.fs.chown, self.home_dir, self.username, "test", recursive=True)
 
-  @classmethod
-  def teardown_class(cls):
-    cls.jobsubd.exit()
-    cls.cluster.shutdown()
+    self.client = make_logged_in_client(username=self.username, is_superuser=False, groupname='test')
+    grant_access(self.username, 'test', 'jobsub')
+    grant_access(self.username, 'test', 'jobbrowser')
+
+    # Ensure access to MR folder
+    self.cluster.fs.do_as_superuser(self.cluster.fs.chmod, '/tmp', 0777, recursive=True)
+
+    self.cluster.fs.setuser(self.username)
+
+  def tearDown(self):
+    try:
+      # Remove user home directories.
+      self.cluster.fs.do_as_superuser(self.cluster.fs.rmtree, self.home_dir)
+    except:
+      pass
 
   def test_uncommon_views(self):
     """
@@ -103,49 +119,55 @@ class TestJobBrowserWithHadoop(object):
     self.client.get("/jobbrowser/queues")
     self.client.get("/jobbrowser/jobbrowser")
 
-    # This is not tested.
-    # assert_equal("{}", self.client.get("/jobbrowser/jobs/%s/setpriority?priority=HIGH" % hadoop_job_id).content)
-
   def test_failed_jobs(self):
     """
     Test jobs with genuine failure, not just killed
     """
     # Create design that will fail because the script file isn't there
-    self.cluster.fs.mkdir("/user/test/jt-test_failed_jobs")
-    response = self.client.post("/jobsub/new/streaming", dict(
-          hadoop_properties='',
-          inputformat_class_0='org.apache.hadoop.mapred.TextInputFormat',
-          input='/user/test/jt-test_failed_jobs',
-          mapper_cmd="python hogalabogus.py",
-          name="bogus stream",
-          num_reduce_tasks="1",
-          outputformat_class_0='org.apache.hadoop.mapred.TextOutputFormat',
-          output='/user/test/out_stream.tmp',
-          reducer_cmd='python hogalabogus.py',
-          save_submit='off',
-          submit='Save',
-    ))
+    INPUT_DIR = self.home_dir + '/input'
+    OUTPUT_DIR = self.home_dir + '/output'
+    try:
+        self.cluster.fs.mkdir(self.home_dir + "/jt-test_failed_jobs")
+        self.cluster.fs.mkdir(INPUT_DIR)
+        self.cluster.fs.rmtree(OUTPUT_DIR)
+    except:
+        # rmtree probably failed here.
+        pass
+    response = self.client.post('/jobsub/new_design/mapreduce', {
+        'wf-name': ['test_failed_jobs-1'],
+        'wf-description': ['description test_failed_jobs-1'],
+        'action-args': [''],
+        'action-jar_path': ['/user/hue/jobsub/examples/hadoop-examples.jar'],
+        'action-archives': ['[]'],
+        'action-job_properties': ['[{"name":"mapred.input.dir","value":"%s"},\
+            {"name":"mapred.output.dir","value":"%s"},\
+            {"name":"mapred.mapper.class","value":"org.apache.hadoop.mapred.lib.dne"},\
+            {"name":"mapred.combiner.class","value":"org.apache.hadoop.mapred.lib.dne"},\
+            {"name":"mapred.reducer.class","value":"org.apache.hadoop.mapred.lib.dne"}]' % (INPUT_DIR, OUTPUT_DIR)],
+        'action-files': ['[]']}, follow=True)
+    designs = json.loads(response.context['designs'])
 
     # Submit the job
-    design_id = response.context["saved"]
-    response = self.client.post("/jobsub/submit/%d" % design_id)
-    watch_id = parse_out_id(response)
-    response = watch_till_complete(self.client, watch_id, timeout_sec=120)
-    job_id = Submission.objects.get(id=watch_id).submission_handle.id
-    hadoop_job_id = get_hadoop_job_id(self.jobsubd, job_id)
+    design_id = designs[0]['id']
+    response = self.client.post("/jobsub/submit_design/%d" % design_id, follow=True)
+    oozie_jobid = response.context['jobid']
+    job = OozieServerProvider.wait_until_completion(oozie_jobid, timeout=500, step=1)
+    hadoop_job_id = get_hadoop_job_id(self.oozie, oozie_jobid, 1)
 
-    # The single job view should have the failed task table
-    response = self.client.get('/jobbrowser/jobs/%s' % (hadoop_job_id,))
-    html = response.content.lower()
-    assert_true('failed task' in html)
+    # Select only killed jobs (should be absent)
+    # Taking advantage of the fact new jobs are at the top of the list!
+    response = self.client.get('/jobbrowser/jobs/?state=killed')
+    assert_false(hadoop_job_id in response.content)
 
     # Select only failed jobs (should be present)
+    # Map job should succeed. Reduce job should fail.
     response = self.client.get('/jobbrowser/jobs/?state=failed')
     assert_true(hadoop_job_id in response.content)
 
-    # Select only killed jobs (should be absent)
-    response = self.client.get('/jobbrowser/jobs/?state=killed')
-    assert_true(hadoop_job_id not in response.content)
+    # The single job view should have the failed task table
+    response = self.client.get('/jobbrowser/jobs/%s' % (hadoop_job_id,))
+    html = response.content.lower()
+    assert_true('failed task' in html)
 
     # The map task should say success (empty input)
     map_task_id = hadoop_job_id.replace('job', 'task') + '_m_000000'
@@ -164,83 +186,32 @@ class TestJobBrowserWithHadoop(object):
     assert_true('_r_000000' in response.content)
     assert_true('_m_000000' not in response.content)
 
-
-  def test_new_jobs(self):
+  def test_kill_job(self):
     """
-    Submit jobs. Let them succeed or fail and view them.
+    Test job in kill state.
     """
-    # Install examples
-    import jobsub.management.commands.jobsub_setup as jobsub_setup
-    if not jobsub_setup.Command().has_been_setup():
-      jobsub_setup.Command().handle()
+    # Clone design
+    assert_equal(0, OozieDesign.objects.filter(owner__username=self.username).count())
+    self.client.post('/jobsub/clone_design/%d' % self.sleep_design_id)
+    assert_equal(1, OozieDesign.objects.filter(owner__username=self.username).count())
 
     # Run the sleep example, since it doesn't require user home directory
-    design_id = JobDesign.objects.get(name__contains="Example: Sleep").id
-    response = self.client.post("/jobsub/submit/%d" % (design_id,), dict(
-                                                            map_sleep_time_millis=1,
-                                                            num_mappers=1,
-                                                            num_reducers=1,
-                                                            reduce_sleep_time_millis=1))
-    watch_id = parse_out_id(response)
-    response = watch_till_complete(self.client, watch_id)
-    job_id = Submission.objects.get(id=watch_id).submission_handle.id
-    hadoop_job_id = get_hadoop_job_id(self.jobsubd, job_id)
-
-    # All jobs page
-    response = self.client.get('/jobbrowser/jobs/')
-    assert_true(hadoop_job_id.lstrip('job_') in response.content)
-
-    # Single job page
-    response = self.client.get('/jobbrowser/jobs/%s' % hadoop_job_id)
-
-    # Check some counters for single job.
-    counters = response.context['job'].counters
-    counters_file_bytes_written = counters['FileSystemCounters']['counters']['FILE_BYTES_WRITTEN']
-    assert_true(counters_file_bytes_written['map'] > 0)
-    assert_true(counters_file_bytes_written['reduce'] > 0)
-    assert_equal(counters_file_bytes_written['displayName'], 'FILE_BYTES_WRITTEN')
-    assert_equal(counters_file_bytes_written['displayName'], 'FILE_BYTES_WRITTEN')
-
-    # We can't just check the complete contents of the python map because the
-    # SLOTS_MILLIS_* entries have a variable number of milliseconds from
-    # run-to-run.
-    assert_equal(response.context['job'].counters['org.apache.hadoop.mapred.JobInProgress$Counter']['counters']['TOTAL_LAUNCHED_MAPS']['total'], 1)
-    assert_equal(response.context['job'].counters['org.apache.hadoop.mapred.JobInProgress$Counter']['counters']['TOTAL_LAUNCHED_REDUCES']['total'], 1)
-    assert_equal(response.context['job'].counters['org.apache.hadoop.mapred.JobInProgress$Counter']['counters']['FALLOW_SLOTS_MILLIS_MAPS']['total'], 0)
-    assert_equal(response.context['job'].counters['org.apache.hadoop.mapred.JobInProgress$Counter']['counters']['FALLOW_SLOTS_MILLIS_REDUCES']['total'], 0)
-    assert_true(response.context['job'].counters['org.apache.hadoop.mapred.JobInProgress$Counter']['counters']['SLOTS_MILLIS_MAPS']['total'] > 0)
-    assert_true(response.context['job'].counters['org.apache.hadoop.mapred.JobInProgress$Counter']['counters']['SLOTS_MILLIS_REDUCES']['total'] > 0)
-
-    # Check conf keys made it
-    assert_equal(response.context['job'].conf_keys['mapredReducerClass'],
-                 'org.apache.hadoop.examples.SleepJob')
-
-    # There should be 4 tasks for this job: cleanup, setup, map, reduce
-    response = self.client.get('/jobbrowser/jobs/%s/tasks' % (hadoop_job_id,))
-    assert_true('Showing 1 to 4 of 4 tasks' in response.content)
-    # Select by tasktype
-    response = self.client.get('/jobbrowser/jobs/%s/tasks?tasktype=reduce' % (hadoop_job_id,))
-    assert_true('Showing 1 to 1 of 1 tasks' in response.content)
-    # Select by taskstate
-    response = self.client.get('/jobbrowser/jobs/%s/tasks?taskstate=succeeded' % (hadoop_job_id,))
-    assert_true('Showing 1 to 4 of 4 tasks' in response.content)
-    # Select by text
-    response = self.client.get('/jobbrowser/jobs/%s/tasks?tasktext=clean' % (hadoop_job_id,))
-    assert_true('Showing 1 to 1 of 1 tasks' in response.content)
-
-    # Run another sleep job but kill it
-    response = self.client.post("/jobsub/submit/%d" % (design_id,), dict(
-                                                            map_sleep_time_millis=1,
-                                                            num_mappers=2000,
-                                                            num_reducers=2000,
-                                                            reduce_sleep_time_millis=1))
-    job_id = parse_out_id(response)
-    hadoop_job_id = get_hadoop_job_id(self.jobsubd, job_id)
-
-    client2 = make_logged_in_client('test_non_superuser', is_superuser=False)
+    design_id = OozieDesign.objects.get(owner__username=self.username).id
+    response = self.client.post("/jobsub/submit_design/%d" % (design_id,),
+      dict(map_sleep_time=1,
+           num_maps=1,
+           num_reduces=1,
+           reduce_sleep_time=1),
+      follow=True)
+    oozie_jobid = response.context['jobid']
+
+    # Wait for a job to be created and fetch job ID
+    hadoop_job_id = get_hadoop_job_id(self.oozie, oozie_jobid, 1)
+
+    client2 = make_logged_in_client('test_non_superuser', is_superuser=False, groupname='test')
+    grant_access('test_non_superuser', 'test', 'jobbrowser')
     response = client2.post('/jobbrowser/jobs/%s/kill' % (hadoop_job_id,))
-    assert_equal("Permission denied.  User test_non_superuser cannot delete user test's job.",
-      response.context["error"])
+    assert_equal("Permission denied.  User test_non_superuser cannot delete user %s's job." % self.username, response.context["error"])
 
     # Make sure that the first map task succeeds before moving on
     # This will keep us from hitting timing-related failures
@@ -254,6 +225,7 @@ class TestJobBrowserWithHadoop(object):
       assert_true(time.time() - start < timeout_sec,
           "Timed out waiting for first mapper to complete")
 
+    # Kill task
     self.client.post('/jobbrowser/jobs/%s/kill' % (hadoop_job_id,))
 
     # It should say killed
@@ -261,15 +233,15 @@ class TestJobBrowserWithHadoop(object):
     html = response.content.lower()
     assert_true(hadoop_job_id in html)
     assert_true('killed' in html)
+
     # Exercise select by taskstate
     self.client.get('/jobbrowser/jobs/%s/tasks?taskstate=failed' % (hadoop_job_id,))
-    self.client.get('/jobbrowser/jobs/%s/tasks?taskstate=pending' % (hadoop_job_id,))
     self.client.get('/jobbrowser/jobs/%s/tasks?taskstate=succeeded' % (hadoop_job_id,))
     self.client.get('/jobbrowser/jobs/%s/tasks?taskstate=running' % (hadoop_job_id,))
     self.client.get('/jobbrowser/jobs/%s/tasks?taskstate=killed' % (hadoop_job_id,))
 
     # Test single task page
-    late_task_id = hadoop_job_id.replace('job', 'task') + '_r_001999'
+    late_task_id = hadoop_job_id.replace('job', 'task') + '_r_000000'
     response = self.client.get('/jobbrowser/jobs/%s/tasks/%s' % (hadoop_job_id, late_task_id))
     assert_false('succeed' in response.content)
     assert_true('killed' in response.content)
@@ -282,14 +254,84 @@ class TestJobBrowserWithHadoop(object):
     assert_false('failed' in response.content)
 
     # Test single attempt page
+    early_task_id = hadoop_job_id.replace('job', 'task') + '_m_000000'
     attempt_id = early_task_id.replace('task', 'attempt') + '_0'
-    response = self.client.get('/jobbrowser/jobs/%s/tasks/%s/attempts/%s' %
+    response = self.client.get('/jobbrowser/jobs/%s/tasks/%s/attempts/%s/logs' %
                           (hadoop_job_id, early_task_id, attempt_id))
     assert_true('syslog' in response.content)
 
     # Test dock jobs
     response = self.client.get('/jobbrowser/dock_jobs/')
-    assert_true('completed' in response.content)
-    # TODO(atm): I'm pretty sure the following test only passes because of
-    # failed jobs which are run in test_failed_jobs
-    assert_true('failed' in response.content)
+    assert_false('completed' in response.content)
+    assert_false('failed' in response.content)
+
+  def test_job(self):
+    """
+    Test new job views.
+
+    The status of the jobs should be the same as the status reported back by oozie.
+    In this case, all jobs should succeed.
+    """
+    # Clone design
+    assert_equal(0, OozieDesign.objects.filter(owner__username=self.username).count())
+    self.client.post('/jobsub/clone_design/%d' % self.sleep_design_id)
+    assert_equal(1, OozieDesign.objects.filter(owner__username=self.username).count())
+
+    # Run the sleep example, since it doesn't require user home directory
+    design_id = OozieDesign.objects.get(owner__username=self.username).id
+    response = self.client.post("/jobsub/submit_design/%d" % (design_id,),
+      dict(map_sleep_time=1,
+           num_maps=1,
+           num_reduces=1,
+           reduce_sleep_time=1),
+      follow=True)
+    oozie_jobid = response.context['jobid']
+    job = OozieServerProvider.wait_until_completion(oozie_jobid, timeout=120, step=1)
+    hadoop_job_id = get_hadoop_job_id(self.oozie, oozie_jobid, 1)
+
+    # All jobs page and fetch job ID
+    # Taking advantage of the fact new jobs are at the top of the list!
+    response = self.client.get('/jobbrowser/jobs/')
+    assert_true(hadoop_job_id in response.content)
+
+    # Make sure job succeeded
+    response = self.client.get('/jobbrowser/jobs/?state=completed')
+    assert_true(hadoop_job_id in response.content)
+    response = self.client.get('/jobbrowser/jobs/?state=failed')
+    assert_false(hadoop_job_id in response.content)
+    response = self.client.get('/jobbrowser/jobs/?state=running')
+    assert_false(hadoop_job_id in response.content)
+    response = self.client.get('/jobbrowser/jobs/?state=killed')
+    assert_false(hadoop_job_id in response.content)
+
+    # Single job page
+    response = self.client.get('/jobbrowser/jobs/%s' % hadoop_job_id)
+
+    # Check some counters for single job.
+    counters = response.context['job'].counters
+    counters_file_bytes_written = counters['org.apache.hadoop.mapreduce.FileSystemCounter']['counters']['FILE_BYTES_WRITTEN']
+    assert_true(counters_file_bytes_written['map'] > 0)
+    assert_true(counters_file_bytes_written['reduce'] > 0)
+
+    # We can't just check the complete contents of the python map because the
+    # SLOTS_MILLIS_* entries have a variable number of milliseconds from
+    # run-to-run.
+    assert_equal(response.context['job'].counters['org.apache.hadoop.mapreduce.JobCounter']['counters']['TOTAL_LAUNCHED_MAPS']['total'], 1)
+    assert_equal(response.context['job'].counters['org.apache.hadoop.mapreduce.JobCounter']['counters']['TOTAL_LAUNCHED_REDUCES']['total'], 1)
+    assert_equal(response.context['job'].counters['org.apache.hadoop.mapreduce.JobCounter']['counters']['FALLOW_SLOTS_MILLIS_MAPS']['total'], 0)
+    assert_equal(response.context['job'].counters['org.apache.hadoop.mapreduce.JobCounter']['counters']['FALLOW_SLOTS_MILLIS_REDUCES']['total'], 0)
+    assert_true(response.context['job'].counters['org.apache.hadoop.mapreduce.JobCounter']['counters']['SLOTS_MILLIS_MAPS']['total'] > 0)
+    assert_true(response.context['job'].counters['org.apache.hadoop.mapreduce.JobCounter']['counters']['SLOTS_MILLIS_REDUCES']['total'] > 0)
+
+    # There should be 4 tasks for this job: cleanup, setup, map, reduce
+    response = self.client.get('/jobbrowser/jobs/%s/tasks' % (hadoop_job_id,))
+    assert_true(len(response.context['page'].object_list), 4)
+    # Select by tasktype
+    response = self.client.get('/jobbrowser/jobs/%s/tasks?tasktype=reduce' % (hadoop_job_id,))
+    assert_true(len(response.context['page'].object_list), 1)
+    # Select by taskstate
+    response = self.client.get('/jobbrowser/jobs/%s/tasks?taskstate=succeeded' % (hadoop_job_id,))
+    assert_true(len(response.context['page'].object_list), 4)
+    # Select by text
+    response = self.client.get('/jobbrowser/jobs/%s/tasks?tasktext=clean' % (hadoop_job_id,))
+    assert_true(len(response.context['page'].object_list), 1)

+ 31 - 21
apps/jobsub/src/jobsub/tests.py

@@ -16,6 +16,7 @@
 # limitations under the License.
 
 import copy
+import logging
 
 from nose.tools import assert_true, assert_false, assert_equal, assert_raises
 from django.contrib.auth.models import User
@@ -30,6 +31,9 @@ from jobsub.models import OozieDesign, OozieMapreduceAction, OozieStreamingActio
 from jobsub.parameterization import recursive_walk, find_variables, substitute_variables
 
 
+LOG = logging.getLogger(__name__)
+
+
 def test_recursive_walk():
   def f(_):
     f.leafs += 1
@@ -180,15 +184,18 @@ class TestJobsubWithHadoop(OozieServerProvider):
 
   def setUp(self):
     OozieServerProvider.setup_class()
-    self.cluster.fs.do_as_user('test', self.cluster.fs.create_home_dir, '/user/test')
-    self.cluster.fs.do_as_superuser(self.cluster.fs.chmod, '/user/test', 0777, True)
-    self.client = make_logged_in_client()
+    self.cluster.fs.do_as_user('test', self.cluster.fs.create_home_dir, '/user/jobsub_test')
+    self.cluster.fs.do_as_superuser(self.cluster.fs.chmod, '/user/jobsub_test', 0777, True)
+    self.client = make_logged_in_client(username='jobsub_test')
+
+    # Ensure access to MR folder
+    self.cluster.fs.do_as_superuser(self.cluster.fs.chmod, '/tmp', 0777, recursive=True)
 
   def test_jobsub_setup(self):
     # User 'test' triggers the setup of the examples.
     # 'hue' home will be deleted, the examples installed in the new one
     # and 'test' will try to access them.
-    self.cluster.fs.setuser('test')
+    self.cluster.fs.setuser('jobsub_test')
 
     username = 'hue'
     home_dir = '/user/%s/' % username
@@ -196,15 +203,15 @@ class TestJobsubWithHadoop(OozieServerProvider):
 
     try:
       data_dir = conf.REMOTE_DATA_DIR.get()
-      self.cluster.fs.setuser(self.cluster.fs.superuser)
-      if self.cluster.fs.exists(home_dir):
-        self.cluster.fs.rmtree(home_dir)
-      self.cluster.fs.setuser('test')
 
       if not jobsub_setup.Command().has_been_setup():
+        self.cluster.fs.setuser(self.cluster.fs.superuser)
+        if self.cluster.fs.exists(home_dir):
+          self.cluster.fs.rmtree(home_dir)
+
         jobsub_setup.Command().handle()
 
-      self.cluster.fs.setuser('test')
+      self.cluster.fs.setuser('jobsub_test')
       stats = self.cluster.fs.stats(home_dir)
       assert_equal(stats['user'], username)
       assert_equal(oct(stats['mode']), '040755') #04 because is a dir
@@ -213,8 +220,10 @@ class TestJobsubWithHadoop(OozieServerProvider):
       assert_equal(stats['user'], username)
       assert_equal(oct(stats['mode']), '041777')
 
+      # Only examples should have been created by 'hue'
       stats = self.cluster.fs.listdir_stats(data_dir)
-      assert_equal(len(stats), 2)
+      sample_stats = filter(lambda stat: stat.user == username, stats)
+      assert_equal(len(sample_stats), 2)
     finally:
       finish()
 
@@ -224,21 +233,22 @@ class TestJobsubWithHadoop(OozieServerProvider):
     """
     if not jobsub_setup.Command().has_been_setup():
       jobsub_setup.Command().handle()
+    self.cluster.fs.setuser('jobsub_test')
 
-    assert_equal(3, OozieDesign.objects.count())
-    assert_equal(2, OozieMapreduceAction.objects.count())
-    assert_equal(1, OozieStreamingAction.objects.count())
+    assert_equal(3, OozieDesign.objects.filter(owner__username='sample').count())
+    assert_equal(2, OozieMapreduceAction.objects.filter(ooziedesign__owner__username='sample').count())
+    assert_equal(1, OozieStreamingAction.objects.filter(ooziedesign__owner__username='sample').count())
 
     # Make sure sample user got created.
     assert_equal(1, User.objects.filter(username='sample').count())
 
     # Clone design
-    assert_equal(0, OozieDesign.objects.filter(owner__username='test').count())
-    jobid = OozieDesign.objects.get(name='sleep_job').id
+    assert_equal(0, OozieDesign.objects.filter(owner__username='jobsub_test').count())
+    jobid = OozieDesign.objects.get(name='sleep_job', owner__username='sample').id
 
     self.client.post('/jobsub/clone_design/%d' % jobid)
-    assert_equal(1, OozieDesign.objects.filter(owner__username='test').count())
-    jobid = OozieDesign.objects.get(owner__username='test').id
+    assert_equal(1, OozieDesign.objects.filter(owner__username='jobsub_test').count())
+    jobid = OozieDesign.objects.get(owner__username='jobsub_test').id
 
     # And now submit and run the sleep sample
     response = self.client.post('/jobsub/submit_design/%d' % jobid, {
@@ -251,19 +261,19 @@ class TestJobsubWithHadoop(OozieServerProvider):
     assert_true(str(jobid) in response.content)
 
     oozie_job_id = response.context['jobid']
-    job = OozieServerProvider.wait_until_completion(oozie_job_id, timeout=60, step=1)
+    job = OozieServerProvider.wait_until_completion(oozie_job_id, timeout=120, step=1)
     logs = OozieServerProvider.oozie.get_job_log(oozie_job_id)
 
     assert_equal('SUCCEEDED', job.status, logs)
 
 
     # Grep
-    n = OozieDesign.objects.filter(owner__username='test').count()
+    n = OozieDesign.objects.filter(owner__username='jobsub_test').count()
     jobid = OozieDesign.objects.get(name='grep_example').id
 
     self.client.post('/jobsub/clone_design/%d' % jobid)
-    assert_equal(n + 1, OozieDesign.objects.filter(owner__username='test').count())
-    jobid = OozieDesign.objects.get(owner__username='test', name__contains='sleep_job').id
+    assert_equal(n + 1, OozieDesign.objects.filter(owner__username='jobsub_test').count())
+    jobid = OozieDesign.objects.get(owner__username='jobsub_test', name__contains='sleep_job').id
 
     # And now submit and run the sleep sample
     response = self.client.post('/jobsub/submit_design/%d' % jobid, {

+ 3 - 0
apps/oozie/src/oozie/tests.py

@@ -147,6 +147,9 @@ class OozieBase(OozieServerProvider):
     self.cluster = OozieServerProvider.cluster
     self.install_examples()
 
+    # Ensure access to MR folder
+    self.cluster.fs.do_as_superuser(self.cluster.fs.chmod, '/tmp', 0777, recursive=True)
+
 
   def install_examples(self):
     global _INITIALIZED

+ 0 - 1
desktop/libs/liboozie/src/liboozie/oozie_api_test.py

@@ -54,7 +54,6 @@ class OozieServerProvider(object):
     cls.shutdown = [callback]
 
   @classmethod
-
   def wait_until_completion(cls, oozie_jobid, timeout=300.0, step=5):
     job = cls.oozie.get_job(oozie_jobid)
     start = time.time()