You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ga...@apache.org on 2013/03/13 01:38:20 UTC

svn commit: r1455764 - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-server/src/main/java/org/apache/hadoop/...

Author: garyh
Date: Wed Mar 13 00:38:19 2013
New Revision: 1455764

URL: http://svn.apache.org/r1455764
Log:
HBASE-7482 Port HBASE-7442 HBase remote CopyTable not working when security enabled to trunk

Modified:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcClientEngine.java
    hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/RandomTimeoutRpcEngine.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=1455764&r1=1455763&r2=1455764&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Wed Mar 13 00:38:19 2013
@@ -615,7 +615,7 @@ public class HConnectionManager {
       // ProtobufRpcClientEngine is the main RpcClientEngine implementation,
       // but we maintain access through an interface to allow overriding for tests
       // RPC engine setup must follow obtaining the cluster ID for token authentication to work
-      this.rpcEngine = new ProtobufRpcClientEngine(this.conf);
+      this.rpcEngine = new ProtobufRpcClientEngine(this.conf, this.clusterId);
     }
 
     /**
@@ -627,41 +627,35 @@ public class HConnectionManager {
     }
 
     private String clusterId = null;
+
     public final void retrieveClusterId(){
-      if (conf.get(HConstants.CLUSTER_ID) != null){
+      if (clusterId != null) {
         return;
       }
 
       // No synchronized here, worse case we will retrieve it twice, that's
       //  not an issue.
-      if (this.clusterId == null){
-        this.clusterId = conf.get(HConstants.CLUSTER_ID);
-        if (this.clusterId == null) {
-          ZooKeeperKeepAliveConnection zkw = null;
-          try {
-            zkw = getKeepAliveZooKeeperWatcher();
-            this.clusterId = ZKClusterId.readClusterIdZNode(zkw);
-            if (clusterId == null) {
-              LOG.info("ClusterId read in ZooKeeper is null");
-            }
-          } catch (KeeperException e) {
-            LOG.warn("Can't retrieve clusterId from Zookeeper", e);
-          } catch (IOException e) {
-            LOG.warn("Can't retrieve clusterId from Zookeeper", e);
-          } finally {
-            if (zkw != null) {
-              zkw.close();
-            }
-          }
-          if (this.clusterId == null) {
-            this.clusterId = "default";
-          }
-
-          LOG.info("ClusterId is " + clusterId);
+      ZooKeeperKeepAliveConnection zkw = null;
+      try {
+        zkw = getKeepAliveZooKeeperWatcher();
+        clusterId = ZKClusterId.readClusterIdZNode(zkw);
+        if (clusterId == null) {
+          LOG.info("ClusterId read in ZooKeeper is null");
+        }
+      } catch (KeeperException e) {
+        LOG.warn("Can't retrieve clusterId from Zookeeper", e);
+      } catch (IOException e) {
+        LOG.warn("Can't retrieve clusterId from Zookeeper", e);
+      } finally {
+        if (zkw != null) {
+          zkw.close();
         }
       }
+      if (clusterId == null) {
+        clusterId = HConstants.CLUSTER_ID_DEFAULT;
+      }
 
-      conf.set(HConstants.CLUSTER_ID, clusterId);
+      LOG.info("ClusterId is " + clusterId);
     }
 
     @Override

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java?rev=1455764&r1=1455763&r2=1455764&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java Wed Mar 13 00:38:19 2013
@@ -1156,7 +1156,7 @@ public class HBaseClient {
    * @param conf configuration
    * @param factory socket factory
    */
