You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jx...@apache.org on 2013/05/10 00:55:43 UTC

svn commit: r1480828 - in /hbase/trunk: hbase-common/src/main/java/org/apache/hadoop/hbase/util/ hbase-common/src/test/java/org/apache/hadoop/hbase/util/ hbase-server/src/main/java/org/apache/hadoop/hbase/util/ hbase-server/src/test/java/org/apache/had...

Author: jxiang
Date: Thu May  9 22:55:42 2013
New Revision: 1480828

URL: http://svn.apache.org/r1480828
Log:
HBASE-8437 Clean up tmp coprocessor jars

Added:
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/KeyLocker.java   (with props)
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorClassLoader.java   (with props)
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestKeyLocker.java   (with props)
Removed:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/util/KeyLocker.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestKeyLocker.java
Modified:
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CoprocessorClassLoader.java
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/ClassLoaderTestHelper.java
    hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDynamicClassLoader.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CoprocessorClassLoader.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CoprocessorClassLoader.java?rev=1480828&r1=1480827&r2=1480828&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CoprocessorClassLoader.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CoprocessorClassLoader.java Thu May  9 22:55:42 2013
@@ -25,9 +25,12 @@ import java.security.AccessController;
 import java.security.PrivilegedAction;
 import java.util.Collection;
 import java.util.Enumeration;
+import java.util.HashSet;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.locks.Lock;
 import java.util.jar.JarEntry;
 import java.util.jar.JarFile;
+import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
@@ -38,6 +41,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.MapMaker;
 
 /**
@@ -120,6 +124,19 @@ public class CoprocessorClassLoader exte
     Pattern.compile("^[^-]+-default\\.xml$")
   };
 
+  private static final Pattern libJarPattern = Pattern.compile("[/]?lib/([^/]+\\.jar)");
+
+  /**
+   * A locker used to synchronize class loader initialization per coprocessor jar file
+   */
+  private static final KeyLocker<String> locker = new KeyLocker<String>();
+
+  /**
+   * A set used to synchronized parent path clean up.  Generally, there
+   * should be only one parent path, but using a set so that we can support more.
+   */
+  static final HashSet<String> parentDirLockSet = new HashSet<String>();
+
   /**
    * Creates a JarClassLoader that loads classes from the given paths.
    */
