You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2015/10/02 00:15:37 UTC

hbase git commit: HBASE-14475 Region split requests are always audited with hbase user rather than request user (Ted Yu)

Repository: hbase
Updated Branches:
  refs/heads/branch-1.0 e67498c48 -> fba3f60be


HBASE-14475 Region split requests are always audited with hbase user rather than request user (Ted Yu)

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java

Amending-Author: Andrew Purtell <ap...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/fba3f60b
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/fba3f60b
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/fba3f60b

Branch: refs/heads/branch-1.0
Commit: fba3f60be88a97196c39cd4ffc0ad6edf5097940
Parents: e67498c
Author: Andrew Purtell <ap...@apache.org>
Authored: Thu Oct 1 12:09:13 2015 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Thu Oct 1 12:29:44 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hbase/ipc/RpcCallContext.java |  9 +++
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 23 ++++++
 .../hbase/regionserver/CompactSplitThread.java  | 75 ++++++++++++++------
 .../hbase/regionserver/CompactionRequestor.java |  8 ++-
 .../hbase/regionserver/HRegionServer.java       |  2 +-
 .../hbase/regionserver/RSRpcServices.java       |  7 +-
 .../hadoop/hbase/regionserver/SplitRequest.java | 39 +++++++---
 .../hbase/regionserver/TestCompaction.java      |  4 +-
 8 files changed, 129 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fba3f60b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
