Index: hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java (revision 1467725) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java (working copy) @@ -216,6 +216,7 @@ } } } + t.close(); assertEquals(count, i); } catch (IOException e) { fail("Failed due to exception"); @@ -327,6 +328,7 @@ // load of split top 3) load of split bottom assertEquals(attemptedCalls.get(), 3); assertExpectedTable(table, ROWCOUNT, 2); + t.close(); } /** Index: hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java (revision 1467725) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java (working copy) @@ -78,6 +78,7 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; @@ -209,6 +210,7 @@ Runtime.getRuntime().availableProcessors()); ThreadFactoryBuilder builder = new ThreadFactoryBuilder(); builder.setNameFormat("LoadIncrementalHFiles-%1$d"); + builder.setDaemon(true); ExecutorService pool = new ThreadPoolExecutor(nrThreads, nrThreads, 60, TimeUnit.SECONDS, new LinkedBlockingQueue(), @@ -282,6 +284,12 @@ } } pool.shutdown(); + try { + pool.awaitTermination(60000, TimeUnit.MILLISECONDS); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + } + if (!pool.isTerminated()) pool.shutdownNow(); if (queue != null && !queue.isEmpty()) { StringBuilder err = new StringBuilder(); err.append("-------------------------------------------------\n"); Index: hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java (revision 1467725) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/SecureBulkLoadEndpoint.java (working copy) @@ -18,9 +18,13 @@ package org.apache.hadoop.hbase.security.access; -import com.google.protobuf.RpcCallback; -import com.google.protobuf.RpcController; -import com.google.protobuf.Service; +import java.io.IOException; +import java.math.BigInteger; +import java.security.PrivilegedAction; +import java.security.SecureRandom; +import java.util.ArrayList; +import java.util.List; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -31,19 +35,20 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.CoprocessorEnvironment; -import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.coprocessor.CoprocessorService; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException; import org.apache.hadoop.hbase.ipc.RequestContext; import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; -import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadService; +import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadRequest; +import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadResponse; import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadRequest; import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.PrepareBulkLoadResponse; -import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadRequest; -import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.CleanupBulkLoadResponse; import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesRequest; import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadHFilesResponse; +import org.apache.hadoop.hbase.protobuf.generated.SecureBulkLoadProtos.SecureBulkLoadService; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.security.SecureBulkLoadUtil; import org.apache.hadoop.hbase.security.User; @@ -54,12 +59,9 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; -import java.io.IOException; -import java.math.BigInteger; -import java.security.PrivilegedAction; -import java.security.SecureRandom; -import java.util.ArrayList; -import java.util.List; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; /** * Coprocessor service for bulk loads in secure mode. @@ -121,7 +123,7 @@ baseStagingDir = SecureBulkLoadUtil.getBaseStagingDir(conf); try { - fs = FileSystem.get(conf); + fs = FileSystem.get(new HBaseConfiguration(conf)); fs.mkdirs(baseStagingDir, PERM_HIDDEN); fs.setPermission(baseStagingDir, PERM_HIDDEN); //no sticky bit in hadoop-1.0, making directory nonempty so it never gets erased