@@ -129,24 +146,24 @@ public class CoprocessorClassLoader exte
 
   private void init(Path path, String pathPrefix,
       Configuration conf) throws IOException {
-    if (path == null) {
-      throw new IOException("The jar path is null");
-    }
-    if (!path.toString().endsWith(".jar")) {
-      throw new IOException(path.toString() + ": not a jar file?");
-    }
-
     // Copy the jar to the local filesystem
-    String parentDirPath =
+    String parentDirStr =
       conf.get(LOCAL_DIR_KEY, DEFAULT_LOCAL_DIR) + TMP_JARS_DIR;
-    File parentDir = new File(parentDirPath);
-    if (!parentDir.mkdirs() && !parentDir.isDirectory()) {
-      throw new RuntimeException("Failed to create local dir " + parentDir.getPath()
-        + ", CoprocessorClassLoader failed to init");
+    synchronized (parentDirLockSet) {
+      if (!parentDirLockSet.contains(parentDirStr)) {
+        Path parentDir = new Path(parentDirStr);
+        FileSystem fs = parentDir.getFileSystem(conf);
+        fs.delete(parentDir, true); // it's ok if the dir doesn't exist now
+        parentDirLockSet.add(parentDirStr);
+        if (!fs.mkdirs(parentDir) && !fs.getFileStatus(parentDir).isDir()) {
+          throw new RuntimeException("Failed to create local dir " + parentDirStr
+            + ", CoprocessorClassLoader failed to init");
+        }
+      }
     }
 
     FileSystem fs = path.getFileSystem(conf);
-    File dst = new File(parentDir, "." + pathPrefix + "."
+    File dst = new File(parentDirStr, "." + pathPrefix + "."
       + path.getName() + "." + System.currentTimeMillis() + ".jar");
     fs.copyToLocalFile(path, new Path(dst.toString()));
     dst.deleteOnExit();
@@ -158,10 +175,12 @@ public class CoprocessorClassLoader exte
       Enumeration<JarEntry> entries = jarFile.entries();
       while (entries.hasMoreElements()) {
         JarEntry entry = entries.nextElement();
-        if (entry.getName().matches("[/]?lib/[^/]+\\.jar")) {
-          File file = new File(parentDir, "." + pathPrefix + "." + path.getName()
-            + "." + System.currentTimeMillis() + "." + entry.getName().substring(5));
-          IOUtils.copyBytes(jarFile.getInputStream(entry), new FileOutputStream(file), conf, true);
+        Matcher m = libJarPattern.matcher(entry.getName());
+        if (m.matches()) {
+          File file = new File(parentDirStr, "." + pathPrefix + "."
+            + path.getName() + "." + System.currentTimeMillis() + "." + m.group(1));
+          IOUtils.copyBytes(jarFile.getInputStream(entry),
+            new FileOutputStream(file), conf, true);
           file.deleteOnExit();
           addURL(file.toURI().toURL());
         }
@@ -173,7 +192,7 @@ public class CoprocessorClassLoader exte
 
   // This method is used in unit test
   public static CoprocessorClassLoader getIfCached(final Path path) {
-    if (path == null) return null; // No class loader for null path
+    Preconditions.checkNotNull(path, "The jar path is null!");
     return classLoadersCache.get(path);
   }
 
@@ -202,27 +221,46 @@ public class CoprocessorClassLoader exte
       final ClassLoader parent, final String pathPrefix,
       final Configuration conf) throws IOException {
     CoprocessorClassLoader cl = getIfCached(path);
+    String pathStr = path.toString();
     if (cl != null) {
-      LOG.debug("Found classloader "+ cl + "for "+ path.toString());
+      LOG.debug("Found classloader "+ cl + " for "+ pathStr);
       return cl;
     }
 
-    cl = AccessController.doPrivileged(new PrivilegedAction<CoprocessorClassLoader>() {
-      @Override
-      public CoprocessorClassLoader run() {
-        return new CoprocessorClassLoader(parent);
-      }
-    });
-
-    cl.init(path, pathPrefix, conf);
-
-    // Cache class loader as a weak value, will be GC'ed when no reference left
-    CoprocessorClassLoader prev = classLoadersCache.putIfAbsent(path, cl);
-    if (prev != null) {
-      // Lost update race, use already added class loader
-      cl = prev;
+    if (!pathStr.endsWith(".jar")) {
+      throw new IOException(pathStr + ": not a jar file?");
+    }
+
+    Lock lock = locker.acquireLock(pathStr);
+    try {
+      cl = getIfCached(path);
+      if (cl != null) {
+        LOG.debug("Found classloader "+ cl + " for "+ pathStr);
+        return cl;
+      }
+
+      cl = AccessController.doPrivileged(
+          new PrivilegedAction<CoprocessorClassLoader>() {
+        @Override
+        public CoprocessorClassLoader run() {
+          return new CoprocessorClassLoader(parent);
+        }
+      });
+
+      cl.init(path, pathPrefix, conf);
+
+      // Cache class loader as a weak value, will be GC'ed when no reference left
+      CoprocessorClassLoader prev = classLoadersCache.putIfAbsent(path, cl);
+      if (prev != null) {
+        // Lost update race, use already added class loader
+        LOG.warn("THIS SHOULD NOT HAPPEN, a class loader"
+          +" is already cached for " + pathStr);
+        cl = prev;
+      }
+      return cl;
+    } finally {
+      lock.unlock();
     }
-    return cl;
   }
 
   @Override

Added: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/KeyLocker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/KeyLocker.java?rev=1480828&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/KeyLocker.java (added)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/KeyLocker.java Thu May  9 22:55:42 2013
@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.util;
+
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * A utility class to manage a set of locks. Each lock is identified by a String which serves
+ * as a key. Typical usage is: <p>
+ * class Example{
+ * private final static KeyLocker<String> locker = new Locker<String>();
+ * <p/>
+ * public void foo(String s){
+ * Lock lock = locker.acquireLock(s);
+ * try {
+ * // whatever
+ * }finally{
+ * lock.unlock();
+ * }
+ * }
+ * }
+ * </p>
+ */
+public class KeyLocker<K extends Comparable<? super K>> {
+  private static final Log LOG = LogFactory.getLog(KeyLocker.class);
+
+  // The number of lock we want to easily support. It's not a maximum.
+  private static final int NB_CONCURRENT_LOCKS = 1000;
+
+  // We need an atomic counter to manage the number of users using the lock and free it when
+  //  it's equal to zero.
+  private final Map<K, Pair<KeyLock<K>, AtomicInteger>> locks =
+    new HashMap<K, Pair<KeyLock<K>, AtomicInteger>>(NB_CONCURRENT_LOCKS);
+
+  /**
+   * Return a lock for the given key. The lock is already locked.
+   *
+   * @param key
+   */
+  public ReentrantLock acquireLock(K key) {
+    if (key == null) throw new IllegalArgumentException("key must not be null");
+
+    Pair<KeyLock<K>, AtomicInteger> lock;
+    synchronized (this) {
+      lock = locks.get(key);
+      if (lock == null) {
+        lock = new Pair<KeyLock<K>, AtomicInteger>(
+          new KeyLock<K>(this, key), new AtomicInteger(1));
+        locks.put(key, lock);
+      } else {
+        lock.getSecond().incrementAndGet();
+      }
+    }
+    lock.getFirst().lock();
+    return lock.getFirst();
+  }
+
+  /**
+   * Acquire locks for a set of keys. The keys will be
+   * sorted internally to avoid possible deadlock.
+   */
+  public Map<K, Lock> acquireLocks(final Set<K> keys) {
+    Map<K, Lock> locks = new HashMap<K, Lock>(keys.size());
+    SortedSet<K> sortedKeys = new TreeSet<K>(keys);
+    for (K key : sortedKeys) {
+      locks.put(key, acquireLock(key));
+    }
+    return locks;
+  }
+
+  /**
+   * Free the lock for the given key.
+   */
+  private synchronized void releaseLock(K key) {
+    Pair<KeyLock<K>, AtomicInteger> lock = locks.get(key);
+    if (lock != null) {
+      if (lock.getSecond().decrementAndGet() == 0) {
+        locks.remove(key);
+      }
+    } else {
+      String message = "Can't release the lock for " + key+", this key is not in the key list." +
+        " known keys are: "+ locks.keySet();
+      LOG.error(message);
+      throw new RuntimeException(message);
+    }
+  }
+
+  static class KeyLock<K extends Comparable<? super K>> extends ReentrantLock {
+    private static final long serialVersionUID = -12432857283423584L;
+
+    private final KeyLocker<K> locker;
+    private final K lockId;
+
+    private KeyLock(KeyLocker<K> locker, K lockId) {
+      super();
+      this.locker = locker;
+      this.lockId = lockId;
+    }
+
+    @Override
+    public void unlock() {
+      super.unlock();
+      locker.releaseLock(lockId);
+    }
+  }
+}

Propchange: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/KeyLocker.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/ClassLoaderTestHelper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/ClassLoaderTestHelper.java?rev=1480828&r1=1480827&r2=1480828&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/ClassLoaderTestHelper.java (original)
+++ hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/ClassLoaderTestHelper.java Thu May  9 22:55:42 2013
@@ -37,6 +37,7 @@ import javax.tools.ToolProvider;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 
 /**
@@ -45,6 +46,8 @@ import org.apache.hadoop.fs.Path;
 public class ClassLoaderTestHelper {
   private static final Log LOG = LogFactory.getLog(ClassLoaderTestHelper.class);
 
+  private static final int BUFFER_SIZE = 4096;
+
   /**
    * Jar a list of files into a jar archive.
    *
@@ -53,7 +56,7 @@ public class ClassLoaderTestHelper {
    */
   private static boolean createJarArchive(File archiveFile, File[] tobeJared) {
     try {
-      byte buffer[] = new byte[4096];
+      byte buffer[] = new byte[BUFFER_SIZE];
       // Open archive file
       FileOutputStream stream = new FileOutputStream(archiveFile);
       JarOutputStream out = new JarOutputStream(stream, new Manifest());
@@ -156,10 +159,55 @@ public class ClassLoaderTestHelper {
     // build a jar file by the classes files
     String jarFileName = className + ".jar";
     File jarFile = new File(folder, jarFileName);
+    jarFile.getParentFile().mkdirs();
     if (!createJarArchive(jarFile,
         new File[]{new File(srcDir.toString(), className + ".class")})){
       assertTrue("Build jar file failed.", false);
     }
     return jarFile;
   }
+
+  /**
+   * Add a list of jar files to another jar file under a specific folder.
+   * It is used to generated coprocessor jar files which can be loaded by
+   * the coprocessor class loader.  It is for testing usage only so we
+   * don't be so careful about stream closing in case any exception.
+   *
+   * @param targetJar the target jar file
+   * @param libPrefix the folder where to put inner jar files
+   * @param srcJars the source inner jar files to be added
+   * @throws Exception if anything doesn't work as expected
+   */
+  public static void addJarFilesToJar(File targetJar,
+      String libPrefix, File... srcJars) throws Exception {
+    FileOutputStream stream = new FileOutputStream(targetJar);
+    JarOutputStream out = new JarOutputStream(stream, new Manifest());
+    byte buffer[] = new byte[BUFFER_SIZE];
+
+    for (File jarFile: srcJars) {
+      // Add archive entry
+      JarEntry jarAdd = new JarEntry(libPrefix + jarFile.getName());
+      jarAdd.setTime(jarFile.lastModified());
+      out.putNextEntry(jarAdd);
+
+      // Write file to archive
+      FileInputStream in = new FileInputStream(jarFile);
+      while (true) {
+        int nRead = in.read(buffer, 0, buffer.length);
+        if (nRead <= 0)
+          break;
+        out.write(buffer, 0, nRead);
+      }
+      in.close();
+    }
+    out.close();
+    stream.close();
+    LOG.info("Adding jar file to outer jar file completed");
+  }
+
+  static String localDirPath(Configuration conf) {
+    return conf.get(ClassLoaderBase.LOCAL_DIR_KEY)
+      + File.separator + "jars" + File.separator;
+  }
+
 }

Added: hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorClassLoader.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorClassLoader.java?rev=1480828&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorClassLoader.java (added)
+++ hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorClassLoader.java Thu May  9 22:55:42 2013
@@ -0,0 +1,109 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test TestCoprocessorClassLoader. More tests are in TestClassLoading
+ */
+@Category(SmallTests.class)
+public class TestCoprocessorClassLoader {
+  private static final Configuration conf = HBaseConfiguration.create();
+
+  private static final HBaseCommonTestingUtility TEST_UTIL = new HBaseCommonTestingUtility();
+
+  @Test
+  public void testCleanupOldJars() throws Exception {
+    String className = "TestCleanupOldJars";
+    String folder = TEST_UTIL.getDataTestDir().toString();
+    File jarFile = ClassLoaderTestHelper.buildJar(
+      folder, className, null, ClassLoaderTestHelper.localDirPath(conf));
+    File tmpJarFile = new File(jarFile.getParent(), "/tmp/" + className + ".test.jar");
+    if (tmpJarFile.exists()) tmpJarFile.delete();
+    assertFalse("tmp jar file should not exist", tmpJarFile.exists());
+    IOUtils.copyBytes(new FileInputStream(jarFile),
+      new FileOutputStream(tmpJarFile), conf, true);
+    assertTrue("tmp jar file should be created", tmpJarFile.exists());
+    Path path = new Path(jarFile.getAbsolutePath());
+    ClassLoader parent = TestCoprocessorClassLoader.class.getClassLoader();
+    CoprocessorClassLoader.parentDirLockSet.clear(); // So that clean up can be triggered
+    ClassLoader classLoader = CoprocessorClassLoader.getClassLoader(path, parent, "111", conf);
+    assertNotNull("Classloader should be created", classLoader);
+    assertFalse("tmp jar file should be removed", tmpJarFile.exists());
+  }
+
+  @Test
+  public void testLibJarName() throws Exception {
+    checkingLibJarName("TestLibJarName.jar", "/lib/");
+  }
+
+  @Test
+  public void testRelativeLibJarName() throws Exception {
+    checkingLibJarName("TestRelativeLibJarName.jar", "lib/");
+  }
+
+  /**
+   * Test to make sure the lib jar file extracted from a coprocessor jar have
+   * the right name.  Otherwise, some existing jar could be override if there are
+   * naming conflicts.
+   */
+  private void checkingLibJarName(String jarName, String libPrefix) throws Exception {
+    File tmpFolder = new File(ClassLoaderTestHelper.localDirPath(conf), "tmp");
+    if (tmpFolder.exists()) { // Clean up the tmp folder
+      for (File f: tmpFolder.listFiles()) {
+        f.delete();
+      }
+    }
+    String className = "CheckingLibJarName";
+    String folder = TEST_UTIL.getDataTestDir().toString();
+    File innerJarFile = ClassLoaderTestHelper.buildJar(
+      folder, className, null, ClassLoaderTestHelper.localDirPath(conf));
+    File targetJarFile = new File(innerJarFile.getParent(), jarName);
+    ClassLoaderTestHelper.addJarFilesToJar(targetJarFile, libPrefix, innerJarFile);
+    Path path = new Path(targetJarFile.getAbsolutePath());
+    ClassLoader parent = TestCoprocessorClassLoader.class.getClassLoader();
+    ClassLoader classLoader = CoprocessorClassLoader.getClassLoader(path, parent, "112", conf);
+    assertNotNull("Classloader should be created", classLoader);
+    String fileToLookFor = "." + className + ".jar";
+    for (String f: tmpFolder.list()) {
+      if (f.endsWith(fileToLookFor) && f.contains(jarName)) {
+        // Cool, found it;
+        return;
+      }
+    }
+    fail("Could not find the expected lib jar file");
+  }
+}

Propchange: hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorClassLoader.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDynamicClassLoader.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDynamicClassLoader.java?rev=1480828&r1=1480827&r2=1480828&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDynamicClassLoader.java (original)
+++ hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestDynamicClassLoader.java Thu May  9 22:55:42 2013
@@ -63,7 +63,8 @@ public class TestDynamicClassLoader {
 
     try {
       String folder = TEST_UTIL.getDataTestDir().toString();
-      ClassLoaderTestHelper.buildJar(folder, className, null, localDirPath());
+      ClassLoaderTestHelper.buildJar(
+        folder, className, null, ClassLoaderTestHelper.localDirPath(conf));
       classLoader.loadClass(className);
     } catch (ClassNotFoundException cnfe) {
       LOG.error("Should be able to load class " + className, cnfe);
@@ -95,11 +96,6 @@ public class TestDynamicClassLoader {
     }
   }
 
-  private String localDirPath() {
-    return conf.get("hbase.local.dir")
-      + File.separator + "jars" + File.separator;
-  }
-
   private void deleteClass(String className) throws Exception {
     String jarFileName = className + ".jar";
     File file = new File(TEST_UTIL.getDataTestDir().toString(), jarFileName);
@@ -110,7 +106,7 @@ public class TestDynamicClassLoader {
     file.delete();
     assertFalse("Should be deleted: " + file.getPath(), file.exists());
 
-    file = new File(localDirPath(), jarFileName);
+    file = new File(ClassLoaderTestHelper.localDirPath(conf), jarFileName);
     file.delete();
     assertFalse("Should be deleted: " + file.getPath(), file.exists());
   }

Added: hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestKeyLocker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestKeyLocker.java?rev=1480828&view=auto
==============================================================================
--- hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestKeyLocker.java (added)
+++ hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestKeyLocker.java Thu May  9 22:55:42 2013
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.util;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.util.concurrent.locks.ReentrantLock;
+
+@Category(SmallTests.class)
+public class TestKeyLocker {
+  @Test
+  public void testLocker(){
+    KeyLocker<String> locker = new KeyLocker();
+    ReentrantLock lock1 = locker.acquireLock("l1");
+    Assert.assertTrue(lock1.isHeldByCurrentThread());
+
+    ReentrantLock lock2 = locker.acquireLock("l2");
+    Assert.assertTrue(lock2.isHeldByCurrentThread());
+    Assert.assertTrue(lock1 != lock2);
+
+    // same key = same lock
+    ReentrantLock lock20 = locker.acquireLock("l2");
+    Assert.assertTrue(lock20 == lock2);
+    Assert.assertTrue(lock2.isHeldByCurrentThread());
+    Assert.assertTrue(lock20.isHeldByCurrentThread());
+
+    // Locks are still reentrant; so with 2 acquires we want two unlocks
+    lock20.unlock();
+    Assert.assertTrue(lock20.isHeldByCurrentThread());
+
+    lock2.unlock();
+    Assert.assertFalse(lock20.isHeldByCurrentThread());
+
+    // The lock object was freed once useless, so we're recreating a new one
+    ReentrantLock lock200 = locker.acquireLock("l2");
+    Assert.assertTrue(lock2 != lock200);
+    lock200.unlock();
+    Assert.assertFalse(lock200.isHeldByCurrentThread());
+
+    // first lock is still there
+    Assert.assertTrue(lock1.isHeldByCurrentThread());
+    lock1.unlock();
+    Assert.assertFalse(lock1.isHeldByCurrentThread());
+  }
+}

Propchange: hbase/trunk/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestKeyLocker.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java?rev=1480828&r1=1480827&r2=1480828&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestClassLoading.java Thu May  9 22:55:42 2013
@@ -56,7 +56,6 @@ public class TestClassLoading {
 
   private static MiniDFSCluster cluster;
 
-  static final int BUFFER_SIZE = 4096;
   static final String tableName = "TestClassLoading";
   static final String cpName1 = "TestCP1";
   static final String cpName2 = "TestCP2";
@@ -397,32 +396,8 @@ public class TestClassLoading {
     File innerJarFile2 = buildCoprocessorJar(cpName2);
     File outerJarFile = new File(TEST_UTIL.getDataTestDir().toString(), "outer.jar");
 
-    byte buffer[] = new byte[BUFFER_SIZE];
-    // TODO: code here and elsewhere in this file is duplicated w/TestClassFinder.
-    //       Some refactoring may be in order...
-    // Open archive file
-    FileOutputStream stream = new FileOutputStream(outerJarFile);
-    JarOutputStream out = new JarOutputStream(stream, new Manifest());
-
-    for (File jarFile: new File[] { innerJarFile1, innerJarFile2 }) {
-      // Add archive entry
-      JarEntry jarAdd = new JarEntry(libPrefix + jarFile.getName());
-      jarAdd.setTime(jarFile.lastModified());
-      out.putNextEntry(jarAdd);
-
-      // Write file to archive
-      FileInputStream in = new FileInputStream(jarFile);
-      while (true) {
-        int nRead = in.read(buffer, 0, buffer.length);
-        if (nRead <= 0)
-          break;
-        out.write(buffer, 0, nRead);
-      }
-      in.close();
-    }
-    out.close();
-    stream.close();
-    LOG.info("Adding jar file to outer jar file completed");
+    ClassLoaderTestHelper.addJarFilesToJar(
+      outerJarFile, libPrefix, innerJarFile1, innerJarFile2);
 
     // copy the jars into dfs
     fs.copyFromLocalFile(new Path(outerJarFile.getPath()),