index fd16346..9c2bc9b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
+import org.apache.hadoop.hbase.security.User;
+
 
 public interface RpcCallContext extends Delayable {
   /**
@@ -36,4 +38,11 @@ public interface RpcCallContext extends Delayable {
    * @return True if the client supports cellblocks, else return all content in pb
    */
   boolean isClientCellBlockSupport();
+
+  /**
+   * Returns the user credentials associated with the current RPC request or
+   * <code>null</code> if no credentials were provided.
+   * @return A User
+   */
+  User getRequestUser();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fba3f60b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index e201f94..a22a15d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -98,6 +98,7 @@ import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslDigestCallbackHan
 import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslGssCallbackHandler;
 import org.apache.hadoop.hbase.security.SaslStatus;
 import org.apache.hadoop.hbase.security.SaslUtil;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -292,6 +293,8 @@ public class RpcServer implements RpcServerInterface {
     protected boolean isError;
     protected TraceInfo tinfo;
 
+    private User user;
+
     Call(int id, final BlockingService service, final MethodDescriptor md, RequestHeader header,
          Message param, CellScanner cellScanner, Connection connection, Responder responder,
          long size, TraceInfo tinfo) {
@@ -309,6 +312,7 @@ public class RpcServer implements RpcServerInterface {
       this.isError = false;
       this.size = size;
       this.tinfo = tinfo;
+      this.user = connection.user == null? null: userProvider.create(connection.user);
     }
 
     @Override
@@ -495,6 +499,15 @@ public class RpcServer implements RpcServerInterface {
         this.responder.doRespond(this);
       }
     }
+
+    public UserGroupInformation getRemoteUser() {
+      return connection.user;
+    }
+
+    @Override
+    public User getRequestUser() {
+      return user;
+    }
   }
 
   /** Listens on the socket. Creates jobs for the handler threads*/
@@ -2306,6 +2319,16 @@ public class RpcServer implements RpcServerInterface {
   }
 
   /**
+   * Returns the user credentials associated with the current RPC request or
+   * <code>null</code> if no credentials were provided.
+   * @return A User
+   */
+  public static User getRequestUser() {
+    RpcCallContext ctx = getCurrentCall();
+    return ctx == null? null: ctx.getRequestUser();
+  }
+
+  /**
    * @param serviceName Some arbitrary string that represents a 'service'.
    * @param services Available service instances
    * @return Matching BlockingServiceAndInterface pair

http://git-wip-us.apache.org/repos/asf/hbase/blob/fba3f60b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
index 970bf55..83d59a6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactSplitThread.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.StringWriter;
+import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
@@ -42,6 +43,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.util.StringUtils;
@@ -236,6 +238,13 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
   }
 
   public synchronized void requestSplit(final HRegion r, byte[] midKey) {
+    requestSplit(r, midKey, null);
+  }
+
+  /*
+   * The User parameter allows the split thread to assume the correct user identity
+   */
+  public synchronized void requestSplit(final HRegion r, byte[] midKey, User user) {
     if (midKey == null) {
       LOG.debug("Region " + r.getRegionNameAsString() +
         " not splittable because midkey=null");
@@ -245,7 +254,7 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
       return;
     }
     try {
-      this.splits.execute(new SplitRequest(r, midKey, this.server));
+      this.splits.execute(new SplitRequest(r, midKey, this.server, user));
       if (LOG.isDebugEnabled()) {
         LOG.debug("Split requested for " + r + ".  " + this);
       }
@@ -263,54 +272,55 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
   @Override
   public synchronized List<CompactionRequest> requestCompaction(final HRegion r, final String why,
       List<Pair<CompactionRequest, Store>> requests) throws IOException {
-    return requestCompaction(r, why, Store.NO_PRIORITY, requests);
+    return requestCompaction(r, why, Store.NO_PRIORITY, requests, null);
   }
 
   @Override
   public synchronized CompactionRequest requestCompaction(final HRegion r, final Store s,
       final String why, CompactionRequest request) throws IOException {
-    return requestCompaction(r, s, why, Store.NO_PRIORITY, request);
+    return requestCompaction(r, s, why, Store.NO_PRIORITY, request, null);
   }
 
   @Override
   public synchronized List<CompactionRequest> requestCompaction(final HRegion r, final String why,
-      int p, List<Pair<CompactionRequest, Store>> requests) throws IOException {
-    return requestCompactionInternal(r, why, p, requests, true);
+      int p, List<Pair<CompactionRequest, Store>> requests, User user) throws IOException {
+    return requestCompactionInternal(r, why, p, requests, true, user);
   }
 
   private List<CompactionRequest> requestCompactionInternal(final HRegion r, final String why,
-      int p, List<Pair<CompactionRequest, Store>> requests, boolean selectNow) throws IOException {
+      int p, List<Pair<CompactionRequest, Store>> requests, boolean selectNow, User user)
+          throws IOException {
     // not a special compaction request, so make our own list
     List<CompactionRequest> ret = null;
     if (requests == null) {
       ret = selectNow ? new ArrayList<CompactionRequest>(r.getStores().size()) : null;
       for (Store s : r.getStores().values()) {
-        CompactionRequest cr = requestCompactionInternal(r, s, why, p, null, selectNow);
+        CompactionRequest cr = requestCompactionInternal(r, s, why, p, null, selectNow, user);
         if (selectNow) ret.add(cr);
       }
     } else {
       Preconditions.checkArgument(selectNow); // only system requests have selectNow == false
       ret = new ArrayList<CompactionRequest>(requests.size());
       for (Pair<CompactionRequest, Store> pair : requests) {
-        ret.add(requestCompaction(r, pair.getSecond(), why, p, pair.getFirst()));
+        ret.add(requestCompaction(r, pair.getSecond(), why, p, pair.getFirst(), user));
       }
     }
     return ret;
   }
 
   public CompactionRequest requestCompaction(final HRegion r, final Store s,
-      final String why, int priority, CompactionRequest request) throws IOException {
-    return requestCompactionInternal(r, s, why, priority, request, true);
+      final String why, int priority, CompactionRequest request, User user) throws IOException {
+    return requestCompactionInternal(r, s, why, priority, request, true, user);
   }
 
   public synchronized void requestSystemCompaction(
       final HRegion r, final String why) throws IOException {
-    requestCompactionInternal(r, why, Store.NO_PRIORITY, null, false);
+    requestCompactionInternal(r, why, Store.NO_PRIORITY, null, false, null);
   }
 
   public void requestSystemCompaction(
       final HRegion r, final Store s, final String why) throws IOException {
-    requestCompactionInternal(r, s, why, Store.NO_PRIORITY, null, false);
+    requestCompactionInternal(r, s, why, Store.NO_PRIORITY, null, false, null);
   }
 
   /**
@@ -322,7 +332,7 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
    *          compaction will be used.
    */
   private synchronized CompactionRequest requestCompactionInternal(final HRegion r, final Store s,
-      final String why, int priority, CompactionRequest request, boolean selectNow)
+      final String why, int priority, CompactionRequest request, boolean selectNow, User user)
           throws IOException {
     if (this.server.isStopped()
         || (r.getTableDesc() != null && !r.getTableDesc().isCompactionEnabled())) {
@@ -339,7 +349,7 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
     // pool; we will do selection there, and move to large pool if necessary.
     ThreadPoolExecutor pool = (selectNow && s.throttleCompaction(compaction.getRequest().getSize()))
       ? longCompactions : shortCompactions;
-    pool.execute(new CompactionRunner(s, r, compaction, pool));
+    pool.execute(new CompactionRunner(s, r, compaction, pool, user));
     if (LOG.isDebugEnabled()) {
       String type = (pool == shortCompactions) ? "Small " : "Large ";
       LOG.debug(type + "Compaction requested: " + (selectNow ? compaction.toString() : "system")
@@ -443,9 +453,10 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
     private CompactionContext compaction;
     private int queuedPriority;
     private ThreadPoolExecutor parent;
+    private User user;
 
     public CompactionRunner(Store store, HRegion region,
-        CompactionContext compaction, ThreadPoolExecutor parent) {
+        CompactionContext compaction, ThreadPoolExecutor parent, User user) {
       super();
       this.store = store;
       this.region = region;
@@ -453,6 +464,7 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
       this.queuedPriority = (this.compaction == null)
           ? store.getCompactPriority() : compaction.getRequest().getPriority();
       this.parent = parent;
+      this.user = user;
     }
 
     @Override
@@ -461,13 +473,7 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
           : ("Store = " + store.toString() + ", pri = " + queuedPriority);
     }
 
-    @Override
-    public void run() {
-      Preconditions.checkNotNull(server);
-      if (server.isStopped()
-          || (region.getTableDesc() != null && !region.getTableDesc().isCompactionEnabled())) {
-        return;
-      }
+    private void doCompaction() {
       // Common case - system compaction without a file selection. Select now.
       if (this.compaction == null) {
         int oldPriority = this.queuedPriority;
@@ -536,6 +542,31 @@ public class CompactSplitThread implements CompactionRequestor, PropagatingConfi
       this.compaction.getRequest().afterExecute();
     }
 
+    @Override
+    public void run() {
+      Preconditions.checkNotNull(server);
+      if (server.isStopped()
+          || (region.getTableDesc() != null && !region.getTableDesc().isCompactionEnabled())) {
+        return;
+      }
+      if (this.user == null) doCompaction();
+      else {
+        try {
+          user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
+            @Override
+            public Void run() throws Exception {
+              doCompaction();
+              return null;
+            }
+          });
+        } catch (InterruptedException ie) {
+          Thread.currentThread().interrupt();
+        } catch (IOException ioe) {
+          LOG.error("Encountered exception while compacting", ioe);
+        }
+      }
+    }
+
     private String formatStackTrace(Exception ex) {
       StringWriter sw = new StringWriter();
       PrintWriter pw = new PrintWriter(sw);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fba3f60b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java
index 93a73e9..1fa4b8e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java
@@ -23,6 +23,7 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Pair;
 
 @InterfaceAudience.Private
@@ -72,12 +73,14 @@ public interface CompactionRequestor {
    * @param requests custom compaction requests. Each compaction must specify the store on which it
    *          is acting. Can be <tt>null</tt> in which case a compaction will be attempted on all
    *          stores for the region.
+   * @user  the effective user
    * @return The created {@link CompactionRequest CompactionRequests} or an empty list if no
    *         compactions were started.
    * @throws IOException
    */
   List<CompactionRequest> requestCompaction(
-    final HRegion r, final String why, int pri, List<Pair<CompactionRequest, Store>> requests
+    final HRegion r, final String why, int pri, List<Pair<CompactionRequest, Store>> requests,
+    User user
   ) throws IOException;
 
   /**
@@ -87,10 +90,11 @@ public interface CompactionRequestor {
    * @param pri Priority of this compaction. minHeap. <=0 is critical
    * @param request custom compaction request to run. {@link Store} and {@link HRegion} for the
    *          request must match the region and store specified here.
+   * @param user
    * @return The created {@link CompactionRequest} or <tt>null</tt> if no compaction was started
    * @throws IOException
    */
   CompactionRequest requestCompaction(
-    final HRegion r, final Store s, final String why, int pri, CompactionRequest request
+    final HRegion r, final Store s, final String why, int pri, CompactionRequest request, User user
   ) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/fba3f60b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 48e93bd..0206d92 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1465,7 +1465,7 @@ public class HRegionServer extends HasThread implements
               } else {
                 this.instance.compactSplitThread.requestCompaction(r, s, getName()
                     + " requests major compaction; use configured priority",
-                  this.majorCompactPriority, null);
+                  this.majorCompactPriority, null, null);
               }
             }
           } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fba3f60b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 20ea971..31ac688 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -1064,10 +1064,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       String log = "User-triggered " + (major ? "major " : "") + "compaction" + familyLogMsg;
       if(family != null) {
         regionServer.compactSplitThread.requestCompaction(region, store, log,
-          Store.PRIORITY_USER, null);
+          Store.PRIORITY_USER, null, RpcServer.getRequestUser());
       } else {
         regionServer.compactSplitThread.requestCompaction(region, log,
-          Store.PRIORITY_USER, null);
+          Store.PRIORITY_USER, null, RpcServer.getRequestUser());
       }
       return CompactRegionResponse.newBuilder().build();
     } catch (IOException ie) {
@@ -1610,7 +1610,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         splitPoint = request.getSplitPoint().toByteArray();
       }
       region.forceSplit(splitPoint);
