Browse Source

[hadoop] Fix unit test for webhdfs

* Moved hadoopfs_test.pyto test_webhdfs.py.
* Create webhdfs file when mode == 'w'.
* Add config for MR1 location
* Fix minicluster unit test
* Minicluster runs against MR1. Updated several Hadoop configs that have
  changed names.
* Added config validator for webhdfs.
* Modified cluster middleware to look only at hdfs filesystems (instead of
  defaulting to the local filesystem).
bc Wong 13 years ago
parent
commit
9bf65752a5

+ 1 - 1
desktop/core/src/desktop/lib/conf.py

@@ -615,7 +615,7 @@ def validate_path(confvar, is_dir=None):
   @return [(confvar, error_msg)] or []
   """
   path = confvar.get()
-  if not os.path.exists(path):
+  if path is None or not os.path.exists(path):
     return [(confvar, 'Path does not exist on local filesystem.')]
   if is_dir is not None:
     if is_dir:

+ 6 - 10
desktop/core/src/desktop/middleware.py

@@ -30,7 +30,7 @@ import django.views.generic.simple
 import django.contrib.auth.views
 
 import desktop.conf
-from desktop.lib import apputil, fsmanager
+from desktop.lib import apputil
 from desktop.lib.django_util import render_json, is_jframe_request, PopupException
 from desktop.log.access import access_log, log_page_hit
 from desktop import appmanager
@@ -119,18 +119,14 @@ class ClusterMiddleware(object):
     """
     has_hadoop = apputil.has_hadoop()
 
-    fs_ref = request.GET.get('fs', request.POST.get('fs', view_kwargs.get('fs')))
+    request.fs_ref = request.REQUEST.get('fs', view_kwargs.get('fs', 'default'))
     if "fs" in view_kwargs:
       del view_kwargs["fs"]
 
-    if fs_ref is None:
-      request.fs_ref, request.fs = fsmanager.get_default_hdfs()
-    else:
-      try:
-        request.fs = fsmanager.get_filesystem(fs_ref)
-        request.fs_ref = fs_ref
-      except KeyError:
-        raise KeyError('Cannot find filesystem called "%s"' % (fs_ref,))
+    try:
+      request.fs = cluster.get_hdfs(request.fs_ref)
+    except KeyError:
+      raise KeyError('Cannot find HDFS called "%s"' % (request.fs_ref,))
 
     if request.user.is_authenticated() and request.fs is not None:
       request.fs.setuser(request.user.username)

+ 39 - 39
desktop/libs/hadoop/src/hadoop/conf.py

@@ -16,20 +16,28 @@
 # limitations under the License.
 """Settings to configure your Hadoop cluster."""
 from desktop.lib.conf import Config, UnspecifiedConfigSection, ConfigSection, validate_path, coerce_bool
-import glob
-import os
+import fnmatch
 import logging
+import os
 
 HADOOP_HOME = Config(
   key="hadoop_home",
-  default=os.environ.get("HADOOP_HOME", "/usr/lib/hadoop-0.20"),
+  default=os.environ.get("HADOOP_HOME", "/usr/lib/hadoop"),
+  help=("Path to directory holding hadoop libs - HADOOP_HOME in " +
+        "hadoop parlance; defaults to environment variable, when" +
+        "set.")
+)
+
+HADOOP_MR1_HOME = Config(
+  key="hadoop_mr1_home",
+  default=os.environ.get("HADOOP_MR1_HOME", "/usr/lib/hadoop-0.20-mapreduce"),
   help=("Path to directory holding hadoop libs - HADOOP_HOME in " +
         "hadoop parlance; defaults to environment variable, when" +
         "set.")
 )
 
 def hadoop_bin_from_hadoop_home():