-  public HBaseClient(Configuration conf, SocketFactory factory) {
+  public HBaseClient(Configuration conf, String clusterId, SocketFactory factory) {
     this.maxIdleTime =
       conf.getInt("hbase.ipc.client.connection.maxidletime", 10000); //10s
     this.maxRetries = conf.getInt("hbase.ipc.client.connect.max.retries", 0);
@@ -1169,7 +1169,7 @@ public class HBaseClient {
     }
     this.conf = conf;
     this.socketFactory = factory;
-    this.clusterId = conf.get(HConstants.CLUSTER_ID, "default");
+    this.clusterId = clusterId != null ? clusterId : HConstants.CLUSTER_ID_DEFAULT;
     this.connections = new PoolMap<ConnectionId, Connection>(
         getPoolType(conf), getPoolSize(conf));
     this.failedServers = new FailedServers(conf);
@@ -1179,8 +1179,8 @@ public class HBaseClient {
    * Construct an IPC client with the default SocketFactory
    * @param conf configuration
    */
-  public HBaseClient(Configuration conf) {
-    this(conf, NetUtils.getDefaultSocketFactory(conf));
+  public HBaseClient(Configuration conf, String clusterId) {
+    this(conf, clusterId, NetUtils.getDefaultSocketFactory(conf));
   }
 
   /**

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcClientEngine.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcClientEngine.java?rev=1455764&r1=1455763&r2=1455764&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcClientEngine.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/ProtobufRpcClientEngine.java Wed Mar 13 00:38:19 2013
@@ -44,8 +44,8 @@ public class ProtobufRpcClientEngine imp
 
   protected HBaseClient client;
 
-  public ProtobufRpcClientEngine(Configuration conf) {
-    this.client = new HBaseClient(conf);
+  public ProtobufRpcClientEngine(Configuration conf, String clusterId) {
+    this.client = new HBaseClient(conf, clusterId);
   }
 
   @Override

Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java?rev=1455764&r1=1455763&r2=1455764&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java Wed Mar 13 00:38:19 2013
@@ -327,8 +327,8 @@ public final class HConstants {
   /** name of the file for unique cluster ID */
   public static final String CLUSTER_ID_FILE_NAME = "hbase.id";
 
-  /** Configuration key storing the cluster ID */
-  public static final String CLUSTER_ID = "hbase.cluster.id";
+  /** Default value for cluster ID */
+  public static final String CLUSTER_ID_DEFAULT = "default-cluster";
 
   // Always store the location of the root table's HRegion.
   // This HRegion is never split.

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java?rev=1455764&r1=1455763&r2=1455764&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java Wed Mar 13 00:38:19 2013
@@ -290,6 +290,13 @@ public class TableMapReduceUtil {
   public static void initCredentials(Job job) throws IOException {
     if (User.isHBaseSecurityEnabled(job.getConfiguration())) {
       try {
+        // init credentials for remote cluster
+        String quorumAddress = job.getConfiguration().get(TableOutputFormat.QUORUM_ADDRESS);
+        if (quorumAddress != null) {
+          Configuration peerConf = HBaseConfiguration.create(job.getConfiguration());
+          ZKUtil.applyClusterKeyToConf(peerConf, quorumAddress);
+          User.getCurrent().obtainAuthTokenForJob(peerConf, job);
+        }
         User.getCurrent().obtainAuthTokenForJob(job.getConfiguration(), job);
       } catch (InterruptedException ie) {
         LOG.info("Interrupted obtaining user authentication token");

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java?rev=1455764&r1=1455763&r2=1455764&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java Wed Mar 13 00:38:19 2013
@@ -182,14 +182,17 @@ implements Configurable {
   @Override
   public void setConf(Configuration otherConf) {
     this.conf = HBaseConfiguration.create(otherConf);
+
     String tableName = this.conf.get(OUTPUT_TABLE);
     if(tableName == null || tableName.length() <= 0) {
       throw new IllegalArgumentException("Must specify table name");
     }
+
     String address = this.conf.get(QUORUM_ADDRESS);
-    int zkClientPort = conf.getInt(QUORUM_PORT, 0);
+    int zkClientPort = this.conf.getInt(QUORUM_PORT, 0);
     String serverClass = this.conf.get(REGION_SERVER_CLASS);
     String serverImpl = this.conf.get(REGION_SERVER_IMPL);
+
     try {
       if (address != null) {
         ZKUtil.applyClusterKeyToConf(this.conf, address);
@@ -198,7 +201,7 @@ implements Configurable {
         this.conf.set(HConstants.REGION_SERVER_IMPL, serverImpl);
       }
       if (zkClientPort != 0) {
-        conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkClientPort);
+        this.conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkClientPort);
       }
       this.table = new HTable(this.conf, tableName);
       this.table.setAutoFlush(false);

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1455764&r1=1455763&r2=1455764&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Wed Mar 13 00:38:19 2013
@@ -406,6 +406,11 @@ public class HRegionServer implements Cl
   private final long startcode;
 
   /**
+   * Unique identifier for the cluster we are a part of.
+   */
+  private String clusterId;
+
+  /**
    * MX Bean for RegionServerInfo
    */
   private ObjectName mxBean = null;
@@ -540,7 +545,7 @@ public class HRegionServer implements Cl
   }
 
   String getClusterId() {
-    return this.conf.get(HConstants.CLUSTER_ID);
+    return this.clusterId;
   }
 
   @Retention(RetentionPolicy.RUNTIME)
@@ -759,11 +764,10 @@ public class HRegionServer implements Cl
     // Since cluster status is now up
     // ID should have already been set by HMaster
     try {
-      String clusterId = ZKClusterId.readClusterIdZNode(this.zooKeeper);
+      clusterId = ZKClusterId.readClusterIdZNode(this.zooKeeper);
       if (clusterId == null) {
         this.abort("Cluster ID has not been set");
       }
-      this.conf.set(HConstants.CLUSTER_ID, clusterId);
       LOG.info("ClusterId : "+clusterId);
     } catch (KeeperException e) {
       this.abort("Failed to retrieve Cluster ID",e);
@@ -833,7 +837,7 @@ public class HRegionServer implements Cl
     movedRegionsCleaner = MovedRegionsCleaner.createAndStart(this);
 
     // Setup RPC client for master communication
-    rpcClientEngine = new ProtobufRpcClientEngine(conf);
+    rpcClientEngine = new ProtobufRpcClientEngine(conf, clusterId);
   }
 
   /**

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/RandomTimeoutRpcEngine.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/RandomTimeoutRpcEngine.java?rev=1455764&r1=1455763&r2=1455764&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/RandomTimeoutRpcEngine.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/RandomTimeoutRpcEngine.java Wed Mar 13 00:38:19 2013
@@ -31,6 +31,7 @@ import java.util.concurrent.atomic.Atomi
 
 import javax.net.SocketFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.IpcProtocol;
 import org.apache.hadoop.hbase.security.User;
 
@@ -48,7 +49,7 @@ public class RandomTimeoutRpcEngine exte
   private static AtomicInteger invokations = new AtomicInteger();
 
   public RandomTimeoutRpcEngine(Configuration conf) {
-    super(conf);
+    super(conf, HConstants.CLUSTER_ID_DEFAULT);
   }
 
   @Override

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java?rev=1455764&r1=1455763&r2=1455764&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java Wed Mar 13 00:38:19 2013
@@ -30,6 +30,7 @@ import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.IpcProtocol;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg;
@@ -73,7 +74,8 @@ public class TestDelayedRpc {
         isa.getHostName(), isa.getPort(), 1, 0, true, conf, 0);
     rpcServer.start();
 
-    ProtobufRpcClientEngine clientEngine = new ProtobufRpcClientEngine(conf);
+    ProtobufRpcClientEngine clientEngine =
+        new ProtobufRpcClientEngine(conf, HConstants.CLUSTER_ID_DEFAULT);
     try {
       TestRpc client = clientEngine.getProxy(TestRpc.class,
           rpcServer.getListenerAddress(), conf, 1000);
@@ -142,7 +144,8 @@ public class TestDelayedRpc {
         isa.getHostName(), isa.getPort(), 1, 0, true, conf, 0);
     rpcServer.start();
 
-    ProtobufRpcClientEngine clientEngine = new ProtobufRpcClientEngine(conf);
+    ProtobufRpcClientEngine clientEngine =
+        new ProtobufRpcClientEngine(conf, HConstants.CLUSTER_ID_DEFAULT);
     try {
       TestRpc client = clientEngine.getProxy(TestRpc.class,
           rpcServer.getListenerAddress(), conf, 1000);
@@ -261,7 +264,8 @@ public class TestDelayedRpc {
         isa.getHostName(), isa.getPort(), 1, 0, true, conf, 0);
     rpcServer.start();
 
-    ProtobufRpcClientEngine clientEngine = new ProtobufRpcClientEngine(conf);
+    ProtobufRpcClientEngine clientEngine =
+        new ProtobufRpcClientEngine(conf, HConstants.CLUSTER_ID_DEFAULT);
     try {
       TestRpc client = clientEngine.getProxy(TestRpc.class,
           rpcServer.getListenerAddress(), conf, 1000);

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java?rev=1455764&r1=1455763&r2=1455764&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestIPC.java Wed Mar 13 00:38:19 2013
@@ -34,6 +34,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.IpcProtocol;
 import org.apache.hadoop.hbase.SmallTests;
 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
@@ -83,8 +84,7 @@ public class TestIPC {
     rpcServer.start();
 
     HBaseClient client = new HBaseClient(
-        conf,
-        spyFactory);
+        conf, HConstants.CLUSTER_ID_DEFAULT, spyFactory);
     InetSocketAddress address = rpcServer.getListenerAddress();
 
     try {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java?rev=1455764&r1=1455763&r2=1455764&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestProtoBufRpc.java Wed Mar 13 00:38:19 2013
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MediumTests;
 import org.apache.hadoop.hbase.IpcProtocol;
 import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto;
@@ -101,7 +102,8 @@ public class TestProtoBufRpc {
 
   @Test
   public void testProtoBufRpc() throws Exception {
-    ProtobufRpcClientEngine clientEngine = new ProtobufRpcClientEngine(conf);
+    ProtobufRpcClientEngine clientEngine =
+        new ProtobufRpcClientEngine(conf, HConstants.CLUSTER_ID_DEFAULT);
     try {
       TestRpcService client = clientEngine.getProxy(TestRpcService.class, addr, conf, 10000);
       // Test ping method

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java?rev=1455764&r1=1455763&r2=1455764&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestHMasterRPCException.java Wed Mar 13 00:38:19 2013
@@ -51,7 +51,8 @@ public class TestHMasterRPCException {
 
     ServerName sm = hm.getServerName();
     InetSocketAddress isa = new InetSocketAddress(sm.getHostname(), sm.getPort());
-    ProtobufRpcClientEngine engine = new ProtobufRpcClientEngine(conf);
+    ProtobufRpcClientEngine engine =
+        new ProtobufRpcClientEngine(conf, HConstants.CLUSTER_ID_DEFAULT);
     try {
       int i = 0;
       //retry the RPC a few times; we have seen SocketTimeoutExceptions if we

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java?rev=1455764&r1=1455763&r2=1455764&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClusterId.java Wed Mar 13 00:38:19 2013
@@ -84,7 +84,7 @@ public class TestClusterId {
 
     String clusterId = ZKClusterId.readClusterIdZNode(TEST_UTIL.getZooKeeperWatcher());
     assertNotNull(clusterId);
-    assertEquals(clusterId, rst.getRegionServer().getConfiguration().get(HConstants.CLUSTER_ID));
+    assertEquals(clusterId, rst.getRegionServer().getClusterId());
   }
 }
 

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java?rev=1455764&r1=1455763&r2=1455764&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java Wed Mar 13 00:38:19 2013
@@ -363,9 +363,8 @@ public class TestTokenAuthentication {
     testuser.doAs(new PrivilegedExceptionAction<Object>() {
       public Object run() throws Exception {
         Configuration c = server.getConfiguration();
-        c.set(HConstants.CLUSTER_ID, clusterId.toString());
         ProtobufRpcClientEngine rpcClient =
-            new ProtobufRpcClientEngine(c);
+            new ProtobufRpcClientEngine(c, clusterId.toString());
         try {
           AuthenticationProtos.AuthenticationService.BlockingInterface proxy =
               HBaseClientRPC.waitForProxy(rpcClient, BlockingAuthenticationService.class,