-      regionServer.compactSplitThread.requestSplit(region, region.checkSplit());
+      regionServer.compactSplitThread.requestSplit(region, region.checkSplit(),
+        RpcServer.getRequestUser());
       return SplitRegionResponse.newBuilder().build();
     } catch (DroppedSnapshotException ex) {
       regionServer.abort("Replay of WAL required. Forcing server shutdown", ex);

http://git-wip-us.apache.org/repos/asf/hbase/blob/fba3f60b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
index 497fd61..2abedd9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitRequest.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -26,6 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.DroppedSnapshotException;
 import org.apache.hadoop.hbase.RemoteExceptionHandler;
 import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.util.StringUtils;
@@ -41,13 +43,15 @@ class SplitRequest implements Runnable {
   private final HRegion parent;
   private final byte[] midKey;
   private final HRegionServer server;
+  private final User user;
   private TableLock tableLock;
 
-  SplitRequest(HRegion region, byte[] midKey, HRegionServer hrs) {
+  SplitRequest(HRegion region, byte[] midKey, HRegionServer hrs, User user) {
     Preconditions.checkNotNull(hrs);
     this.parent = region;
     this.midKey = midKey;
     this.server = hrs;
+    this.user = user;
   }
 
   @Override
@@ -55,13 +59,7 @@ class SplitRequest implements Runnable {
     return "regionName=" + parent + ", midKey=" + Bytes.toStringBinary(midKey);
   }
 
-  @Override
-  public void run() {
-    if (this.server.isStopping() || this.server.isStopped()) {
-      LOG.debug("Skipping split because server is stopping=" +
-        this.server.isStopping() + " or stopped=" + this.server.isStopped());
-      return;
-    }
+  private void doSplitting() {
     boolean success = false;
     server.metricsRegionServer.incrSplitRequest();
     long startTime = EnvironmentEdgeManager.currentTime();
@@ -147,6 +145,31 @@ class SplitRequest implements Runnable {
     }
   }
 
+  @Override
+  public void run() {
+    if (this.server.isStopping() || this.server.isStopped()) {
+      LOG.debug("Skipping split because server is stopping=" +
+        this.server.isStopping() + " or stopped=" + this.server.isStopped());
+      return;
+    }
+    if (this.user == null) doSplitting();
+    else {
+      try {
+        user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            doSplitting();
+            return null;
+          }
+        });
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+      } catch (IOException ioe) {
+        LOG.error("Encountered exception while splitting", ioe);
+      }
+    }
+  }
+
   protected void releaseTableLock() {
     if (this.tableLock != null) {
       try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/fba3f60b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
index 46ab053..3971ebc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java
@@ -284,7 +284,7 @@ public class TestCompaction {
 
     CountDownLatch latch = new CountDownLatch(1);
     TrackableCompactionRequest request = new TrackableCompactionRequest(latch);
-    thread.requestCompaction(r, store, "test custom comapction", Store.PRIORITY_USER, request);
+    thread.requestCompaction(r, store, "test custom comapction", Store.PRIORITY_USER, request,null);
     // wait for the latch to complete.
     latch.await();
 
@@ -320,7 +320,7 @@ public class TestCompaction {
     }
 
     thread.requestCompaction(r, "test mulitple custom comapctions", Store.PRIORITY_USER,
-      Collections.unmodifiableList(requests));
+      Collections.unmodifiableList(requests), null);
 
     // wait for the latch to complete.
     latch.await();