-  """Returns $HADOOP_HOME/bin/hadoop-0.20"""
+  """Returns $HADOOP_HOME/bin/hadoop"""
   return os.path.join(HADOOP_HOME.get(), "bin/hadoop")
 
 HADOOP_BIN = Config("hadoop_bin",
@@ -44,21 +52,24 @@ HADOOP_CONF_DIR = Config(
   help="If set, directory to pass to hadoop_bin (from hadoop configuration) as the --config flag.",
 )
 
-def find_jar(desired_glob, root=None):
+def find_file_recursive(desired_glob, root=None):
   if root is None:
     root_f = lambda: HADOOP_HOME.get()
   else:
-    root_f = lambda: root
+    root_f = lambda: not callable(root) and root or root()
+
   def f():
-    pattern = os.path.join(root_f(), desired_glob)
-    possibilities = glob.glob(pattern)
-    if len(possibilities) == 0:
-      logging.error("Trouble finding jars matching %s" % (pattern,))
-      return None
-    else:
-      if len(possibilities) != 1:
-        logging.warning("Found multiple jars matching %s: %s" % (pattern, possibilities))
-      return possibilities[0]
+    for dirpath, dirnames, filenames in os.walk(root_f()):
+      matches = fnmatch.filter(filenames, desired_glob)
+      if matches:
+        if len(matches) != 1:
+          logging.warning("Found multiple jars matching %s: %s" %
+                          (desired_glob, matches))
+        return os.path.join(dirpath, matches[0])
+
+    logging.error("Trouble finding jars matching %s" % (desired_glob,))
+    return None
+
   if root is None:
     root_str = "$HADOOP_HOME"
   else:
@@ -66,56 +77,45 @@ def find_jar(desired_glob, root=None):
   f.__doc__ = "Finds %s/%s" % (root_str, desired_glob)
   return f
 
-def find_examples_jar():
-  """
-  Finds $HADOOP_HOME/hadoop-*examples*.jar
-  """
-  return find_jar("hadoop-*examples*.jar")
-
 HADOOP_EXAMPLES_JAR = Config(
   key="hadoop_examples_jar",
-  dynamic_default=find_examples_jar(),
+  dynamic_default=find_file_recursive("hadoop-*examples*.jar", lambda: HADOOP_MR1_HOME.get()),
   help="Path to the hadoop-examples.jar (used for tests and jobdesigner setup)",
   type=str,
   private=True)
 
 HADOOP_STREAMING_JAR = Config(
   key="hadoop_streaming_jar",
-  dynamic_default=find_jar(os.path.join("contrib", "streaming", "hadoop-*streaming*.jar")),
+  dynamic_default=find_file_recursive("hadoop-*streaming*.jar"),
   help="Path to the hadoop-streaming.jar (used by jobdesigner)",
   type=str,
   private=True)
 
 HADOOP_TEST_JAR = Config("hadoop_test_jar",
   help="[Used by testing code.] Path to hadoop-test.jar",
-  dynamic_default=find_jar("hadoop-*test*.jar"),
+  dynamic_default=find_file_recursive("hadoop-*test*.jar", lambda: HADOOP_MR1_HOME.get()),
   type=str,
   private=True)
 
 HADOOP_PLUGIN_CLASSPATH = Config("hadoop_plugin_classpath",
   help="[Used only in testing code.] Path to the Hadoop plugin jar.",
   type=str,
-  dynamic_default=find_jar("../../java-lib/hue-plugins-*.jar", root=os.path.dirname(__file__)),
-  private=True)
-
-HADOOP_EXTRA_CLASSPATH_STRING = Config('hadoop_extra_classpath_entries',
-  help='[Used only in testing code.] String to add to the end of the HADOOP_CLASSPATH environment variable when calling bin/hadoop.',
-  type=str,
-  dynamic_default=find_jar("../../static-group-mapping/java-lib/static-group-mapping-*.jar", root=os.path.dirname(__file__)),
+  dynamic_default=find_file_recursive("hue-plugins-*.jar",
+                root=os.path.join(os.path.dirname(__file__), '..', '..', 'java-lib')),
   private=True)
 
 SUDO_SHELL_JAR = Config("hadoop_sudo_shell_jar",
   help="Tool that allows a proxy user UGI to be used to upload files.",
   type=str,
-  dynamic_default=find_jar("../../sudo-shell/java-lib/sudo-shell-*.jar",
-                           root=os.path.dirname(__file__)),
+  dynamic_default=find_file_recursive("sudo-shell-*.jar",
+                root=os.path.join(os.path.dirname(__file__), '..', '..', 'sudo-shell', 'java-lib')),
   private=True)
 
 CREDENTIALS_MERGER_JAR = Config("hadoop_credentials_merger_jar",
   help="Tool that is capable of merging multiple files containing delegation tokens into one.",
   type=str,
-  dynamic_default=find_jar("../../credentials-merger/java-lib/credentials-merger-*.jar",
-                           root=os.path.dirname(__file__)),
+  dynamic_default=find_file_recursive("credentials-merger-*.jar",
+                root=os.path.join(os.path.dirname(__file__), '..', '..', 'credentials-merger', 'java-lib')),
   private=True)
 
 HDFS_CLUSTERS = UnspecifiedConfigSection(
@@ -133,8 +133,8 @@ HDFS_CLUSTERS = UnspecifiedConfigSection(
                             type=int),
       WEBHDFS_URL=Config("webhdfs_url",
                          help="The URL to WebHDFS/HttpFs service. Defaults to " +
-                         "the WebHDFS URL on the NameNode. To use the legacy" +
-                         "Thrift plugin communication mechanism, this must be" +
+                         "the WebHDFS URL on the NameNode. To use the legacy " +
+                         "Thrift plugin communication mechanism, this must be " +
                          "set to an empty value.",
                          type=str, default=None),
       NN_KERBEROS_PRINCIPAL=Config("nn_kerberos_principal", help="Kerberos principal for NameNode",
@@ -179,7 +179,7 @@ def config_validator():
 
   Called by core check_config() view.
   """
-  from hadoop.fs import hadoopfs
+  from hadoop.fs import webhdfs
   from hadoop import job_tracker
   res = [ ]
 
@@ -195,7 +195,7 @@ def config_validator():
   # HDFS_CLUSTERS
   for name in HDFS_CLUSTERS.keys():
     cluster = HDFS_CLUSTERS[name]
-    res.extend(hadoopfs.test_fs_configuration(cluster, HADOOP_BIN))
+    res.extend(webhdfs.test_fs_configuration(cluster))
 
   # MR_CLUSTERS
   for name in MR_CLUSTERS.keys():

+ 22 - 15
desktop/libs/hadoop/src/hadoop/fs/hadoopfs.py

@@ -192,6 +192,27 @@ class Hdfs(object):
       return res[1:]
     return res
 
+  @staticmethod
+  def urlsplit(url):
+    """
+    Take an HDFS path (hdfs://nn:port/foo) or just (/foo) and split it into
+    the standard urlsplit's 5-tuple.
+    """
+    i = url.find('://')
+    if i == -1:
+      # Not found. Treat the entire argument as an HDFS path
+      return ('hdfs', '', normpath(url), '', '')
+    if url[:i] != 'hdfs':
+      # Default to standard for non-hdfs
+      return urlparse.urlsplit(url)
+    url = url[i+3:]
+    i = url.find('/')
+    if i == -1:
+      # Everything is netloc. Assume path is root.
+      return ('hdfs', url, '/', '', '')
+    netloc = url[:i]
+    path = url[i:]
+    return ('hdfs', netloc, normpath(path), '', '')
 
 class HadoopFileSystem(Hdfs):
   """
@@ -662,21 +683,7 @@ class HadoopFileSystem(Hdfs):
     Take an HDFS path (hdfs://nn:port/foo) or just (/foo) and split it into
     the standard urlsplit's 5-tuple.
     """
-    i = url.find('://')
-    if i == -1:
-      # Not found. Treat the entire argument as an HDFS path
-      return ('hdfs', '', normpath(url), '', '')
-    if url[:i] != 'hdfs':
-      # Default to standard for non-hdfs
-      return urlparse.urlsplit(url)
-    url = url[i+3:]
-    i = url.find('/')
-    if i == -1:
-      # Everything is netloc. Assume path is root.
-      return ('hdfs', url, '/', '', '')
-    netloc = url[:i]
-    path = url[i:]
-    return ('hdfs', netloc, normpath(path), '', '')
+    return Hdfs.urlsplit(url)
 
 
 def require_open(func):

+ 0 - 368
desktop/libs/hadoop/src/hadoop/fs/hadoopfs_test.py

@@ -1,368 +0,0 @@
-#!/usr/bin/env python
-# -*- coding: utf-8 -*-
-# Licensed to Cloudera, Inc. under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  Cloudera, Inc. 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.
-"""
-Tests for Hadoop FS.
-"""
-from nose.tools import assert_false, assert_true, assert_equals, assert_raises
-from nose.plugins.attrib import attr
-import logging
-import posixfile
-import random
-from threading import Thread
-
-from hadoop import mini_cluster
-from hadoop.fs.exceptions import PermissionDeniedException
-from hadoop.fs.hadoopfs import HadoopFileSystem
-
-LOG = logging.getLogger(__name__)
-
-@attr('requires_hadoop')
-def test_hadoopfs():
-  """
-  Minimal tests for a few basic file system operations.
-  """
-  cluster = mini_cluster.shared_cluster()
-  try:
-    fs = cluster.fs
-    fs.setuser(cluster.superuser)
-    f = fs.open("/fortest.txt", "w")
-    try:
-      f.write("hello")
-      f.close()
-      assert_equals("hello", fs.open("/fortest.txt").read())
-      assert_equals(5, fs.stats("/fortest.txt")["size"])
-      assert_true(fs.isfile("/fortest.txt"))
-      assert_false(fs.isfile("/"))
-      assert_true(fs.isdir("/"))
-      assert_false(fs.isdir("/fortest.txt"))
-    finally:
-      fs.remove("/fortest.txt")
-  finally:
-    cluster.shutdown()
-
-@attr('requires_hadoop')
-def test_seek():
-  """Test for DESKTOP-293 - ensure seek works in python2.4"""
-  cluster = mini_cluster.shared_cluster()
-  try:
-    fs = cluster.fs
-    fs.setuser(cluster.superuser)
-    f = fs.open("/fortest.txt", "w")
-    try:
-      f.write("hello")
-      f.close()
-
-      f = fs.open("/fortest.txt", "r")
-      f.seek(0, posixfile.SEEK_SET)
-      assert_equals("he", f.read(2))
-      f.seek(1, posixfile.SEEK_SET)
-      assert_equals("el", f.read(2))
-      f.seek(-1, posixfile.SEEK_END)
-      assert_equals("o", f.read())
-      f.seek(0, posixfile.SEEK_SET)
-      f.seek(2, posixfile.SEEK_CUR)
-      assert_equals("ll", f.read(2))
-    finally:
-      fs.remove("/fortest.txt")
-  finally:
-    cluster.shutdown()
-
-@attr('requires_hadoop')
-def test_seek_across_blocks():
-  """Makes a file with a lot of blocks, seeks around"""
-  cluster = mini_cluster.shared_cluster()
-  try:
-    fs = cluster.fs
-    fs.setuser(cluster.superuser)
-    f = fs.open("/fortest-blocks.txt", "w", block_size=1024)
-    try:
-      data = "abcdefghijklmnopqrstuvwxyz" * 3000
-      f.write(data)
-      f.close()
-
-      for i in xrange(1, 10):
-        f = fs.open("/fortest-blocks.txt", "r")
-
-        for j in xrange(1, 100):
-          offset = random.randint(0, len(data) - 1)
-          f.seek(offset, posixfile.SEEK_SET)
-          assert_equals(data[offset:offset+50], f.read(50))
-        f.close()
-
-    finally:
-      fs.remove("/fortest-blocks.txt")
-  finally:
-    cluster.shutdown()
-
-
-@attr('requires_hadoop')
-def test_exceptions():
-  """
-  Tests that appropriate exceptions are raised.
-  """
-  cluster = mini_cluster.shared_cluster()
-  try:
-    fs = cluster.fs
-    fs.setuser(cluster.superuser)
-    f = fs.open("/for_exception_test.txt", "w")
-    f.write("foo")
-    f.close()
-    fs.chmod("/for_exception_test.txt", 0400)
-    fs.setuser("notsuperuser")
-    f = fs.open("/for_exception_test.txt")
-    # Arguably, this should have thrown already, at open, but
-    # we throw the exception lazily, when getting block locations.
-    assert_raises(PermissionDeniedException, f.read)
-
-    assert_raises(IOError, fs.open, "/test/doesnotexist.txt")
-  finally:
-    cluster.shutdown()
-
-@attr('requires_hadoop')
-def test_two_files_open():
-  """
-  See DESKTOP-510.  There was a bug where you couldn't open two files at
-  the same time.  It boils down to a close_fds=True issue.  If this doesn't
-  hang, all is good.
-  """
-  cluster = mini_cluster.shared_cluster()
-  try:
-    fs = cluster.fs
-    fs.setuser(cluster.superuser)
-    f1 = fs.open("/test_one.txt", "w")
-    f2 = fs.open("/test_two.txt", "w")
-    f1.write("foo")
-    f2.write("bar")
-    f1.close()
-    f2.close()
-    # This should work, not hang, etc.
-  finally:
-    cluster.shutdown()
-
-
-def test_urlsplit():
-  """Test HadoopFS urlsplit"""
-  url = 'hdfs://nn.no.port/foo/bar'
-  assert_equals(('hdfs', 'nn.no.port', '/foo/bar', '', ''), HadoopFileSystem.urlsplit(url))
-  url = 'hdfs://nn:8020/foo/bar'
-  assert_equals(('hdfs', 'nn:8020', '/foo/bar', '', ''), HadoopFileSystem.urlsplit(url))
-  url = 'hdfs://nn:8020//foo//bar'
-  assert_equals(('hdfs', 'nn:8020', '/foo/bar', '', ''), HadoopFileSystem.urlsplit(url))
-  url = 'hdfs://nn:8020'
-  assert_equals(('hdfs', 'nn:8020', '/', '', ''), HadoopFileSystem.urlsplit(url))
-  url = '/foo/bar'
-  assert_equals(('hdfs', '', '/foo/bar', '', ''), HadoopFileSystem.urlsplit(url))
-  url = 'foo//bar'
-  assert_equals(('hdfs', '', 'foo/bar', '', ''), HadoopFileSystem.urlsplit(url))
-
-@attr('requires_hadoop')
-def test_quota_argument_smarts():
-  """
-  Test invalid quota parameters
-  """
-  cluster = mini_cluster.shared_cluster()
-  fs = cluster.fs
-  try:
-    fs.setuser(cluster.superuser)
-    fs.mkdir("/tmp/foo2", 0777)
-
-    fs.set_diskspace_quota("/tmp/foo2", 1)
-    fs.set_namespace_quota("/tmp/foo2", 1)
-
-    assert_raises(ValueError, fs.set_diskspace_quota, "/tmp/foo2", -5)
-
-    assert_raises(ValueError, fs.set_diskspace_quota, '/', 10)
-    assert_raises(ValueError, fs.set_namespace_quota, '/', 10)
-
-    fs.set_diskspace_quota("/tmp/foo2", 1.1) # This should actually fail i think
-  finally:
-    fs.rmtree("/tmp/foo2")
-    cluster.shutdown()
-
-@attr('requires_hadoop')
-def test_quota_space():
-  """
-  Lets make sure we can violate the quota in regards to diskspace
-  """
-
-  cluster = mini_cluster.shared_cluster()
-  fs = cluster.fs
-  try:
-    fs.setuser(cluster.superuser)
-    if fs.exists('/tmp/foo2'):
-      fs.rmtree('/tmp/foo2')
-    fs.mkdir("/tmp/foo2", 0777) # this also tests more restrictive subdirectories
-    ONE_HUNDRED_192_MEGS = 1024 * 1024 * 192
-
-    fs.set_diskspace_quota("/tmp/foo2", ONE_HUNDRED_192_MEGS)
-    assert_equals(fs.get_diskspace_quota("/tmp/foo2"), ONE_HUNDRED_192_MEGS)
-
-    f = fs.open('/tmp/foo2/asdf', 'w')	 # we should be able to do this
-    f.write('a')
-    f.close()
-
-    assert_equals(fs.get_diskspace_quota("/tmp/foo2"), ONE_HUNDRED_192_MEGS)
-
-    fs.set_diskspace_quota("/tmp/foo2", 1)
-    assert_equals(fs.get_diskspace_quota("/tmp/foo2"), 1)
-
-    f = fs.open('/tmp/foo2/asdfsd', 'w')
-    f.write('a')
-    assert_raises(IOError, f.close)
-
-    fs.clear_diskspace_quota("/tmp/foo2")
-    assert_equals(fs.get_diskspace_quota("/tmp/foo2"), None)
-
-    f = fs.open('/tmp/foo2/asdfsda', 'w')
-    f.write('a')
-    f.close()
-
-    fs.mkdir("/tmp/baz/bar", 0777)  # this tests more permissive subdirectories
-    fs.set_diskspace_quota("/tmp/baz", 1)
-    fs.set_diskspace_quota("/tmp/baz/bar", ONE_HUNDRED_192_MEGS)
-
-    f = fs.open('/tmp/baz/bar', 'w')
-    f.write('aaaa') #should violate the subquota
-    assert_raises(IOError, f.close)
-
-  finally:
-    if fs.exists('/tmp/baz'):
-      fs.rmtree("/tmp/baz")
-    if fs.exists('/tmp/foo2'):
-      fs.rmtree("/tmp/foo2")
-    cluster.shutdown()
-
-@attr('requires_hadoop')
-def test_quota_namespace_count():
-  """
-  Lets make sure we can violate the number of names in a directory limitation
-  """
-  cluster = mini_cluster.shared_cluster()
-  try:
-    fs = cluster.fs
-    fs.setuser(cluster.superuser)
-    if fs.exists('/tmp/foo2'):
-      fs.rmtree('/tmp/foo2')
-    fs.mkdir("/tmp/foo2", 0777)
-
-    # check the get_namespace_quota function
-    fs.set_namespace_quota("/tmp/foo2", 4)
-    assert_equals(fs.get_namespace_quota("/tmp/foo2"), 4)
-
-    # violate the namespace count
-    for i in range(3):
-      f = fs.open('/tmp/foo2/works' + str(i), 'w')
-      f.write('a')
-      f.close()
-
-    f = fs.open('/tmp/foo2/asdfsdc', 'w')
-    f.write('a')
-    assert_raises(IOError, f.close)
-
-    # Check summary stats
-    summary = fs.get_usage_and_quota('/tmp/foo2')
-    assert_equals(3, summary["file_count"])
-    assert_equals(4, summary["file_quota"])
-    assert_equals(None, summary["space_quota"])
-    assert_true(None is not summary["space_used"])
-
-    # make sure the clear works
-    fs.clear_namespace_quota("/tmp/foo2")
-    assert_equals(fs.get_namespace_quota("/tmp/foo2"), None)
-
-    f = fs.open('/tmp/foo2/asdfsdd', 'w')
-    f.write('a')
-    f.close()
-  finally:
-    if fs.exists('/tmp/foo2'):
-      fs.rmtree("/tmp/foo2")
-    cluster.shutdown()
-
-
-@attr('requires_hadoop')
-def test_i18n_namespace():
-  cluster = mini_cluster.shared_cluster()
-  cluster.fs.setuser(cluster.superuser)
-
-  def check_existence(name, parent, present=True):
-    assertion = present and assert_true or assert_false
-    listing = cluster.fs.listdir(parent)
-    assertion(name in listing, "%s should be in %s" % (name, listing))
-
-  name = u'pt-Olá_ch-你好_ko-안녕_ru-Здравствуйте'
-  prefix = '/tmp/i18n'
-  dir_path = '%s/%s' % (prefix, name)
-  file_path = '%s/%s' % (dir_path, name)
-
-  try:
-    # Create a directory
-    cluster.fs.mkdir(dir_path)
-    # Directory is there
-    check_existence(name, prefix)
-
-    # Create a file (same name) in the directory
-    cluster.fs.open(file_path, 'w').close()
-    # File is there
-    check_existence(name, dir_path)
-
-    # Test rename
-    new_file_path = file_path + '.new'
-    cluster.fs.rename(file_path, new_file_path)
-    # New file is there
-    check_existence(name + '.new', dir_path)
-
-    # Test remove
-    cluster.fs.remove(new_file_path)
-    check_existence(name + '.new', dir_path, present=False)
-
-    # Test rmtree
-    cluster.fs.rmtree(dir_path)
-    check_existence(name, prefix, present=False)
-
-    # Test exception can handle non-ascii characters
-    try:
-      cluster.fs.rmtree(dir_path)
-    except IOError, ex:
-      LOG.info('Successfully caught error: %s' % (ex,))
-  finally:
-    try:
-      cluster.fs.rmtree(prefix)
-    except Exception, ex:
-      LOG.error('Failed to cleanup %s: %s' % (prefix, ex))
-    cluster.shutdown()
-
-@attr('requires_hadoop')
-def test_threadedness():
-  # Start a second thread to change the user, and
-  # make sure that isn't reflected.
-  cluster = mini_cluster.shared_cluster()
-  try:
-    fs = cluster.fs
-    fs.setuser("alpha")
-    class T(Thread):
-      def run(self):
-        fs.setuser("beta")
-        assert_equals("beta", fs.user)
-    t = T()
-    t.start()
-    t.join()
-    assert_equals("alpha", fs.user)
-    fs.setuser("gamma")
-    assert_equals("gamma", fs.user)
-  finally:
-    cluster.shutdown()

+ 225 - 0
desktop/libs/hadoop/src/hadoop/fs/test_webhdfs.py

@@ -0,0 +1,225 @@
+#!/usr/bin/env python
+# -*- coding: utf-8 -*-
+# Licensed to Cloudera, Inc. under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  Cloudera, Inc. 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.
+"""
+Tests for Hadoop FS.
+"""
+from nose.tools import assert_false, assert_true, assert_equals, assert_raises
+from nose.plugins.attrib import attr
+import logging
+import posixfile
+import random
+from threading import Thread
+
+from hadoop import pseudo_hdfs4
+from hadoop.fs.exceptions import WebHdfsException
+from hadoop.fs.hadoopfs import Hdfs
+
+LOG = logging.getLogger(__name__)
+
+@attr('requires_hadoop')
+def test_webhdfs():
+  """
+  Minimal tests for a few basic file system operations.
+  """
+  cluster = pseudo_hdfs4.shared_cluster()
+  fs = cluster.fs
+  fs.setuser(cluster.superuser)
+  f = fs.open("/fortest.txt", "w")
+  try:
+    f.write("hello")
+    f.close()
+    assert_equals("hello", fs.open("/fortest.txt").read())
+    assert_equals(5, fs.stats("/fortest.txt")["size"])
+    assert_true(fs.isfile("/fortest.txt"))
+    assert_false(fs.isfile("/"))
+    assert_true(fs.isdir("/"))
+    assert_false(fs.isdir("/fortest.txt"))
+  finally:
+    fs.remove("/fortest.txt")
+
+@attr('requires_hadoop')
+def test_seek():
+  """Test for DESKTOP-293 - ensure seek works in python2.4"""
+  cluster = pseudo_hdfs4.shared_cluster()
+  fs = cluster.fs
+  fs.setuser(cluster.superuser)
+  f = fs.open("/fortest.txt", "w")
+  try:
+    f.write("hello")
+    f.close()
+
+    f = fs.open("/fortest.txt", "r")
+    f.seek(0, posixfile.SEEK_SET)
+    assert_equals("he", f.read(2))
+    f.seek(1, posixfile.SEEK_SET)
+    assert_equals("el", f.read(2))
+    f.seek(-1, posixfile.SEEK_END)
+    assert_equals("o", f.read())
+    f.seek(0, posixfile.SEEK_SET)
+    f.seek(2, posixfile.SEEK_CUR)
+    assert_equals("ll", f.read(2))
+  finally:
+    fs.remove("/fortest.txt")
+
+@attr('requires_hadoop')
+def test_seek_across_blocks():
+  """Makes a file with a lot of blocks, seeks around"""
+  cluster = pseudo_hdfs4.shared_cluster()
+  fs = cluster.fs
+  fs.setuser(cluster.superuser)
+  fs.create("/fortest-blocks.txt", replication=1, blocksize=1024)
+  f = fs.open("/fortest-blocks.txt", "w")
+  try:
+    data = "abcdefghijklmnopqrstuvwxyz" * 3000
+    f.write(data)
+    f.close()
+
+    for i in xrange(1, 10):
+      f = fs.open("/fortest-blocks.txt", "r")
+
+      for j in xrange(1, 100):
+        offset = random.randint(0, len(data) - 1)
+        f.seek(offset, posixfile.SEEK_SET)
+        assert_equals(data[offset:offset+50], f.read(50))
+      f.close()
+
+  finally:
+    fs.remove("/fortest-blocks.txt")
+
+
+@attr('requires_hadoop')
+def test_exceptions():
+  """
+  Tests that appropriate exceptions are raised.
+  """
+  cluster = pseudo_hdfs4.shared_cluster()
+  fs = cluster.fs
+  fs.setuser(cluster.superuser)
+  f = fs.open("/for_exception_test.txt", "w")
+  f.write("foo")
+  f.close()
+  fs.chmod("/for_exception_test.txt", 0400)
+  fs.setuser("notsuperuser")
+  f = fs.open("/for_exception_test.txt")
+
+  assert_raises(WebHdfsException, f.read)
+  assert_raises(IOError, fs.open, "/test/doesnotexist.txt")
+
+@attr('requires_hadoop')
+def test_two_files_open():
+  """
+  See DESKTOP-510.  There was a bug where you couldn't open two files at
+  the same time.  It boils down to a close_fds=True issue.  If this doesn't
+  hang, all is good.
+  """
+  cluster = pseudo_hdfs4.shared_cluster()
+  fs = cluster.fs
+  fs.setuser(cluster.superuser)
+  f1 = fs.open("/test_one.txt", "w")
+  f2 = fs.open("/test_two.txt", "w")
+  f1.write("foo")
+  f2.write("bar")
+  f1.close()
+  f2.close()
+  # This should work, not hang, etc.
+
+
+def test_urlsplit():
+  """Test Hdfs urlsplit"""
+  url = 'hdfs://nn.no.port/foo/bar'
+  assert_equals(('hdfs', 'nn.no.port', '/foo/bar', '', ''), Hdfs.urlsplit(url))
+  url = 'hdfs://nn:8020/foo/bar'
+  assert_equals(('hdfs', 'nn:8020', '/foo/bar', '', ''), Hdfs.urlsplit(url))
+  url = 'hdfs://nn:8020//foo//bar'
+  assert_equals(('hdfs', 'nn:8020', '/foo/bar', '', ''), Hdfs.urlsplit(url))
+  url = 'hdfs://nn:8020'
+  assert_equals(('hdfs', 'nn:8020', '/', '', ''), Hdfs.urlsplit(url))
+  url = '/foo/bar'
+  assert_equals(('hdfs', '', '/foo/bar', '', ''), Hdfs.urlsplit(url))
+  url = 'foo//bar'
+  assert_equals(('hdfs', '', 'foo/bar', '', ''), Hdfs.urlsplit(url))
+
+
+@attr('requires_hadoop')
+def test_i18n_namespace():
+  cluster = pseudo_hdfs4.shared_cluster()
+  cluster.fs.setuser(cluster.superuser)
+
+  def check_existence(name, parent, present=True):
+    assertion = present and assert_true or assert_false
+    listing = cluster.fs.listdir(parent)
+    assertion(name in listing, "%s should be in %s" % (name, listing))
+
+  name = u'pt-Olá_ch-你好_ko-안녕_ru-Здравствуйте'
+  prefix = '/tmp/i18n'
+  dir_path = '%s/%s' % (prefix, name)
+  file_path = '%s/%s' % (dir_path, name)
+
+  try:
+    # Create a directory
+    cluster.fs.mkdir(dir_path)
+    # Directory is there
+    check_existence(name, prefix)
+
+    # Create a file (same name) in the directory
+    cluster.fs.open(file_path, 'w').close()
+    # File is there
+    check_existence(name, dir_path)
+
+    # Test rename
+    new_file_path = file_path + '.new'
+    cluster.fs.rename(file_path, new_file_path)
+    # New file is there
+    check_existence(name + '.new', dir_path)
+
+    # Test remove
+    cluster.fs.remove(new_file_path)
+    check_existence(name + '.new', dir_path, present=False)
+
+    # Test rmtree
+    cluster.fs.rmtree(dir_path)
+    check_existence(name, prefix, present=False)
+
+    # Test exception can handle non-ascii characters
+    try:
+      cluster.fs.rmtree(dir_path)
+    except IOError, ex:
+      LOG.info('Successfully caught error: %s' % (ex,))
+  finally:
+    try:
+      cluster.fs.rmtree(prefix)
+    except Exception, ex:
+      LOG.error('Failed to cleanup %s: %s' % (prefix, ex))
+
+@attr('requires_hadoop')
+def test_threadedness():
+  # Start a second thread to change the user, and
+  # make sure that isn't reflected.
+  cluster = pseudo_hdfs4.shared_cluster()
+  fs = cluster.fs
+  fs.setuser("alpha")
+  class T(Thread):
+    def run(self):
+      fs.setuser("beta")
+      assert_equals("beta", fs.user)
+  t = T()
+  t.start()
+  t.join()
+  assert_equals("alpha", fs.user)
+  fs.setuser("gamma")
+  assert_equals("gamma", fs.user)

+ 4 - 4
desktop/libs/hadoop/src/hadoop/fs/upload.py

@@ -31,8 +31,8 @@ import time
 
 from django.core.files.uploadhandler import \
     FileUploadHandler, StopFutureHandlers, StopUpload
-from desktop.lib import fsmanager
-import hadoop.fs.hadoopfs
+import hadoop.cluster
+import hadoop.fs.webhdfs
 
 UPLOAD_SUBDIR = 'hue-uploads'
 LOG = logging.getLogger(__name__)
@@ -53,7 +53,7 @@ class HDFStemporaryUploadedFile(object):
     try:
       self._fs = request.fs
     except AttributeError:
-      _, self._fs = fsmanager.get_default_hdfs()
+      self._fs = hadoop.cluster.get_hdfs()
 
     # Don't want to handle this upload if we don't have an HDFS
     if not self._fs:
@@ -62,7 +62,7 @@ class HDFStemporaryUploadedFile(object):
     # We want to set the user to be the superuser. But any operation
     # in the fs needs a username, including the retrieval of the superuser.
     # So we first set it to the DEFAULT_USER to break this chicken-&-egg.
-    self._fs.setuser(hadoop.fs.hadoopfs.DEFAULT_USER)
+    self._fs.setuser(hadoop.fs.webhdfs.DEFAULT_USER)
     self._fs.setuser(self._fs.superuser)
 
     self._path = self._fs.mktemp(

+ 58 - 13
desktop/libs/hadoop/src/hadoop/fs/webhdfs.py

@@ -94,7 +94,7 @@ class WebHdfs(Hdfs):
   
   @property
   def user(self):
-    return self.thread_local
+    return self._thread_local.user
 
   def _getparams(self):
     return { "user.name" : self._thread_local.user }
@@ -124,8 +124,8 @@ class WebHdfs(Hdfs):
 
     Get directory entry names without stats.
     """
-    dirents = self.listdir_stats(self, path, glob)
-    return [ x.path for x in dirents ]
+    dirents = self.listdir_stats(path, glob)
+    return [ Hdfs.basename(x.path) for x in dirents ]
 
   def get_content_summary(self, path):
     """
@@ -147,7 +147,7 @@ class WebHdfs(Hdfs):
       json = self._root.get(path, params)
       return WebHdfsStat(json['FileStatus'], path)
     except WebHdfsException, ex:
-      if ex.server_exc == 'FileNotFoundException':
+      if ex.server_exc == 'FileNotFoundException' or ex.code == 404:
         return None
       raise ex
 
@@ -248,6 +248,7 @@ class WebHdfs(Hdfs):
       raise IOError("Rename failed: %s -> %s" %
                     (smart_str(old), smart_str(new)))
 
+
   def chown(self, path, user=None, group=None):
     """chown(path, user=None, group=None)"""
     path = Hdfs.normpath(path)
@@ -333,6 +334,9 @@ class WebHdfs(Hdfs):
     params['op'] = 'APPEND'
     self._invoke_with_redirect('POST', path, params, data)
 
+  @staticmethod
+  def urlsplit(url):
+    return Hdfs.urlsplit(url)
 
   def _invoke_with_redirect(self, method, path, params=None, data=None):
     """
@@ -404,9 +408,11 @@ class File(object):
       if self._stat.isDir:
         raise IOError(errno.EISDIR, "Is a directory: '%s'" % (smart_str(path),))
     except IOError, ex:
-      if ex.errno == errno.ENOENT and mode == 'r':
+      if ex.errno == errno.ENOENT and 'w' in self._mode:
+        self._fs.create(self._path)
+        self.stat()
+      else:
         raise ex
-      self._stat = None
 
   def seek(self, offset, whence=0):
     """Set the file pointer to the given spot. @see file.seek"""
@@ -439,13 +445,7 @@ class File(object):
   def append(self, data):
     if 'w' not in self._mode:
       raise IOError(errno.EINVAL, "File not open for writing")
-
-    if self._stat is None:
-      # File not there yet.
-      self._fs.create(self._path, data=data)
-      self.stat()
-    else:
-      self._fs.append(self._path, data=data)
+    self._fs.append(self._path, data=data)
 
   def flush(self):
     pass
@@ -473,3 +473,48 @@ def _get_service_url(hdfs_config):
   host = hdfs_config.NN_HOST.get()
   port = hdfs_config.NN_HTTP_PORT.get()
   return "http://%s:%s/webhdfs/v1" % (host, port)
+
+
+
+def test_fs_configuration(fs_config):
+  """
+  This is a config validation method. Returns a list of
+    [ (config_variable, error_message) ]
+  """
+  fs = WebHdfs.from_config(fs_config)
+  fs.setuser(fs.superuser)
+
+  # Access root
+  try:
+    statbuf = fs.stats('/')
+  except Exception, ex:
+    LOG.info("%s -- Validation error: %s" % (fs, ex))
+    return [(fs_config.WEBHDFS_URL, 'Failed to access filesystem root')]
+
+  # Write a file
+  tmpname = fs.mktemp(prefix='hue_config_validation')
+  try:
+    fs.create(tmpname)
+  except Exception, ex:
+    LOG.info("%s -- Validation error: %s" % (fs, ex))
+    return [(fs_config.WEBHDFS_URL,
+            'Failed to create temporary file "%s"' % (tmpname,))]
+
+  # Check superuser has super power
+  try:  # Finally: delete tmpname
+    try:
+      fs.chown(tmpname, fs.superuser)
+    except Exception, ex:
+      LOG.info("%s -- Validation error: %s" % (fs, ex))
+      return [(fs_config.WEBHDFS_URL,
+              'Failed to chown file. Please make sure that the filesystem root '
+              'is owned by the cluster superuser ("hdfs" in most cases).')]
+  finally:
+    try:
+      fs.remove(tmpname)
+    except Exception, ex:
+      LOG.error("Failed to remove '%s': %s" % (tmpname, ex))
+      return [(fs_config.WEBHDFS_URL,
+              'Failed to remove temporary file "%s"' % (tmpname,))]
+
+  return [ ]

+ 10 - 20
desktop/libs/hadoop/src/hadoop/mini_cluster.py

@@ -46,7 +46,6 @@ import urllib2
 from hadoop.fs.hadoopfs import HadoopFileSystem
 from hadoop.job_tracker import LiveJobTracker
 import hadoop.cluster
-import desktop.lib.fsmanager
 
 # Starts mini cluster suspended until a debugger attaches to it.
 DEBUG_HADOOP=False
@@ -56,14 +55,8 @@ USE_STDERR=os.environ.get("MINI_CLUSTER_USE_STDERR", False)
 CLEANUP_TMP_DIR=os.environ.get("MINI_CLUSTER_CLEANUP", True)
 # How long to wait for cluster to start up.  (seconds)
 MAX_CLUSTER_STARTUP_TIME = 120.0
-# Class to use for the cluster's GMSP.
-CLUSTER_GMSP = 'org.apache.hadoop.security.StaticUserGroupMapping'
-# List of classes to be used as plugins for the NN of the cluster.
-CLUSTER_NN_PLUGINS = 'org.apache.hadoop.thriftfs.NamenodePlugin'
 # List of classes to be used as plugins for the JT of the cluster.
 CLUSTER_JT_PLUGINS = 'org.apache.hadoop.thriftfs.ThriftJobTrackerPlugin'
-# List of classes to be used as plugins for the DN(s) of the cluster.
-CLUSTER_DN_PLUGINS = 'org.apache.hadoop.thriftfs.DatanodePlugin'
 # MR Task Scheduler. By default use the FIFO scheduler
 CLUSTER_TASK_SCHEDULER='org.apache.hadoop.mapred.JobQueueTaskScheduler'
 # MR queue names
@@ -139,12 +132,9 @@ rpc.class=org.apache.hadoop.metrics.spi.NoEmitMetricsContext
     _write_static_group_mapping(TEST_USER_GROUP_MAPPING,
       tmppath('ugm.properties'))
 
-    core_configs = {'hadoop.security.group.mapping': CLUSTER_GMSP,
-      'hadoop.security.static.group.mapping.file': tmppath('ugm.properties'),
+    core_configs = {
       'hadoop.proxyuser.%s.groups' % (self.superuser,): 'users,supergroup',
       'hadoop.proxyuser.%s.hosts' % (self.superuser,): 'localhost',
-      'dfs.namenode.plugins': CLUSTER_NN_PLUGINS,
-      'dfs.datanode.plugins': CLUSTER_DN_PLUGINS,
       'mapred.jobtracker.plugins': CLUSTER_JT_PLUGINS}
 
     extra_configs.update(STARTUP_CONFIGS)
@@ -162,7 +152,8 @@ rpc.class=org.apache.hadoop.metrics.spi.NoEmitMetricsContext
 
     details_file = file(tmppath("details.json"), "w+")
     try:
-      args = [ hadoop.conf.HADOOP_BIN.get(), "jar",
+      args = [ os.path.join(hadoop.conf.HADOOP_MR1_HOME.get(), 'bin', 'hadoop'),
+        "jar",
         hadoop.conf.HADOOP_TEST_JAR.get(),
         "minicluster",
         "-writeConfig", tmppath("config.xml"), 
@@ -213,9 +204,11 @@ rpc.class=org.apache.hadoop.metrics.spi.NoEmitMetricsContext
         hadoop.conf.HADOOP_TEST_JAR.get(),
         # -- END JAVA TRIVIA --
         hadoop.conf.HADOOP_PLUGIN_CLASSPATH.get(),
-        hadoop.conf.HADOOP_EXTRA_CLASSPATH_STRING.get()])
+        # Due to CDH-4537, we need to add test dependencies to run minicluster
+        os.path.join(os.path.dirname(__file__), 'test_jars', '*'),
+      ])
       env["HADOOP_HEAPSIZE"] = "128"
-      env["HADOOP_HOME"] = hadoop.conf.HADOOP_HOME.get()
+      env["HADOOP_HOME"] = hadoop.conf.HADOOP_MR1_HOME.get()
       env["HADOOP_LOG_DIR"] = self.log_dir
       env["USER"] = self.superuser
       if "JAVA_HOME" in os.environ:
@@ -274,11 +267,10 @@ rpc.class=org.apache.hadoop.metrics.spi.NoEmitMetricsContext
     hadoop.conf.HADOOP_CONF_DIR.set_for_testing(self.config_dir)
 
     write_config(self.config, tmppath("conf/core-site.xml"), 
-      ["fs.default.name", "jobclient.completion.poll.interval",
-       "fs.checkpoint.period", "fs.checkpoint.dir",
-       'hadoop.security.group.mapping', 'hadoop.security.static.group.mapping.file',
+      ["fs.defaultFS", "jobclient.completion.poll.interval",
+       "dfs.namenode.checkpoint.period", "dfs.namenode.checkpoint.dir",
        'hadoop.proxyuser.'+self.superuser+'.groups', 'hadoop.proxyuser.'+self.superuser+'.hosts'])
-    write_config(self.config, tmppath("conf/hdfs-site.xml"), ["fs.default.name", "dfs.http.address", "dfs.secondary.http.address"])
+    write_config(self.config, tmppath("conf/hdfs-site.xml"), ["fs.defaultFS", "dfs.namenode.http-address", "dfs.namenode.secondary.http-address"])
     # mapred.job.tracker isn't written out into self.config, so we fill
     # that one out more manually.
     write_config({ 'mapred.job.tracker': 'localhost:%d' % self.jobtracker_port },
@@ -420,7 +412,6 @@ def shared_cluster(conf=False):
   if conf:
     closers.extend([
       hadoop.conf.HDFS_CLUSTERS["default"].NN_HOST.set_for_testing("localhost"),
-      hadoop.conf.HDFS_CLUSTERS["default"].NN_THRIFT_PORT.set_for_testing(cluster.fs.thrift_port),
       hadoop.conf.HDFS_CLUSTERS["default"].NN_HDFS_PORT.set_for_testing(cluster.namenode_port),
       hadoop.conf.MR_CLUSTERS["default"].JT_HOST.set_for_testing("localhost"),
       hadoop.conf.MR_CLUSTERS["default"].JT_THRIFT_PORT.set_for_testing(cluster.jt.thrift_port),
@@ -429,7 +420,6 @@ def shared_cluster(conf=False):
     # This is djanky (that's django for "janky").
     # Caches are tricky w.r.t. to to testing;
     # perhaps there are better patterns?
-    desktop.lib.fsmanager.reset()
     old = hadoop.cluster.clear_caches()
 
   def finish():

+ 18 - 10
desktop/libs/hadoop/src/hadoop/pseudo_hdfs4.py

@@ -40,7 +40,7 @@ class PseudoHdfs4(object):
   """This class runs HDFS (CDH4) locally, in pseudo-distributed mode"""
 
   def __init__(self):
-    self._tmpdir = tempfile.mkdtemp()
+    self._tmpdir = tempfile.mkdtemp(prefix='tmp_hue_')
     self._superuser = pwd.getpwuid(os.getuid()).pw_name
     self._fs = None
 
@@ -144,17 +144,18 @@ class PseudoHdfs4(object):
     # More stuff to setup in the environment
     env = dict(
       HADOOP_CONF_DIR = conf_dir,
-      HADOOP_CLASSPATH = ":".join([
-        hadoop.conf.HADOOP_EXTRA_CLASSPATH_STRING.get()]),
       HADOOP_HEAPSIZE = "128",
       HADOOP_LOG_DIR = self._log_dir,
       USER = self.superuser,
       LANG = "en_US.UTF-8",
+      PATH = os.environ['PATH'],
     )
 
     if "JAVA_HOME" in os.environ:
       env['JAVA_HOME'] = os.environ['JAVA_HOME']
 
+    LOG.debug("Environment:\n" + "\n".join([ str(x) for x in sorted(env.items()) ]))
+
     # Format HDFS
     self._format(conf_dir, env)
 
@@ -176,10 +177,20 @@ class PseudoHdfs4(object):
             'namenode', '-format')
     LOG.info('Formatting HDFS: %s' % (args,))
 
-    ignore = file('/dev/null', 'w+')
-    ret = subprocess.call(args, env=env, stdout=ignore, stderr=ignore)
-    if ret != 0:
-      raise RuntimeError('Failed to format namenode')
+    stdout = tempfile.TemporaryFile()
+    stderr = tempfile.TemporaryFile()
+    try:
+      ret = subprocess.call(args, env=env, stdout=stdout, stderr=stderr)
+      if ret != 0:
+        stdout.seek(0)
+        stderr.seek(0)
+        raise RuntimeError('Failed to format namenode\n'
+                           '=== Stdout ===:\n%s\n'
+                           '=== Stderr ===:\n%s' %
+                           (stdout.read(), stderr.read()))
+    finally:
+      stdout.close()
+      stderr.close()
 
 
   def _is_ready(self):
@@ -260,8 +271,6 @@ class PseudoHdfs4(object):
       'fs.default.name': self._fs_default_name,
       'hadoop.security.authorization': 'true',
       'hadoop.security.authentication': 'simple',
-      'hadoop.security.group.mapping': CLUSTER_GMSP,
-      'hadoop.security.static.group.mapping.file': ugm_properties,
       'hadoop.proxyuser.%s.groups' % (self.superuser,): 'users,supergroup',
       'hadoop.proxyuser.%s.hosts' % (self.superuser,): 'localhost',
       'hadoop.tmp.dir': self._tmppath('hadoop_tmp_dir'),
@@ -304,7 +313,6 @@ def shared_cluster():
       hadoop.conf.HDFS_CLUSTERS['default'].NN_HDFS_PORT.set_for_testing(cluster.namenode_port),
     ]
 
-    desktop.lib.fsmanager.reset()
     old = hadoop.cluster.clear_caches()
 
     def restore_config():

BIN
desktop/libs/hadoop/src/hadoop/test_jars/hadoop-hdfs-0.23.0-cdh4b1-tests.jar


BIN
desktop/libs/hadoop/src/hadoop/test_jars/junit-4.8.1.jar


BIN
desktop/libs/hadoop/src/hadoop/test_jars/mockito-all-1.8.5.jar


+ 17 - 21
desktop/libs/hadoop/src/hadoop/tests.py

@@ -23,8 +23,6 @@ import os
 from nose.tools import assert_true, assert_equal, assert_false
 from nose.plugins.attrib import attr
 
-import desktop.views
-
 from desktop.lib.django_test_util import make_logged_in_client
 from hadoop import cluster
 from hadoop import conf
@@ -37,9 +35,9 @@ def test_live_jobtracker():
   Checks that LiveJobTracker never raises
   exceptions for most of its calls.
   """
-  cluster = mini_cluster.shared_cluster()
   try:
-    jt = cluster.jt
+    minicluster = mini_cluster.shared_cluster()
+    jt = minicluster.jt
     # Make sure that none of the following
     # raise.
     assert_true(jt.queues())
@@ -56,7 +54,7 @@ def test_live_jobtracker():
     assert_true(jt.get_current_time())
     # not tested: get_job_xml
   finally:
-    cluster.shutdown()
+    minicluster.shutdown()
 
 
 def test_confparse():
@@ -117,21 +115,22 @@ def test_tricky_confparse():
 def test_config_validator_basic():
   reset = (
     conf.HADOOP_STREAMING_JAR.set_for_testing('/tmp'),
-    conf.HDFS_CLUSTERS['default'].NN_THRIFT_PORT.set_for_testing(1),
+    conf.HDFS_CLUSTERS['default'].WEBHDFS_URL.set_for_testing('http://not.the.re:50070/'),
     conf.MR_CLUSTERS['default'].JT_THRIFT_PORT.set_for_testing(70000),
   )
+  old = cluster.clear_caches()
   try:
     cli = make_logged_in_client()
     resp = cli.get('/debug/check_config')
     assert_true('hadoop.hadoop_streaming_jar' in resp.content)
     assert_true('Not a file' in resp.content)
-    assert_true('hadoop.hdfs_clusters.default' in resp.content)
-    assert_true('Failed to contact Namenode plugin' in resp.content)
+    assert_true('hadoop.hdfs_clusters.default.webhdfs_url' in resp.content)
     assert_true('hadoop.mapred_clusters.default.thrift_port' in resp.content)
     assert_true('Port should be' in resp.content)
   finally:
     for old_conf in reset:
       old_conf()
+    cluster.restore_caches(old)
 
 
 @attr('requires_hadoop')
@@ -140,36 +139,32 @@ def test_config_validator_more():
 
   # We don't actually use the mini_cluster. But the cluster sets up the correct
   # configuration that forms the test basis.
-  cluster = mini_cluster.shared_cluster()
-  if not cluster.fs.exists('/tmp'):
-    cluster.fs.setuser(cluster.fs.superuser)
-    cluster.fs.mkdir('/tmp', 0777)
+  minicluster = mini_cluster.shared_cluster()
   cli = make_logged_in_client()
 
   reset = (
-    conf.HADOOP_BIN.set_for_testing(cluster.fs.hadoop_bin_path),
-    conf.HDFS_CLUSTERS['default'].NN_HOST.set_for_testing('localhost'),
-    conf.HDFS_CLUSTERS['default'].NN_HDFS_PORT.set_for_testing(22),
-    conf.HDFS_CLUSTERS["default"].NN_THRIFT_PORT.set_for_testing(cluster.fs.thrift_port),
     conf.MR_CLUSTERS["default"].JT_HOST.set_for_testing("localhost"),
     conf.MR_CLUSTERS['default'].JT_THRIFT_PORT.set_for_testing(23),
   )
+  old = cluster.clear_caches()
   try:
     resp = cli.get('/debug/check_config')
 
-    assert_false('Failed to contact Namenode plugin' in resp.content)
-    assert_false('Failed to see HDFS root' in resp.content)
-    assert_true('Failed to upload files' in resp.content)
+    assert_false('Failed to access filesystem root' in resp.content)
+    assert_false('Failed to create' in resp.content)
+    assert_false('Failed to chown' in resp.content)
+    assert_false('Failed to delete' in resp.content)
     assert_true('Failed to contact JobTracker plugin' in resp.content)
   finally:
     for old_conf in reset:
       old_conf()
-    cluster.shutdown()
+    cluster.restore_caches(old)
+    minicluster.shutdown()
 
 
 def test_non_default_cluster():
   NON_DEFAULT_NAME = 'non_default'
-  cluster.clear_caches()
+  old = cluster.clear_caches()
   reset = (
     conf.HDFS_CLUSTERS.set_for_testing({ NON_DEFAULT_NAME: { } }),
     conf.MR_CLUSTERS.set_for_testing({ NON_DEFAULT_NAME: { } }),
@@ -187,3 +182,4 @@ def test_non_default_cluster():
   finally:
     for old_conf in reset:
       old_conf()
+    cluster.restore_caches(old)