Repository: hbase
Updated Branches:
  refs/heads/HBASE-14850 cda4e6a51 -> a2291fb3a (forced update)


http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
index 8b2b733..45093bb 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestHBaseAdminNoCluster.java
@@ -35,6 +35,8 @@ import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
 import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
 import 
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
@@ -310,6 +312,14 @@ public class TestHBaseAdminNoCluster {
           }
         });
     
Mockito.when(connection.getKeepAliveMasterService()).thenReturn(masterAdmin);
+    RpcControllerFactory rpcControllerFactory = 
Mockito.mock(RpcControllerFactory.class);
+    
Mockito.when(connection.getRpcControllerFactory()).thenReturn(rpcControllerFactory);
+    Mockito.when(rpcControllerFactory.newController()).thenReturn(
+      Mockito.mock(PayloadCarryingRpcController.class));
+
+    // we need a real retrying caller
+    RpcRetryingCallerFactory callerFactory = new 
RpcRetryingCallerFactory(configuration);
+    
Mockito.when(connection.getRpcRetryingCallerFactory()).thenReturn(callerFactory);
 
     Admin admin = null;
     try {

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
index 77771ba..d4d319a 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java
@@ -285,7 +285,7 @@ public class TestScannersFromClientSide {
   private void verifyExpectedCounts(Table table, Scan scan, int 
expectedRowCount,
       int expectedCellCount) throws Exception {
     ResultScanner scanner = table.getScanner(scan);
-    
+
     int rowCount = 0;
     int cellCount = 0;
     Result r = null;
@@ -609,7 +609,7 @@ public class TestScannersFromClientSide {
     byte[] regionName = hri.getRegionName();
     int i = cluster.getServerWith(regionName);
     HRegionServer rs = cluster.getRegionServer(i);
-    ProtobufUtil.closeRegion(
+    ProtobufUtil.closeRegion(null,
       rs.getRSRpcServices(), rs.getServerName(), regionName);
     long startTime = EnvironmentEdgeManager.currentTime();
     long timeOut = 300000;
@@ -627,7 +627,7 @@ public class TestScannersFromClientSide {
     RegionStates states = master.getAssignmentManager().getRegionStates();
     states.regionOffline(hri);
     states.updateRegionState(hri, State.OPENING);
-    ProtobufUtil.openRegion(rs.getRSRpcServices(), rs.getServerName(), hri);
+    ProtobufUtil.openRegion(null, rs.getRSRpcServices(), rs.getServerName(), 
hri);
     startTime = EnvironmentEdgeManager.currentTime();
     while (true) {
       if (rs.getOnlineRegion(regionName) != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java
new file mode 100644
index 0000000..b1b3b23
--- /dev/null
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java
@@ -0,0 +1,76 @@
+/**
+ * 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.ipc;
+
+import java.io.IOException;
+
+public class DelegatingRpcScheduler extends RpcScheduler {
+  protected RpcScheduler delegate;
+
+  public DelegatingRpcScheduler(RpcScheduler delegate) {
+    this.delegate = delegate;
+  }
+
+  @Override
+  public void stop() {
+    delegate.stop();
+  }
+  @Override
+  public void start() {
+    delegate.start();
+  }
+  @Override
+  public void init(Context context) {
+    delegate.init(context);
+  }
+  @Override
+  public int getReplicationQueueLength() {
+    return delegate.getReplicationQueueLength();
+  }
+
+  @Override
+  public int getPriorityQueueLength() {
+    return delegate.getPriorityQueueLength();
+  }
+
+  @Override
+  public int getGeneralQueueLength() {
+    return delegate.getGeneralQueueLength();
+  }
+
+  @Override
+  public int getActiveRpcHandlerCount() {
+    return delegate.getActiveRpcHandlerCount();
+  }
+
+  @Override
+  public boolean dispatch(CallRunner task) throws IOException, 
InterruptedException {
+    return delegate.dispatch(task);
+  }
+
+  @Override
+  public long getNumGeneralCallsDropped() {
+    return delegate.getNumGeneralCallsDropped();
+  }
+
+  @Override
+  public long getNumLifoModeSwitches() {
+    return delegate.getNumLifoModeSwitches();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
index 052e05c..32e3058 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFilesSplitRecovery.java
@@ -196,7 +196,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
           ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
         if (hri.getTable().equals(table)) {
           // splitRegion doesn't work if startkey/endkey are null
-          ProtobufUtil.split(hrs.getRSRpcServices(), hri, rowkey(ROWCOUNT / 
2)); // hard code split
+          ProtobufUtil.split(null, hrs.getRSRpcServices(), hri, 
rowkey(ROWCOUNT / 2));
         }
       }
 
@@ -480,6 +480,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
     final AtomicInteger countedLqis = new AtomicInteger();
     LoadIncrementalHFiles loader = new 
LoadIncrementalHFiles(util.getConfiguration()) {
 
+      @Override
       protected List<LoadQueueItem> groupOrSplit(
           Multimap<ByteBuffer, LoadQueueItem> regionGroups,
           final LoadQueueItem item, final Table htable,

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
index 142437c..b2ffc3e 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.hbase.master;
 
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import java.net.InetAddress;
 
@@ -32,6 +34,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import 
org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -81,7 +84,7 @@ public class TestClockSkewDetection {
 
       @Override
       public void abort(String why, Throwable e) {}
-      
+
       @Override
       public boolean isAborted() {
         return false;
@@ -103,10 +106,11 @@ public class TestClockSkewDetection {
 
       @Override
       public ClusterConnection getClusterConnection() {
-        // TODO Auto-generated method stub
-        return null;
+        ClusterConnection conn = mock(ClusterConnection.class);
+        
when(conn.getRpcControllerFactory()).thenReturn(mock(RpcControllerFactory.class));
+        return conn;
       }
-    }, null, false);
+    }, null, true);
 
     LOG.debug("regionServerStartup 1");
     InetAddress ia1 = InetAddress.getLocalHost();
@@ -135,7 +139,7 @@ public class TestClockSkewDetection {
       //we want an exception
       LOG.info("Recieved expected exception: "+e);
     }
-    
+
     try {
       // Master Time < Region Server Time
       LOG.debug("Test: Master Time < Region Server Time");
@@ -151,7 +155,7 @@ public class TestClockSkewDetection {
       // we want an exception
       LOG.info("Recieved expected exception: " + e);
     }
-    
+
     // make sure values above warning threshold but below max threshold don't 
kill
     LOG.debug("regionServerStartup 4");
     InetAddress ia4 = InetAddress.getLocalHost();
@@ -160,7 +164,7 @@ public class TestClockSkewDetection {
     request.setServerStartCode(-1);
     request.setServerCurrentTime(System.currentTimeMillis() - warningSkew * 2);
     sm.regionServerStartup(request.build(), ia4);
-    
+
     // make sure values above warning threshold but below max threshold don't 
kill
     LOG.debug("regionServerStartup 5");
     InetAddress ia5 = InetAddress.getLocalHost();
@@ -169,7 +173,7 @@ public class TestClockSkewDetection {
     request.setServerStartCode(-1);
     request.setServerCurrentTime(System.currentTimeMillis() + warningSkew * 2);
     sm.regionServerStartup(request.build(), ia5);
-    
+
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
index 03b43f1..0ee75a8 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
@@ -114,7 +114,7 @@ public class TestRegionServerNoMaster {
       return;
     }
 
-    ProtobufUtil.openRegion(hrs.getRSRpcServices(),
+    ProtobufUtil.openRegion(null, hrs.getRSRpcServices(),
       hrs.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
     while (true) {
       sn = mtl.getMetaRegionLocation(zkw);

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
index cbc1a90..22a9748 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java
@@ -372,7 +372,7 @@ public class SecureTestUtil {
             BlockingRpcChannel service = 
acl.coprocessorService(HConstants.EMPTY_START_ROW);
             AccessControlService.BlockingInterface protocol =
                 AccessControlService.newBlockingStub(service);
-            ProtobufUtil.grant(protocol, user, actions);
+            ProtobufUtil.grant(null, protocol, user, actions);
           }
         }
         return null;
@@ -395,7 +395,7 @@ public class SecureTestUtil {
             BlockingRpcChannel service = 
acl.coprocessorService(HConstants.EMPTY_START_ROW);
             AccessControlService.BlockingInterface protocol =
                 AccessControlService.newBlockingStub(service);
-            ProtobufUtil.revoke(protocol, user, actions);
+            ProtobufUtil.revoke(null, protocol, user, actions);
           }
         }
         return null;
@@ -418,7 +418,7 @@ public class SecureTestUtil {
             BlockingRpcChannel service = 
acl.coprocessorService(HConstants.EMPTY_START_ROW);
             AccessControlService.BlockingInterface protocol =
                 AccessControlService.newBlockingStub(service);
-            ProtobufUtil.grant(protocol, user, namespace, actions);
+            ProtobufUtil.grant(null, protocol, user, namespace, actions);
           }
         }
         return null;
@@ -483,7 +483,7 @@ public class SecureTestUtil {
             BlockingRpcChannel service = 
acl.coprocessorService(HConstants.EMPTY_START_ROW);
             AccessControlService.BlockingInterface protocol =
                 AccessControlService.newBlockingStub(service);
-            ProtobufUtil.revoke(protocol, user, namespace, actions);
+            ProtobufUtil.revoke(null, protocol, user, namespace, actions);
           }
         }
         return null;
@@ -507,7 +507,7 @@ public class SecureTestUtil {
             BlockingRpcChannel service = 
acl.coprocessorService(HConstants.EMPTY_START_ROW);
             AccessControlService.BlockingInterface protocol =
                 AccessControlService.newBlockingStub(service);
-            ProtobufUtil.grant(protocol, user, table, family, qualifier, 
actions);
+            ProtobufUtil.grant(null, protocol, user, table, family, qualifier, 
actions);
           }
         }
         return null;
@@ -573,7 +573,7 @@ public class SecureTestUtil {
             BlockingRpcChannel service = 
acl.coprocessorService(HConstants.EMPTY_START_ROW);
             AccessControlService.BlockingInterface protocol =
                 AccessControlService.newBlockingStub(service);
-            ProtobufUtil.revoke(protocol, user, table, family, qualifier, 
actions);
+            ProtobufUtil.revoke(null, protocol, user, table, family, 
qualifier, actions);
           }
         }
         return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 7b1454d..37c42a0 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -1165,7 +1165,7 @@ public class TestAccessController extends SecureTestUtil {
           BlockingRpcChannel service = 
acl.coprocessorService(TEST_TABLE.getName());
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.grant(protocol, USER_RO.getShortName(), TEST_TABLE, 
TEST_FAMILY, null,
+          ProtobufUtil.grant(null, protocol, USER_RO.getShortName(), 
TEST_TABLE, TEST_FAMILY, null,
             Action.READ);
         }
         return null;
@@ -1180,7 +1180,7 @@ public class TestAccessController extends SecureTestUtil {
           BlockingRpcChannel service = 
acl.coprocessorService(TEST_TABLE.getName());
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.revoke(protocol, USER_RO.getShortName(), TEST_TABLE, 
TEST_FAMILY, null,
+          ProtobufUtil.revoke(null, protocol, USER_RO.getShortName(), 
TEST_TABLE, TEST_FAMILY, null,
             Action.READ);
         }
         return null;
@@ -1195,7 +1195,7 @@ public class TestAccessController extends SecureTestUtil {
           BlockingRpcChannel service = 
acl.coprocessorService(TEST_TABLE.getName());
           AccessControlService.BlockingInterface protocol =
               AccessControlService.newBlockingStub(service);
-          ProtobufUtil.getUserPermissions(protocol, TEST_TABLE);
+          ProtobufUtil.getUserPermissions(null, protocol, TEST_TABLE);
         }
         return null;
       }
@@ -1209,7 +1209,7 @@ public class TestAccessController extends SecureTestUtil {
           BlockingRpcChannel service = 
acl.coprocessorService(HConstants.EMPTY_START_ROW);
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.getUserPermissions(protocol);
+          ProtobufUtil.getUserPermissions(null, protocol);
         }
         return null;
       }
@@ -1620,7 +1620,7 @@ public class TestAccessController extends SecureTestUtil {
         BlockingRpcChannel service = 
acl.coprocessorService(tableName.getName());
         AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-        perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+        perms = ProtobufUtil.getUserPermissions(null, protocol, tableName);
       } finally {
         acl.close();
       }
@@ -1647,7 +1647,7 @@ public class TestAccessController extends SecureTestUtil {
         BlockingRpcChannel service = 
acl.coprocessorService(tableName.getName());
         AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-        perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+        perms = ProtobufUtil.getUserPermissions(null, protocol, tableName);
       } finally {
         acl.close();
       }
@@ -1671,7 +1671,7 @@ public class TestAccessController extends SecureTestUtil {
         BlockingRpcChannel service = 
acl.coprocessorService(tableName.getName());
         AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-        perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+        perms = ProtobufUtil.getUserPermissions(null, protocol, tableName);
       } finally {
         acl.close();
       }
@@ -1691,7 +1691,7 @@ public class TestAccessController extends SecureTestUtil {
         BlockingRpcChannel service = 
acl.coprocessorService(tableName.getName());
         AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-        perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+        perms = ProtobufUtil.getUserPermissions(null, protocol, tableName);
       } finally {
         acl.close();
       }
@@ -1711,7 +1711,7 @@ public class TestAccessController extends SecureTestUtil {
         BlockingRpcChannel service = 
acl.coprocessorService(tableName.getName());
         AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-        perms = ProtobufUtil.getUserPermissions(protocol, tableName);
+        perms = ProtobufUtil.getUserPermissions(null, protocol, tableName);
       } finally {
         acl.close();
       }
@@ -1734,7 +1734,7 @@ public class TestAccessController extends SecureTestUtil {
       BlockingRpcChannel service = 
acl.coprocessorService(HConstants.EMPTY_START_ROW);
       AccessControlService.BlockingInterface protocol =
         AccessControlService.newBlockingStub(service);
-      perms = ProtobufUtil.getUserPermissions(protocol);
+      perms = ProtobufUtil.getUserPermissions(null, protocol);
     } finally {
       acl.close();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/05200976/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
index 96cd299..d5834fd 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
@@ -233,6 +233,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
   @Test
   public void testModifyNamespace() throws Exception {
     AccessTestAction modifyNamespace = new AccessTestAction() {
+      @Override
       public Object run() throws Exception {
         
ACCESS_CONTROLLER.preModifyNamespace(ObserverContext.createAndPrepare(CP_ENV, 
null),
           NamespaceDescriptor.create(TEST_NAMESPACE).addConfiguration("abc", 
"156").build());
@@ -359,7 +360,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
               acl.coprocessorService(HConstants.EMPTY_START_ROW);
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.grant(protocol, testUser, TEST_NAMESPACE, Action.WRITE);
+          ProtobufUtil.grant(null, protocol, testUser, TEST_NAMESPACE, 
Action.WRITE);
         } finally {
           acl.close();
           connection.close();
@@ -376,7 +377,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
           BlockingRpcChannel service = 
acl.coprocessorService(HConstants.EMPTY_START_ROW);
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.grant(protocol, USER_GROUP_NS_ADMIN.getShortName(),
+          ProtobufUtil.grant(null, protocol, 
USER_GROUP_NS_ADMIN.getShortName(),
             TEST_NAMESPACE, Action.READ);
         }
         return null;
@@ -384,6 +385,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
     };
 
     AccessTestAction revokeAction = new AccessTestAction() {
+      @Override
       public Object run() throws Exception {
         Connection connection = ConnectionFactory.createConnection(conf);
         Table acl = connection.getTable(AccessControlLists.ACL_TABLE_NAME);
@@ -392,7 +394,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
               acl.coprocessorService(HConstants.EMPTY_START_ROW);
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.revoke(protocol, testUser, TEST_NAMESPACE, 
Action.WRITE);
+          ProtobufUtil.revoke(null, protocol, testUser, TEST_NAMESPACE, 
Action.WRITE);
         } finally {
           acl.close();
           connection.close();
@@ -402,6 +404,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
     };
 
     AccessTestAction revokeNamespaceAction = new AccessTestAction() {
+      @Override
       public Object run() throws Exception {
         Connection connection = ConnectionFactory.createConnection(conf);
         Table acl = connection.getTable(AccessControlLists.ACL_TABLE_NAME);
@@ -410,7 +413,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
               acl.coprocessorService(HConstants.EMPTY_START_ROW);
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.revoke(protocol, USER_GROUP_NS_ADMIN.getShortName(),
+          ProtobufUtil.revoke(null, protocol, 
USER_GROUP_NS_ADMIN.getShortName(),
             TEST_NAMESPACE, Action.READ);
         } finally {
           acl.close();
@@ -429,7 +432,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
           BlockingRpcChannel service = 
acl.coprocessorService(HConstants.EMPTY_START_ROW);
           AccessControlService.BlockingInterface protocol =
             AccessControlService.newBlockingStub(service);
-          ProtobufUtil.getUserPermissions(protocol, 
Bytes.toBytes(TEST_NAMESPACE));
+          ProtobufUtil.getUserPermissions(null, protocol, 
Bytes.toBytes(TEST_NAMESPACE));
         } finally {
           acl.close();
           connection.close();

Reply via email to