diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java index 8b576d8..c88c1e1 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService; @@ -38,10 +37,9 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService; // classes and unit tests only. public interface ClusterConnection extends HConnection { - /** @return - true if the master server is running */ + /** @return - true if the master server is running and fully initialized. */ @Override - boolean isMasterRunning() - throws MasterNotRunningException, ZooKeeperConnectionException; + boolean isMasterRunning()throws MasterNotRunningException; /** * Use this api to check if the table has been created with the specified number of diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java index c57064b..749ff86 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionAdapter.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService; @@ -117,8 +116,7 @@ class ConnectionAdapter implements ClusterConnection { } @Override - public boolean isMasterRunning() throws MasterNotRunningException, - ZooKeeperConnectionException { + public boolean isMasterRunning() throws MasterNotRunningException { return wrappedConnection.isMasterRunning(); } diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java index ad18e23..f326ef5 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java @@ -65,6 +65,8 @@ import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture; import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase; import org.apache.hadoop.hbase.client.coprocessor.Batch; +import org.apache.hadoop.hbase.coordination.ConfigBasedMasterAddressesProvider; +import org.apache.hadoop.hbase.coordination.MasterAddressesProvider; import org.apache.hadoop.hbase.exceptions.RegionMovedException; import org.apache.hadoop.hbase.exceptions.RegionOpeningException; import org.apache.hadoop.hbase.ipc.RpcClient; @@ -75,6 +77,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse; +import org.apache.hadoop.hbase.protobuf.generated.MasterProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest; @@ -165,11 +168,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ExceptionUtil; import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.ipc.RemoteException; -import org.apache.zookeeper.KeeperException; import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.BlockingRpcChannel; @@ -542,7 +541,9 @@ class ConnectionManager { } } - /** Encapsulates connection to zookeeper and regionservers.*/ + /** + * Encapsulates connection to masters and region servers. + */ @edu.umd.cs.findbugs.annotations.SuppressWarnings( value="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION", justification="Access to the conncurrent hash map is under a lock so should be fine.") @@ -560,17 +561,13 @@ class ConnectionManager { // package protected for the tests ClusterStatusListener clusterStatusListener; - private final Object metaRegionLock = new Object(); - // We have a single lock for master & zk to prevent deadlocks. Having - // one lock for ZK and one lock for master is not possible: - // When creating a connection to master, we need a connection to ZK to get - // its address. But another thread could have taken the ZK lock, and could - // be waiting for the master lock => deadlock. - private final Object masterAndZKLock = new Object(); + // We used to have single lock for connection to both Master and ZooKeeper + // As we don't connect to ZK from client anymore, this lock only controls connections + // to Master + private final Object masterLock = new Object(); - private long keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE; private final DelayedClosing delayedClosing = DelayedClosing.createAndStart(this); @@ -602,6 +599,11 @@ class ConnectionManager { */ Registry registry; + /** + * Implementation of master addresses provider, able to lookup active masters. + */ + MasterAddressesProvider masterAddressesProvider; + HConnectionImplementation(Configuration conf, boolean managed) throws IOException { this(conf, managed, null, null); } @@ -623,10 +625,12 @@ class ConnectionManager { this.user = user; this.batchPool = pool; this.managed = managed; + this.masterAddressesProvider = setupMasterAddressesProvider(); this.registry = setupRegistry(); retrieveClusterId(); - this.rpcClient = new RpcClient(this.conf, this.clusterId); + // TODO: fix this hack, as we're not passing true ClusterId here + this.rpcClient = new RpcClient(this.conf, HConstants.DEFAULT_CLUSTER_ID.toString()); this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(conf); this.rpcControllerFactory = RpcControllerFactory.instantiate(conf); @@ -778,8 +782,8 @@ class ConnectionManager { */ private Registry setupRegistry() throws IOException { String registryClass = this.conf.get("hbase.client.registry.impl", - ZooKeeperRegistry.class.getName()); - Registry registry = null; + RpcRegistry.class.getName()); + Registry registry; try { registry = (Registry)Class.forName(registryClass).newInstance(); } catch (Throwable t) { @@ -790,6 +794,25 @@ class ConnectionManager { } /** + * @return initialized instance of {@link MasterAddressesProvider}, + * which can be used to get current active master. + * @throws IOException + */ + private MasterAddressesProvider setupMasterAddressesProvider() throws IOException { + String masterAddressesProviderClass = this.conf.get(HConstants.MASTERS_ADDRESSES_PROVIDER, + ConfigBasedMasterAddressesProvider.class.getName()); + MasterAddressesProvider masterAddressesProvider; + try { + masterAddressesProvider = (MasterAddressesProvider) + Class.forName(masterAddressesProviderClass).newInstance(); + } catch (Throwable t) { + throw new IOException(t); + } + masterAddressesProvider.init(conf); + return masterAddressesProvider; + } + + /** * For tests only. * @param rpcClient Client we should use instead. * @return Previous rpcClient @@ -821,12 +844,12 @@ class ConnectionManager { protected String clusterId = null; void retrieveClusterId() { - if (clusterId != null) return; + /*if (clusterId != null) return; this.clusterId = this.registry.getClusterId(); - if (clusterId == null) { + if (clusterId == null) {*/ clusterId = HConstants.CLUSTER_ID_DEFAULT; - LOG.debug("clusterid came back null, using default " + clusterId); - } + /*LOG.debug("clusterid came back null, using default " + clusterId); + }*/ } @Override @@ -834,39 +857,23 @@ class ConnectionManager { return this.conf; } - private void checkIfBaseNodeAvailable(ZooKeeperWatcher zkw) - throws MasterNotRunningException { - String errorMsg; - try { - if (ZKUtil.checkExists(zkw, zkw.baseZNode) == -1) { - errorMsg = "The node " + zkw.baseZNode+" is not in ZooKeeper. " - + "It should have been written by the master. " - + "Check the value configured in 'zookeeper.znode.parent'. " - + "There could be a mismatch with the one configured in the master."; - LOG.error(errorMsg); - throw new MasterNotRunningException(errorMsg); - } - } catch (KeeperException e) { - errorMsg = "Can't get connection to ZooKeeper: " + e.getMessage(); - LOG.error(errorMsg); - throw new MasterNotRunningException(errorMsg, e); - } - } - /** - * @return true if the master is running, throws an exception otherwise + * @return true if the master is running and initialized, throws an exception otherwise * @throws MasterNotRunningException - if the master is not running - * @throws ZooKeeperConnectionException */ @Override - public boolean isMasterRunning() - throws MasterNotRunningException, ZooKeeperConnectionException { - // When getting the master connection, we check it's running, - // so if there is no exception, it means we've been able to get a - // connection on a running master + public boolean isMasterRunning() throws MasterNotRunningException { MasterKeepAliveConnection m = getKeepAliveMasterService(); - m.close(); - return true; + + try { + IsMasterRunningResponse response = + m.isMasterRunning(null, RequestConverter.buildIsMasterRunningRequest()); + return response != null? response.getIsMasterRunning(): false; + } catch (ServiceException ex) { + throw new MasterNotRunningException(ex); + } finally { + m.close(); + } } @Override @@ -1103,10 +1110,10 @@ class ConnectionManager { private RegionLocations locateMeta(final TableName tableName, boolean useCache, int replicaId) throws IOException { // HBASE-10785: We cache the location of the META itself, so that we are not overloading - // zookeeper with one request for every region lookup. We cache the META with empty row + // master with one request for every region lookup. We cache the META with empty row // key in MetaCache. byte[] metaCacheKey = HConstants.EMPTY_START_ROW; // use byte[0] as the row for meta - RegionLocations locations = null; + RegionLocations locations; if (useCache) { locations = getCachedLocation(tableName, metaCacheKey); if (locations != null) { @@ -1125,7 +1132,7 @@ class ConnectionManager { } } - // Look up from zookeeper + // Look up meta location (implementation determines where to get it from) locations = this.registry.getMetaRegionLocation(); if (locations != null) { cacheLocation(tableName, locations); @@ -1410,45 +1417,38 @@ class ConnectionManager { * protobuf services nor their interfaces. Let the caller do appropriate casting. * @return A stub for master services. * @throws IOException - * @throws KeeperException * @throws ServiceException */ - private Object makeStubNoRetries() throws IOException, KeeperException, ServiceException { - ZooKeeperKeepAliveConnection zkw; - try { - zkw = getKeepAliveZooKeeperWatcher(); - } catch (IOException e) { - ExceptionUtil.rethrowIfInterrupt(e); - throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e); - } - try { - checkIfBaseNodeAvailable(zkw); - ServerName sn = MasterAddressTracker.getMasterAddress(zkw); - if (sn == null) { - String msg = "ZooKeeper available but no active master location found"; - LOG.info(msg); - throw new MasterNotRunningException(msg); - } - if (isDeadServer(sn)) { - throw new MasterNotRunningException(sn + " is dead."); - } - // Use the security info interface name as our stub key - String key = getStubKey(getServiceName(), sn.getHostAndPort()); - connectionLock.putIfAbsent(key, key); - Object stub = null; - synchronized (connectionLock.get(key)) { - stub = stubs.get(key); - if (stub == null) { - BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(sn, user, rpcTimeout); - stub = makeStub(channel); - isMasterRunning(); - stubs.put(key, stub); + private Object makeStubNoRetries() throws IOException, ServiceException { + ServerName sn = masterAddressesProvider.getActiveMaster(); + if (sn == null) { + String msg = "No active master location found"; + LOG.info(msg); + throw new MasterNotRunningException(msg); + } + if (isDeadServer(sn)) { + throw new MasterNotRunningException(sn + " is dead."); + } + // Use the security info interface name as our stub key + String key = getStubKey(getServiceName(), sn.getHostAndPort()); + connectionLock.putIfAbsent(key, key); + Object stub; + synchronized (connectionLock.get(key)) { + stub = stubs.get(key); + if (stub == null) { + BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(sn, user, rpcTimeout); + stub = makeStub(channel); + try { + // we don't make this check anymore, since we may want to connect to + // HMaster which isn't fully initialized yet. + //isMasterRunning(); + } catch (Throwable e) { + throw new MasterNotRunningException(); } + stubs.put(key, stub); } - return stub; - } finally { - zkw.close(); } + return stub; } /** @@ -1460,7 +1460,7 @@ class ConnectionManager { Object makeStub() throws MasterNotRunningException { // The lock must be at the beginning to prevent multiple master creations // (and leaks) in a multithread context - synchronized (masterAndZKLock) { + synchronized (masterLock) { Exception exceptionCaught = null; Object stub = null; int tries = 0; @@ -1470,8 +1470,6 @@ class ConnectionManager { stub = makeStubNoRetries(); } catch (IOException e) { exceptionCaught = e; - } catch (KeeperException e) { - exceptionCaught = e; } catch (ServiceException e) { exceptionCaught = e; } @@ -1593,48 +1591,13 @@ class ConnectionManager { return serviceName + "@" + rsHostnamePort; } - private ZooKeeperKeepAliveConnection keepAliveZookeeper; - private AtomicInteger keepAliveZookeeperUserCount = new AtomicInteger(0); - private boolean canCloseZKW = true; - // keepAlive time, in ms. No reason to make it configurable. private static final long keepAlive = 5 * 60 * 1000; /** - * Retrieve a shared ZooKeeperWatcher. You must close it it once you've have finished with it. - * @return The shared instance. Never returns null. - */ - ZooKeeperKeepAliveConnection getKeepAliveZooKeeperWatcher() - throws IOException { - synchronized (masterAndZKLock) { - if (keepAliveZookeeper == null) { - if (this.closed) { - throw new IOException(toString() + " closed"); - } - // We don't check that our link to ZooKeeper is still valid - // But there is a retry mechanism in the ZooKeeperWatcher itself - keepAliveZookeeper = new ZooKeeperKeepAliveConnection(conf, this.toString(), this); - } - keepAliveZookeeperUserCount.addAndGet(1); - keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE; - return keepAliveZookeeper; - } - } - - void releaseZooKeeperWatcher(final ZooKeeperWatcher zkw) { - if (zkw == null){ - return; - } - if (keepAliveZookeeperUserCount.addAndGet(-1) <= 0 ){ - keepZooKeeperWatcherAliveUntil = System.currentTimeMillis() + keepAlive; - } - } - - /** - * Creates a Chore thread to check the connections to master & zookeeper - * and close them when they reach their closing time ( - * {@link MasterServiceState#keepAliveUntil} and - * {@link #keepZooKeeperWatcherAliveUntil}). Keep alive time is + * Creates a Chore thread to check connection to master + * and close it when it reach their closing time ( + * {@link MasterServiceState#keepAliveUntil}). Keep alive time is * managed by the release functions and the variable {@link #keepAlive} */ private static class DelayedClosing extends Chore implements Stoppable { @@ -1670,16 +1633,7 @@ class ConnectionManager { @Override protected void chore() { - synchronized (hci.masterAndZKLock) { - if (hci.canCloseZKW) { - if (System.currentTimeMillis() > - hci.keepZooKeeperWatcherAliveUntil) { - - hci.closeZooKeeperWatcher(); - hci.keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE; - } - } - closeMasterProtocol(hci.masterServiceState); + synchronized (hci.masterLock) { closeMasterProtocol(hci.masterServiceState); } } @@ -1695,19 +1649,6 @@ class ConnectionManager { } } - private void closeZooKeeperWatcher() { - synchronized (masterAndZKLock) { - if (keepAliveZookeeper != null) { - LOG.info("Closing zookeeper sessionid=0x" + - Long.toHexString( - keepAliveZookeeper.getRecoverableZooKeeper().getSessionId())); - keepAliveZookeeper.internalClose(); - keepAliveZookeeper = null; - } - keepAliveZookeeperUserCount.set(0); - } - } - final MasterServiceState masterServiceState = new MasterServiceState(this); @Override @@ -1715,6 +1656,10 @@ class ConnectionManager { return getKeepAliveMasterService(); } + MasterAddressesProvider getMasterAddressesProvider() { + return masterAddressesProvider; + } + private void resetMasterServiceState(final MasterServiceState mss) { mss.userCount++; mss.keepAliveUntil = Long.MAX_VALUE; @@ -1723,7 +1668,7 @@ class ConnectionManager { @Override public MasterKeepAliveConnection getKeepAliveMasterService() throws MasterNotRunningException { - synchronized (masterAndZKLock) { + synchronized (masterLock) { if (!isKeepAliveMasterConnectedAndRunning(this.masterServiceState)) { MasterServiceStubMaker stubMaker = new MasterServiceStubMaker(); this.masterServiceState.stub = stubMaker.makeStub(); @@ -1985,6 +1930,16 @@ class ConnectionManager { } @Override + public MasterProtos.GetMetaLocationResponse getMetaRegionLocation(RpcController controller, MasterProtos.GetMetaLocationRequest request) throws ServiceException { + return stub.getMetaRegionLocation(controller, request); + } + + @Override + public MasterProtos.GetTableStateResponse getTableState(RpcController controller, MasterProtos.GetTableStateRequest request) throws ServiceException { + return stub.getTableState(controller, request); + } + + @Override public void close() { release(this.mss); } @@ -2045,7 +2000,7 @@ class ConnectionManager { void releaseMaster(MasterServiceState mss) { if (mss.getStub() == null) return; - synchronized (masterAndZKLock) { + synchronized (masterLock) { --mss.userCount; if (mss.userCount <= 0) { mss.keepAliveUntil = System.currentTimeMillis() + keepAlive; @@ -2066,7 +2021,7 @@ class ConnectionManager { * connection itself. */ private void closeMaster() { - synchronized (masterAndZKLock) { + synchronized (masterLock) { closeMasterService(masterServiceState); } } @@ -2272,26 +2227,14 @@ class ConnectionManager { @Override public void abort(final String msg, Throwable t) { - if (t instanceof KeeperException.SessionExpiredException - && keepAliveZookeeper != null) { - synchronized (masterAndZKLock) { - if (keepAliveZookeeper != null) { - LOG.warn("This client just lost it's session with ZooKeeper," + - " closing it." + - " It will be recreated next time someone needs it", t); - closeZooKeeperWatcher(); - } - } + if (t != null) { + LOG.fatal(msg, t); } else { - if (t != null) { - LOG.fatal(msg, t); - } else { - LOG.fatal(msg); - } - this.aborted = true; - close(); - this.closed = true; + LOG.fatal(msg); } + this.aborted = true; + close(); + this.closed = true; } @Override @@ -2342,7 +2285,6 @@ class ConnectionManager { closeMaster(); shutdownBatchPool(); this.closed = true; - closeZooKeeperWatcher(); this.stubs.clear(); if (clusterStatusListener != null) { clusterStatusListener.close(); diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 10e4d04..f7857d9 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -2392,14 +2392,13 @@ public class HBaseAdmin implements Admin { /** * Check to see if HBase is running. Throw an exception if not. - * We consider that HBase is running if ZooKeeper and Master are running. + * We consider that HBase is running if at least 1 master is running. * * @param conf system configuration * @throws MasterNotRunningException if the master is not running - * @throws ZooKeeperConnectionException if unable to connect to zookeeper */ public static void checkHBaseAvailable(Configuration conf) - throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException, IOException { + throws MasterNotRunningException, ServiceException, IOException { Configuration copyOfConf = HBaseConfiguration.create(conf); // We set it to make it fail as soon as possible if HBase is not available @@ -2411,27 +2410,7 @@ public class HBaseAdmin implements Admin { HConnectionManager.getConnection(copyOfConf); try { - // Check ZK first. - // If the connection exists, we may have a connection to ZK that does - // not work anymore - ZooKeeperKeepAliveConnection zkw = null; - try { - zkw = connection.getKeepAliveZooKeeperWatcher(); - zkw.getRecoverableZooKeeper().getZooKeeper().exists( - zkw.baseZNode, false); - - } catch (IOException e) { - throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e); - } catch (InterruptedException e) { - throw (InterruptedIOException) - new InterruptedIOException("Can't connect to ZooKeeper").initCause(e); - } catch (KeeperException e) { - throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e); - } finally { - if (zkw != null) { - zkw.close(); - } - } + // We used to check connection to ZooKeeper here too. Not anymore. // Check Master connection.isMasterRunning(); diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java index 77e9a5e..e75d753 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java @@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService; @@ -167,8 +166,7 @@ public interface HConnection extends Abortable, Closeable { /** @return - true if the master server is running * @deprecated internal method, do not use thru HConnection */ @Deprecated - boolean isMasterRunning() - throws MasterNotRunningException, ZooKeeperConnectionException; + boolean isMasterRunning() throws MasterNotRunningException; /** * A table that isTableEnabled == false and isTableDisabled == false diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionKey.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionKey.java index c9d99b2..79b9c11 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionKey.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionKey.java @@ -40,10 +40,9 @@ import org.apache.hadoop.hbase.security.UserProvider; */ class HConnectionKey { final static String[] CONNECTION_PROPERTIES = new String[] { - HConstants.ZOOKEEPER_QUORUM, HConstants.ZOOKEEPER_ZNODE_PARENT, - HConstants.ZOOKEEPER_CLIENT_PORT, - HConstants.ZOOKEEPER_RECOVERABLE_WAITTIME, - HConstants.HBASE_CLIENT_PAUSE, HConstants.HBASE_CLIENT_RETRIES_NUMBER, + HConstants.MASTERS_QUORUM, + HConstants.HBASE_CLIENT_PAUSE, + HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.HBASE_META_SCANNER_CACHING, HConstants.HBASE_CLIENT_INSTANCE_ID, diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRegistry.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRegistry.java new file mode 100644 index 0000000..a8b3d4f --- /dev/null +++ hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRegistry.java @@ -0,0 +1,139 @@ +/* + * + * 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.client; + +import com.google.protobuf.ServiceException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.ClusterStatus; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.RequestConverter; +import org.apache.hadoop.hbase.protobuf.generated.MasterProtos; +import org.apache.hadoop.hbase.rest.client.Client; +import org.apache.hadoop.hbase.rest.client.Cluster; + +import java.io.IOException; + +import static org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table; + +/** + * Registry implementation which reads information making RPC calls to HBase master + * (except reading public ClusterId, which is done via HTTP endpoint to bypass authentication). + */ +public class RpcRegistry implements Registry { + static final Log LOG = LogFactory.getLog(RpcRegistry.class); + + private static final String MIMETYPE_TEXT = "text/plain"; + + // Needs an instance of hci to function. Set after construct this instance. + ConnectionManager.HConnectionImplementation hci; + + @Override + public void init(HConnection connection) { + if (!(connection instanceof ConnectionManager.HConnectionImplementation)) { + throw new RuntimeException("This registry depends on HConnectionImplementation"); + } + this.hci = (ConnectionManager.HConnectionImplementation)connection; + } + + @Override + public RegionLocations getMetaRegionLocation() throws IOException { + MasterKeepAliveConnection master = hci.getKeepAliveMasterService(); + try { + MasterProtos.GetMetaLocationRequest req = + RequestConverter.buildGetMetaLocationRequest(); + ServerName serverName = ProtobufUtil.toServerName( + master.getMetaRegionLocation(null, req).getMetaLocation()); + + if (LOG.isTraceEnabled()) { + LOG.trace("Looked up meta region location, connection=" + this + + "; serverName=" + ((serverName == null) ? "null" : serverName)); + } + if (serverName == null) return null; + HRegionLocation loc = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, serverName, 0); + return new RegionLocations(new HRegionLocation[] {loc}); + } catch (ServiceException se) { + throw ProtobufUtil.getRemoteException(se); + } finally { + master.close(); + } + } + + @Override + public String getClusterId() { + ServerName master = hci.getMasterAddressesProvider().getActiveMaster(); + int masterInfoPort = hci.getConfiguration().getInt(HConstants.MASTER_INFO_PORT, + HConstants.DEFAULT_MASTER_INFOPORT); + Cluster cluster = new Cluster().add(master.getHostname(), masterInfoPort); + Client client = new Client(cluster); + try { + return new String(client.get("/clusterId", MIMETYPE_TEXT).getBody()); + } catch (IOException e) { + e.printStackTrace(); + } + return null; + } + + @Override + public boolean isTableOnlineState(TableName tableName, boolean enabled) throws IOException { + MasterKeepAliveConnection master = hci.getKeepAliveMasterService(); + try { + MasterProtos.GetTableStateRequest req = + RequestConverter.buildGetTableStateRequest(tableName); + Table.State state = master.getTableState(null, req).getTableState().getState(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Master reported, that table " + tableName.getNameAsString() + " is " + state); + } + + return enabled? Table.State.ENABLED.equals(state) : Table.State.DISABLED.equals(state); + } catch (ServiceException se) { + throw ProtobufUtil.getRemoteException(se); + } finally { + master.close(); + } + } + + @Override + public int getCurrentNrHRS() throws IOException { + MasterKeepAliveConnection master = hci.getKeepAliveMasterService(); + try { + MasterProtos.GetClusterStatusRequest req = + RequestConverter.buildGetClusterStatusRequest(); + int numRSs = ClusterStatus.convert(master.getClusterStatus(null, req). + getClusterStatus()).getServersSize(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Master reported " + numRSs + " live region servers in the cluster"); + } + return numRSs; + } catch (ServiceException se) { + throw ProtobufUtil.getRemoteException(se); + } finally { + master.close(); + } + } +} diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java deleted file mode 100644 index 9b987b5..0000000 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java +++ /dev/null @@ -1,56 +0,0 @@ -/** - * - * 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.client; - - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; - -import java.io.IOException; - -/** - * We inherit the current ZooKeeperWatcher implementation to change the semantic - * of the close: the new close won't immediately close the connection but - * will have a keep alive. See {@link HConnection}. - * This allows to make it available with a consistent interface. The whole - * ZooKeeperWatcher use in HConnection will be then changed to remove the - * watcher part. - * - * This class is intended to be used internally by HBase classes; but not by - * final user code. Hence it's package protected. - */ -class ZooKeeperKeepAliveConnection extends ZooKeeperWatcher{ - ZooKeeperKeepAliveConnection( - Configuration conf, String descriptor, - ConnectionManager.HConnectionImplementation conn) throws IOException { - super(conf, descriptor, conn); - } - - @Override - public void close() { - if (this.abortable != null) { - ((ConnectionManager.HConnectionImplementation)abortable).releaseZooKeeperWatcher(this); - } - } - - void internalClose(){ - super.close(); - } -} diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java deleted file mode 100644 index 9123d50..0000000 --- hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperRegistry.java +++ /dev/null @@ -1,131 +0,0 @@ -/** - * 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.client; - -import java.io.IOException; -import java.io.InterruptedIOException; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; -import org.apache.hadoop.hbase.zookeeper.ZKClusterId; -import org.apache.hadoop.hbase.zookeeper.ZKTableStateClientSideReader; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.zookeeper.KeeperException; - -/** - * A cluster registry that stores to zookeeper. - */ -class ZooKeeperRegistry implements Registry { - static final Log LOG = LogFactory.getLog(ZooKeeperRegistry.class); - // Needs an instance of hci to function. Set after construct this instance. - ConnectionManager.HConnectionImplementation hci; - - @Override - public void init(HConnection connection) { - if (!(connection instanceof ConnectionManager.HConnectionImplementation)) { - throw new RuntimeException("This registry depends on HConnectionImplementation"); - } - this.hci = (ConnectionManager.HConnectionImplementation)connection; - } - - @Override - public RegionLocations getMetaRegionLocation() throws IOException { - ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher(); - - try { - if (LOG.isTraceEnabled()) { - LOG.trace("Looking up meta region location in ZK," + " connection=" + this); - } - ServerName servername = new MetaTableLocator().blockUntilAvailable(zkw, hci.rpcTimeout); - if (LOG.isTraceEnabled()) { - LOG.trace("Looked up meta region location, connection=" + this + - "; serverName=" + ((servername == null) ? "null" : servername)); - } - if (servername == null) return null; - HRegionLocation loc = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, servername, 0); - return new RegionLocations(new HRegionLocation[] {loc}); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - return null; - } finally { - zkw.close(); - } - } - - private String clusterId = null; - - @Override - public String getClusterId() { - if (this.clusterId != null) return this.clusterId; - // No synchronized here, worse case we will retrieve it twice, that's - // not an issue. - ZooKeeperKeepAliveConnection zkw = null; - try { - zkw = hci.getKeepAliveZooKeeperWatcher(); - this.clusterId = ZKClusterId.readClusterIdZNode(zkw); - if (this.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(); - } - return this.clusterId; - } - - @Override - public boolean isTableOnlineState(TableName tableName, boolean enabled) - throws IOException { - ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher(); - try { - if (enabled) { - return ZKTableStateClientSideReader.isEnabledTable(zkw, tableName); - } - return ZKTableStateClientSideReader.isDisabledTable(zkw, tableName); - } catch (KeeperException e) { - throw new IOException("Enable/Disable failed", e); - } catch (InterruptedException e) { - throw new InterruptedIOException(); - } finally { - zkw.close(); - } - } - - @Override - public int getCurrentNrHRS() throws IOException { - ZooKeeperKeepAliveConnection zkw = hci.getKeepAliveZooKeeperWatcher(); - try { - // We go to zk rather than to master to get count of regions to avoid - // HTable having a Master dependency. See HBase-2828 - return ZKUtil.getNumberOfChildren(zkw, zkw.rsZNode); - } catch (KeeperException ke) { - throw new IOException("Unexpected ZooKeeper exception", ke); - } finally { - zkw.close(); - } - } -} \ No newline at end of file diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/coordination/ConfigBasedMasterAddressesProvider.java hbase-client/src/main/java/org/apache/hadoop/hbase/coordination/ConfigBasedMasterAddressesProvider.java new file mode 100644 index 0000000..6071735 --- /dev/null +++ hbase-client/src/main/java/org/apache/hadoop/hbase/coordination/ConfigBasedMasterAddressesProvider.java @@ -0,0 +1,79 @@ +/* + * + * 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.coordination; + +import org.apache.hadoop.HadoopIllegalArgumentException; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ServerName; + +import java.util.ArrayList; +import java.util.List; + +/** + * This implementation of {@link MasterAddressesProvider} loads list of masters + * from hbase-site.xml. + */ +@InterfaceAudience.Private +public class ConfigBasedMasterAddressesProvider implements MasterAddressesProvider { + private List masters; + private int masterPort; + + private int index = 0; + + @Override + public void init(Configuration conf) { + masterPort = conf.getInt(HConstants.MASTER_PORT, HConstants.DEFAULT_MASTER_PORT); + String mastersQuorumString = conf.get(HConstants.MASTERS_QUORUM); + + if (mastersQuorumString == null || mastersQuorumString.isEmpty()) { + throw new HadoopIllegalArgumentException("Property " + + HConstants.MASTERS_QUORUM + " isn't defined in configuration"); + } + + if (mastersQuorumString.matches(".*\\s.*")) { + throw new HadoopIllegalArgumentException("Property " + + HConstants.MASTERS_QUORUM + " must not contain whitespaces"); + } + masters = new ArrayList(); + + for (String m : mastersQuorumString.split(",")) { + masters.add(m); + } + } + + @Override + public ServerName getActiveMaster() { + ServerName sn; + + // simplest round-robin + synchronized (this) { + sn = ServerName.valueOf(masters.get(index), masterPort, 0L); + index = (index + 1) % masters.size(); + } + return sn; + } + + @Override + public List getAllMasters() { + return masters; + } +} diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/coordination/MasterAddressesProvider.java hbase-client/src/main/java/org/apache/hadoop/hbase/coordination/MasterAddressesProvider.java new file mode 100644 index 0000000..5367f99 --- /dev/null +++ hbase-client/src/main/java/org/apache/hadoop/hbase/coordination/MasterAddressesProvider.java @@ -0,0 +1,56 @@ +/* + * + * 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.coordination; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ServerName; + +import java.util.List; + +/** + * Implementations of this interface can obtain the list of masters running in + * HBase cluster (from config file, DNS, or some other source) and select the + * next active master that client can connect to. + */ +@InterfaceAudience.Private +public interface MasterAddressesProvider { + + /** + * Initialize addresses provider. + * @param conf + */ + void init(Configuration conf); + + /** + * @return next active master client can connect to. Returns: + * - single active masters if cluster is running in active-backups configuration + * - next active master in the list, if running in multiple active masters configuration + * (no specific ordering is guaranteed) + * - null, if no active master found. + */ + ServerName getActiveMaster(); + + /** + * @return List of all masters in HBase cluster, each entry is address of + * individual master (may be ip or domain address), or empty list is no masters found. + */ + List getAllMasters(); +} diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java index e7d0934..2cb655a 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java @@ -20,9 +20,10 @@ package org.apache.hadoop.hbase.protobuf; import java.io.IOException; import java.util.List; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.protobuf.generated.MasterProtos; import org.apache.hadoop.hbase.util.ByteStringer; -import org.apache.commons.configuration.Configuration; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.CellScannable; import org.apache.hadoop.hbase.DoNotRetryIOException; @@ -1515,4 +1516,26 @@ public final class RequestConverter { } return builder.build(); } + + /** + * Creates a protocol buffer GetMetaLocationRequest + * + * @return a GetMetaLocationRequest + */ + public static MasterProtos.GetMetaLocationRequest buildGetMetaLocationRequest() { + return MasterProtos.GetMetaLocationRequest.newBuilder().build(); + } + + /** + * Creates a protocol buffer GetTableStateRequest + * + * @return a GetTableStateRequest + */ + public static MasterProtos.GetTableStateRequest buildGetTableStateRequest(TableName tableName) { + HBaseProtos.TableName protoTableName = HBaseProtos.TableName.newBuilder(). + setNamespace(ByteStringer.wrap(tableName.getNamespace())). + setQualifier(ByteStringer.wrap(tableName.getQualifier())).build(); + + return MasterProtos.GetTableStateRequest.newBuilder().setTableName(protoTableName).build(); + } } diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java hbase-client/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java new file mode 100644 index 0000000..9382944 --- /dev/null +++ hbase-client/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java @@ -0,0 +1,506 @@ +/* + * + * 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.rest.client; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.commons.httpclient.Header; +import org.apache.commons.httpclient.HttpClient; +import org.apache.commons.httpclient.HttpMethod; +import org.apache.commons.httpclient.HttpVersion; +import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager; +import org.apache.commons.httpclient.URI; +import org.apache.commons.httpclient.methods.ByteArrayRequestEntity; +import org.apache.commons.httpclient.methods.DeleteMethod; +import org.apache.commons.httpclient.methods.GetMethod; +import org.apache.commons.httpclient.methods.HeadMethod; +import org.apache.commons.httpclient.methods.PostMethod; +import org.apache.commons.httpclient.methods.PutMethod; +import org.apache.commons.httpclient.params.HttpClientParams; +import org.apache.commons.httpclient.params.HttpConnectionManagerParams; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * A wrapper around HttpClient which provides some useful function and + * semantics for interacting with the REST gateway. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class Client { + public static final Header[] EMPTY_HEADER_ARRAY = new Header[0]; + + private static final Log LOG = LogFactory.getLog(Client.class); + + private HttpClient httpClient; + private Cluster cluster; + + private Map extraHeaders; + + /** + * Default Constructor + */ + public Client() { + this(null); + } + + /** + * Constructor + * @param cluster the cluster definition + */ + public Client(Cluster cluster) { + this.cluster = cluster; + MultiThreadedHttpConnectionManager manager = + new MultiThreadedHttpConnectionManager(); + HttpConnectionManagerParams managerParams = manager.getParams(); + managerParams.setConnectionTimeout(2000); // 2 s + managerParams.setDefaultMaxConnectionsPerHost(10); + managerParams.setMaxTotalConnections(100); + extraHeaders = new ConcurrentHashMap(); + this.httpClient = new HttpClient(manager); + HttpClientParams clientParams = httpClient.getParams(); + clientParams.setVersion(HttpVersion.HTTP_1_1); + } + + /** + * Shut down the client. Close any open persistent connections. + */ + public void shutdown() { + MultiThreadedHttpConnectionManager manager = + (MultiThreadedHttpConnectionManager) httpClient.getHttpConnectionManager(); + manager.shutdown(); + } + + /** + * @return the wrapped HttpClient + */ + public HttpClient getHttpClient() { + return httpClient; + } + + /** + * Add extra headers. These extra headers will be applied to all http + * methods before they are removed. If any header is not used any more, + * client needs to remove it explicitly. + */ + public void addExtraHeader(final String name, final String value) { + extraHeaders.put(name, value); + } + + /** + * Get an extra header value. + */ + public String getExtraHeader(final String name) { + return extraHeaders.get(name); + } + + /** + * Get all extra headers (read-only). + */ + public Map getExtraHeaders() { + return Collections.unmodifiableMap(extraHeaders); + } + + /** + * Remove an extra header. + */ + public void removeExtraHeader(final String name) { + extraHeaders.remove(name); + } + + /** + * Execute a transaction method given only the path. Will select at random + * one of the members of the supplied cluster definition and iterate through + * the list until a transaction can be successfully completed. The + * definition of success here is a complete HTTP transaction, irrespective + * of result code. + * @param cluster the cluster definition + * @param method the transaction method + * @param headers HTTP header values to send + * @param path the properly urlencoded path + * @return the HTTP response code + * @throws IOException + */ + public int executePathOnly(Cluster cluster, HttpMethod method, + Header[] headers, String path) throws IOException { + IOException lastException; + if (cluster.nodes.size() < 1) { + throw new IOException("Cluster is empty"); + } + int start = (int)Math.round((cluster.nodes.size() - 1) * Math.random()); + int i = start; + do { + cluster.lastHost = cluster.nodes.get(i); + try { + StringBuilder sb = new StringBuilder(); + sb.append("http://"); + sb.append(cluster.lastHost); + sb.append(path); + URI uri = new URI(sb.toString(), true); + return executeURI(method, headers, uri.toString()); + } catch (IOException e) { + lastException = e; + } + } while (++i != start && i < cluster.nodes.size()); + throw lastException; + } + + /** + * Execute a transaction method given a complete URI. + * @param method the transaction method + * @param headers HTTP header values to send + * @param uri a properly urlencoded URI + * @return the HTTP response code + * @throws IOException + */ + public int executeURI(HttpMethod method, Header[] headers, String uri) + throws IOException { + method.setURI(new URI(uri, true)); + for (Map.Entry e: extraHeaders.entrySet()) { + method.addRequestHeader(e.getKey(), e.getValue()); + } + if (headers != null) { + for (Header header: headers) { + method.addRequestHeader(header); + } + } + long startTime = System.currentTimeMillis(); + int code = httpClient.executeMethod(method); + long endTime = System.currentTimeMillis(); + if (LOG.isDebugEnabled()) { + LOG.debug(method.getName() + " " + uri + " " + code + " " + + method.getStatusText() + " in " + (endTime - startTime) + " ms"); + } + return code; + } + + /** + * Execute a transaction method. Will call either executePathOnly + * or executeURI depending on whether a path only is supplied in + * 'path', or if a complete URI is passed instead, respectively. + * @param cluster the cluster definition + * @param method the HTTP method + * @param headers HTTP header values to send + * @param path the properly urlencoded path or URI + * @return the HTTP response code + * @throws IOException + */ + public int execute(Cluster cluster, HttpMethod method, Header[] headers, + String path) throws IOException { + if (path.startsWith("/")) { + return executePathOnly(cluster, method, headers, path); + } + return executeURI(method, headers, path); + } + + /** + * @return the cluster definition + */ + public Cluster getCluster() { + return cluster; + } + + /** + * @param cluster the cluster definition + */ + public void setCluster(Cluster cluster) { + this.cluster = cluster; + } + + /** + * Send a HEAD request + * @param path the path or URI + * @return a Response object with response detail + * @throws IOException + */ + public Response head(String path) throws IOException { + return head(cluster, path, null); + } + + /** + * Send a HEAD request + * @param cluster the cluster definition + * @param path the path or URI + * @param headers the HTTP headers to include in the request + * @return a Response object with response detail + * @throws IOException + */ + public Response head(Cluster cluster, String path, Header[] headers) + throws IOException { + HeadMethod method = new HeadMethod(); + try { + int code = execute(cluster, method, null, path); + headers = method.getResponseHeaders(); + return new Response(code, headers, null); + } finally { + method.releaseConnection(); + } + } + + /** + * Send a GET request + * @param path the path or URI + * @return a Response object with response detail + * @throws IOException + */ + public Response get(String path) throws IOException { + return get(cluster, path); + } + + /** + * Send a GET request + * @param cluster the cluster definition + * @param path the path or URI + * @return a Response object with response detail + * @throws IOException + */ + public Response get(Cluster cluster, String path) throws IOException { + return get(cluster, path, EMPTY_HEADER_ARRAY); + } + + /** + * Send a GET request + * @param path the path or URI + * @param accept Accept header value + * @return a Response object with response detail + * @throws IOException + */ + public Response get(String path, String accept) throws IOException { + return get(cluster, path, accept); + } + + /** + * Send a GET request + * @param cluster the cluster definition + * @param path the path or URI + * @param accept Accept header value + * @return a Response object with response detail + * @throws IOException + */ + public Response get(Cluster cluster, String path, String accept) + throws IOException { + Header[] headers = new Header[1]; + headers[0] = new Header("Accept", accept); + return get(cluster, path, headers); + } + + /** + * Send a GET request + * @param path the path or URI + * @param headers the HTTP headers to include in the request, + * Accept must be supplied + * @return a Response object with response detail + * @throws IOException + */ + public Response get(String path, Header[] headers) throws IOException { + return get(cluster, path, headers); + } + + /** + * Send a GET request + * @param c the cluster definition + * @param path the path or URI + * @param headers the HTTP headers to include in the request + * @return a Response object with response detail + * @throws IOException + */ + public Response get(Cluster c, String path, Header[] headers) + throws IOException { + GetMethod method = new GetMethod(); + try { + int code = execute(c, method, headers, path); + headers = method.getResponseHeaders(); + byte[] body = method.getResponseBody(); + InputStream in = method.getResponseBodyAsStream(); + return new Response(code, headers, body, in); + } finally { + method.releaseConnection(); + } + } + + /** + * Send a PUT request + * @param path the path or URI + * @param contentType the content MIME type + * @param content the content bytes + * @return a Response object with response detail + * @throws IOException + */ + public Response put(String path, String contentType, byte[] content) + throws IOException { + return put(cluster, path, contentType, content); + } + + /** + * Send a PUT request + * @param cluster the cluster definition + * @param path the path or URI + * @param contentType the content MIME type + * @param content the content bytes + * @return a Response object with response detail + * @throws IOException + */ + public Response put(Cluster cluster, String path, String contentType, + byte[] content) throws IOException { + Header[] headers = new Header[1]; + headers[0] = new Header("Content-Type", contentType); + return put(cluster, path, headers, content); + } + + /** + * Send a PUT request + * @param path the path or URI + * @param headers the HTTP headers to include, Content-Type must be + * supplied + * @param content the content bytes + * @return a Response object with response detail + * @throws IOException + */ + public Response put(String path, Header[] headers, byte[] content) + throws IOException { + return put(cluster, path, headers, content); + } + + /** + * Send a PUT request + * @param cluster the cluster definition + * @param path the path or URI + * @param headers the HTTP headers to include, Content-Type must be + * supplied + * @param content the content bytes + * @return a Response object with response detail + * @throws IOException + */ + public Response put(Cluster cluster, String path, Header[] headers, + byte[] content) throws IOException { + PutMethod method = new PutMethod(); + try { + method.setRequestEntity(new ByteArrayRequestEntity(content)); + int code = execute(cluster, method, headers, path); + headers = method.getResponseHeaders(); + content = method.getResponseBody(); + return new Response(code, headers, content); + } finally { + method.releaseConnection(); + } + } + + /** + * Send a POST request + * @param path the path or URI + * @param contentType the content MIME type + * @param content the content bytes + * @return a Response object with response detail + * @throws IOException + */ + public Response post(String path, String contentType, byte[] content) + throws IOException { + return post(cluster, path, contentType, content); + } + + /** + * Send a POST request + * @param cluster the cluster definition + * @param path the path or URI + * @param contentType the content MIME type + * @param content the content bytes + * @return a Response object with response detail + * @throws IOException + */ + public Response post(Cluster cluster, String path, String contentType, + byte[] content) throws IOException { + Header[] headers = new Header[1]; + headers[0] = new Header("Content-Type", contentType); + return post(cluster, path, headers, content); + } + + /** + * Send a POST request + * @param path the path or URI + * @param headers the HTTP headers to include, Content-Type must be + * supplied + * @param content the content bytes + * @return a Response object with response detail + * @throws IOException + */ + public Response post(String path, Header[] headers, byte[] content) + throws IOException { + return post(cluster, path, headers, content); + } + + /** + * Send a POST request + * @param cluster the cluster definition + * @param path the path or URI + * @param headers the HTTP headers to include, Content-Type must be + * supplied + * @param content the content bytes + * @return a Response object with response detail + * @throws IOException + */ + public Response post(Cluster cluster, String path, Header[] headers, + byte[] content) throws IOException { + PostMethod method = new PostMethod(); + try { + method.setRequestEntity(new ByteArrayRequestEntity(content)); + int code = execute(cluster, method, headers, path); + headers = method.getResponseHeaders(); + content = method.getResponseBody(); + return new Response(code, headers, content); + } finally { + method.releaseConnection(); + } + } + + /** + * Send a DELETE request + * @param path the path or URI + * @return a Response object with response detail + * @throws IOException + */ + public Response delete(String path) throws IOException { + return delete(cluster, path); + } + + /** + * Send a DELETE request + * @param cluster the cluster definition + * @param path the path or URI + * @return a Response object with response detail + * @throws IOException + */ + public Response delete(Cluster cluster, String path) throws IOException { + DeleteMethod method = new DeleteMethod(); + try { + int code = execute(cluster, method, null, path); + Header[] headers = method.getResponseHeaders(); + byte[] content = method.getResponseBody(); + return new Response(code, headers, content); + } finally { + method.releaseConnection(); + } + } +} diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java hbase-client/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java new file mode 100644 index 0000000..83eee01 --- /dev/null +++ hbase-client/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java @@ -0,0 +1,103 @@ +/* + * + * 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.rest.client; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * A list of 'host:port' addresses of HTTP servers operating as a single + * entity, for example multiple redundant web service gateways. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class Cluster { + protected List nodes = + Collections.synchronizedList(new ArrayList()); + protected String lastHost; + + /** + * Constructor + */ + public Cluster() {} + + /** + * Constructor + * @param nodes a list of service locations, in 'host:port' format + */ + public Cluster(List nodes) { + nodes.addAll(nodes); + } + + /** + * @return true if no locations have been added, false otherwise + */ + public boolean isEmpty() { + return nodes.isEmpty(); + } + + /** + * Add a node to the cluster + * @param node the service location in 'host:port' format + */ + public Cluster add(String node) { + nodes.add(node); + return this; + } + + /** + * Add a node to the cluster + * @param name host name + * @param port service port + */ + public Cluster add(String name, int port) { + StringBuilder sb = new StringBuilder(); + sb.append(name); + sb.append(':'); + sb.append(port); + return add(sb.toString()); + } + + /** + * Remove a node from the cluster + * @param node the service location in 'host:port' format + */ + public Cluster remove(String node) { + nodes.remove(node); + return this; + } + + /** + * Remove a node from the cluster + * @param name host name + * @param port service port + */ + public Cluster remove(String name, int port) { + StringBuilder sb = new StringBuilder(); + sb.append(name); + sb.append(':'); + sb.append(port); + return remove(sb.toString()); + } +} diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java hbase-client/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java new file mode 100644 index 0000000..6e617e8 --- /dev/null +++ hbase-client/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java @@ -0,0 +1,155 @@ +/* + * + * 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.rest.client; + +import java.io.InputStream; + +import org.apache.commons.httpclient.Header; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * The HTTP result code, response headers, and body of a HTTP response. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class Response { + private int code; + private Header[] headers; + private byte[] body; + private InputStream stream; + + /** + * Constructor + * @param code the HTTP response code + */ + public Response(int code) { + this(code, null, null); + } + + /** + * Constructor + * @param code the HTTP response code + * @param headers the HTTP response headers + */ + public Response(int code, Header[] headers) { + this(code, headers, null); + } + + /** + * Constructor + * @param code the HTTP response code + * @param headers the HTTP response headers + * @param body the response body, can be null + */ + public Response(int code, Header[] headers, byte[] body) { + this.code = code; + this.headers = headers; + this.body = body; + } + + /** + * Constructor + * @param code the HTTP response code + * @param headers headers the HTTP response headers + * @param body the response body, can be null + * @param in Inputstream if the response had one. + */ + public Response(int code, Header[] headers, byte[] body, InputStream in) { + this.code = code; + this.headers = headers; + this.body = body; + this.stream = in; + } + + /** + * @return the HTTP response code + */ + public int getCode() { + return code; + } + + /** + * Gets the input stream instance. + * + * @return an instance of InputStream class. + */ + public InputStream getStream(){ + return this.stream; + } + + /** + * @return the HTTP response headers + */ + public Header[] getHeaders() { + return headers; + } + + public String getHeader(String key) { + for (Header header: headers) { + if (header.getName().equalsIgnoreCase(key)) { + return header.getValue(); + } + } + return null; + } + + /** + * @return the value of the Location header + */ + public String getLocation() { + return getHeader("Location"); + } + + /** + * @return true if a response body was sent + */ + public boolean hasBody() { + return body != null; + } + + /** + * @return the HTTP response body + */ + public byte[] getBody() { + return body; + } + + /** + * @param code the HTTP response code + */ + public void setCode(int code) { + this.code = code; + } + + /** + * @param headers the HTTP response headers + */ + public void setHeaders(Header[] headers) { + this.headers = headers; + } + + /** + * @param body the response body + */ + public void setBody(byte[] body) { + this.body = body; + } +} diff --git hbase-client/src/test/java/org/apache/hadoop/hbase/coordination/TestConfigBasedMasterAddressesProvider.java hbase-client/src/test/java/org/apache/hadoop/hbase/coordination/TestConfigBasedMasterAddressesProvider.java new file mode 100644 index 0000000..3fb1a0e --- /dev/null +++ hbase-client/src/test/java/org/apache/hadoop/hbase/coordination/TestConfigBasedMasterAddressesProvider.java @@ -0,0 +1,94 @@ +/* + * + * 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.coordination; + +import org.apache.hadoop.HadoopIllegalArgumentException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.SmallTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.List; + +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.core.Is.is; +import static org.hamcrest.core.IsNot.not; +import static org.junit.Assert.assertThat; + +/** + * Tests {@link ConfigBasedMasterAddressesProvider} + */ +@Category(SmallTests.class) +public class TestConfigBasedMasterAddressesProvider { + + @Test(expected = HadoopIllegalArgumentException.class) + public void testExceptionIsThrownIfQuorumPropertyIsNotDefined() { + Configuration conf = new Configuration(); + MasterAddressesProvider masterAddressesProvider = new ConfigBasedMasterAddressesProvider(); + masterAddressesProvider.init(conf); + } + + @Test(expected = HadoopIllegalArgumentException.class) + public void testExceptionIsThrownIfQuorumPropertyContainsWhitespaces() { + Configuration conf = new Configuration(); + conf.set(HConstants.MASTERS_QUORUM, "localhost1, localhost2"); + MasterAddressesProvider masterAddressesProvider = new ConfigBasedMasterAddressesProvider(); + masterAddressesProvider.init(conf); + } + + @Test + public void testGetAllMasters() { + Configuration conf = new Configuration(); + conf.set(HConstants.MASTERS_QUORUM, "localhost1,localhost2"); + MasterAddressesProvider masterAddressesProvider = new ConfigBasedMasterAddressesProvider(); + masterAddressesProvider.init(conf); + List masters = masterAddressesProvider.getAllMasters(); + assertThat(masters, not(nullValue())); + assertThat(masters.size(), is(2)); + } + + @Test + public void testRoundRobinWithSingleMaster() { + Configuration conf = new Configuration(); + conf.set(HConstants.MASTERS_QUORUM, "localhost"); + MasterAddressesProvider masterAddressesProvider = new ConfigBasedMasterAddressesProvider(); + masterAddressesProvider.init(conf); + ServerName master1 = masterAddressesProvider.getActiveMaster(); + ServerName master2 = masterAddressesProvider.getActiveMaster(); + assertThat(master1, is(master2)); + } + + @Test + public void testRoundRobinWithMultipleMasters() { + Configuration conf = new Configuration(); + conf.set(HConstants.MASTERS_QUORUM, "localhost1,localhost2"); + MasterAddressesProvider masterAddressesProvider = new ConfigBasedMasterAddressesProvider(); + masterAddressesProvider.init(conf); + ServerName master1 = masterAddressesProvider.getActiveMaster(); + ServerName master2 = masterAddressesProvider.getActiveMaster(); + ServerName master1Again = masterAddressesProvider.getActiveMaster(); + ServerName master2Again = masterAddressesProvider.getActiveMaster(); + assertThat(master1, is(not(master2))); + assertThat(master1, is(master1Again)); + assertThat(master2, is(master2Again)); + } +} diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 93209fd..3116ffd 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1017,6 +1017,14 @@ public final class HConstants { public static final String HBASE_COORDINATED_STATE_MANAGER_CLASS = "hbase.coordinated.state.manager.class"; + /** Name of HBase masters quorum configuration parameter. */ + public static final String MASTERS_QUORUM = "hbase.masters.quorum"; + + /** + * Configuration key for setting masters addresses provider class name. + */ + public static final String MASTERS_ADDRESSES_PROVIDER = "hbase.masters.addresses.provider"; + private HConstants() { // Can't be instantiated with this ctor. } diff --git hbase-common/src/main/resources/hbase-default.xml hbase-common/src/main/resources/hbase-default.xml index d7bdfc7..627412f 100644 --- hbase-common/src/main/resources/hbase-default.xml +++ hbase-common/src/main/resources/hbase-default.xml @@ -1356,4 +1356,23 @@ possible configurations would overwhelm and obscure the important. hbase.http.staticuser.user dr.stack + + hbase.masters.addresses.provider + org.apache.hadoop.hbase.coordination.ConfigBasedMasterAddressesProvider + Fully qualified name of class implementing master address provider. + MasterAddressProvider interface enables clients to lookup next active master available + or retrive the list of all known masters (active or inactive). + + + hbase.masters.quorum + localhost + Comma separated list of servers running HBase Masters. If cluster + runs coordinated masters (multiple active masters), then this list contains + all active masters and client could connect to any one. If cluster runs with + 1 active + multiple backup masters, this list contains list of all nodes which + are allowed to run master, but client may only successfully connect to active master + (backup masters will look dead from client's point of view). + Client will use node addresses from this list and hbase.regionserver.port to connect. + + diff --git hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java index ee1ab67..d87d3b0 100644 --- hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java +++ hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java @@ -40626,6 +40626,2015 @@ public final class MasterProtos { // @@protoc_insertion_point(class_scope:IsProcedureDoneResponse) } + public interface GetMetaLocationRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + } + /** + * Protobuf type {@code GetMetaLocationRequest} + */ + public static final class GetMetaLocationRequest extends + com.google.protobuf.GeneratedMessage + implements GetMetaLocationRequestOrBuilder { + // Use GetMetaLocationRequest.newBuilder() to construct. + private GetMetaLocationRequest(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private GetMetaLocationRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final GetMetaLocationRequest defaultInstance; + public static GetMetaLocationRequest getDefaultInstance() { + return defaultInstance; + } + + public GetMetaLocationRequest getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private GetMetaLocationRequest( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetMetaLocationRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetMetaLocationRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public GetMetaLocationRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new GetMetaLocationRequest(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private void initFields() { + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest) obj; + + boolean result = true; + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code GetMetaLocationRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetMetaLocationRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetMetaLocationRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetMetaLocationRequest_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest build() { + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest(this); + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + + // @@protoc_insertion_point(builder_scope:GetMetaLocationRequest) + } + + static { + defaultInstance = new GetMetaLocationRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:GetMetaLocationRequest) + } + + public interface GetMetaLocationResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional .ServerName meta_location = 1; + /** + * optional .ServerName meta_location = 1; + */ + boolean hasMetaLocation(); + /** + * optional .ServerName meta_location = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getMetaLocation(); + /** + * optional .ServerName meta_location = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getMetaLocationOrBuilder(); + } + /** + * Protobuf type {@code GetMetaLocationResponse} + */ + public static final class GetMetaLocationResponse extends + com.google.protobuf.GeneratedMessage + implements GetMetaLocationResponseOrBuilder { + // Use GetMetaLocationResponse.newBuilder() to construct. + private GetMetaLocationResponse(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private GetMetaLocationResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final GetMetaLocationResponse defaultInstance; + public static GetMetaLocationResponse getDefaultInstance() { + return defaultInstance; + } + + public GetMetaLocationResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private GetMetaLocationResponse( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = metaLocation_.toBuilder(); + } + metaLocation_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(metaLocation_); + metaLocation_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetMetaLocationResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetMetaLocationResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public GetMetaLocationResponse parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new GetMetaLocationResponse(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // optional .ServerName meta_location = 1; + public static final int META_LOCATION_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName metaLocation_; + /** + * optional .ServerName meta_location = 1; + */ + public boolean hasMetaLocation() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .ServerName meta_location = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getMetaLocation() { + return metaLocation_; + } + /** + * optional .ServerName meta_location = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getMetaLocationOrBuilder() { + return metaLocation_; + } + + private void initFields() { + metaLocation_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (hasMetaLocation()) { + if (!getMetaLocation().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, metaLocation_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, metaLocation_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse) obj; + + boolean result = true; + result = result && (hasMetaLocation() == other.hasMetaLocation()); + if (hasMetaLocation()) { + result = result && getMetaLocation() + .equals(other.getMetaLocation()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasMetaLocation()) { + hash = (37 * hash) + META_LOCATION_FIELD_NUMBER; + hash = (53 * hash) + getMetaLocation().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code GetMetaLocationResponse} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetMetaLocationResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetMetaLocationResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getMetaLocationFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (metaLocationBuilder_ == null) { + metaLocation_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance(); + } else { + metaLocationBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetMetaLocationResponse_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse build() { + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (metaLocationBuilder_ == null) { + result.metaLocation_ = metaLocation_; + } else { + result.metaLocation_ = metaLocationBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse.getDefaultInstance()) return this; + if (other.hasMetaLocation()) { + mergeMetaLocation(other.getMetaLocation()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (hasMetaLocation()) { + if (!getMetaLocation().isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // optional .ServerName meta_location = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName metaLocation_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder> metaLocationBuilder_; + /** + * optional .ServerName meta_location = 1; + */ + public boolean hasMetaLocation() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .ServerName meta_location = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName getMetaLocation() { + if (metaLocationBuilder_ == null) { + return metaLocation_; + } else { + return metaLocationBuilder_.getMessage(); + } + } + /** + * optional .ServerName meta_location = 1; + */ + public Builder setMetaLocation(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) { + if (metaLocationBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + metaLocation_ = value; + onChanged(); + } else { + metaLocationBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .ServerName meta_location = 1; + */ + public Builder setMetaLocation( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder builderForValue) { + if (metaLocationBuilder_ == null) { + metaLocation_ = builderForValue.build(); + onChanged(); + } else { + metaLocationBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .ServerName meta_location = 1; + */ + public Builder mergeMetaLocation(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName value) { + if (metaLocationBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + metaLocation_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance()) { + metaLocation_ = + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder(metaLocation_).mergeFrom(value).buildPartial(); + } else { + metaLocation_ = value; + } + onChanged(); + } else { + metaLocationBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .ServerName meta_location = 1; + */ + public Builder clearMetaLocation() { + if (metaLocationBuilder_ == null) { + metaLocation_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.getDefaultInstance(); + onChanged(); + } else { + metaLocationBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * optional .ServerName meta_location = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder getMetaLocationBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getMetaLocationFieldBuilder().getBuilder(); + } + /** + * optional .ServerName meta_location = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder getMetaLocationOrBuilder() { + if (metaLocationBuilder_ != null) { + return metaLocationBuilder_.getMessageOrBuilder(); + } else { + return metaLocation_; + } + } + /** + * optional .ServerName meta_location = 1; + */ + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder> + getMetaLocationFieldBuilder() { + if (metaLocationBuilder_ == null) { + metaLocationBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerNameOrBuilder>( + metaLocation_, + getParentForChildren(), + isClean()); + metaLocation_ = null; + } + return metaLocationBuilder_; + } + + // @@protoc_insertion_point(builder_scope:GetMetaLocationResponse) + } + + static { + defaultInstance = new GetMetaLocationResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:GetMetaLocationResponse) + } + + public interface GetTableStateRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional .TableName table_name = 1; + /** + * optional .TableName table_name = 1; + */ + boolean hasTableName(); + /** + * optional .TableName table_name = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName(); + /** + * optional .TableName table_name = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder(); + } + /** + * Protobuf type {@code GetTableStateRequest} + */ + public static final class GetTableStateRequest extends + com.google.protobuf.GeneratedMessage + implements GetTableStateRequestOrBuilder { + // Use GetTableStateRequest.newBuilder() to construct. + private GetTableStateRequest(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private GetTableStateRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final GetTableStateRequest defaultInstance; + public static GetTableStateRequest getDefaultInstance() { + return defaultInstance; + } + + public GetTableStateRequest getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private GetTableStateRequest( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = tableName_.toBuilder(); + } + tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(tableName_); + tableName_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetTableStateRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetTableStateRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public GetTableStateRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new GetTableStateRequest(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // optional .TableName table_name = 1; + public static final int TABLE_NAME_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_; + /** + * optional .TableName table_name = 1; + */ + public boolean hasTableName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .TableName table_name = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() { + return tableName_; + } + /** + * optional .TableName table_name = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() { + return tableName_; + } + + private void initFields() { + tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (hasTableName()) { + if (!getTableName().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, tableName_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, tableName_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest) obj; + + boolean result = true; + result = result && (hasTableName() == other.hasTableName()); + if (hasTableName()) { + result = result && getTableName() + .equals(other.getTableName()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasTableName()) { + hash = (37 * hash) + TABLE_NAME_FIELD_NUMBER; + hash = (53 * hash) + getTableName().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code GetTableStateRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetTableStateRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetTableStateRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getTableNameFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (tableNameBuilder_ == null) { + tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance(); + } else { + tableNameBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetTableStateRequest_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest build() { + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (tableNameBuilder_ == null) { + result.tableName_ = tableName_; + } else { + result.tableName_ = tableNameBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest.getDefaultInstance()) return this; + if (other.hasTableName()) { + mergeTableName(other.getTableName()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (hasTableName()) { + if (!getTableName().isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // optional .TableName table_name = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_; + /** + * optional .TableName table_name = 1; + */ + public boolean hasTableName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .TableName table_name = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() { + if (tableNameBuilder_ == null) { + return tableName_; + } else { + return tableNameBuilder_.getMessage(); + } + } + /** + * optional .TableName table_name = 1; + */ + public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) { + if (tableNameBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + tableName_ = value; + onChanged(); + } else { + tableNameBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .TableName table_name = 1; + */ + public Builder setTableName( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) { + if (tableNameBuilder_ == null) { + tableName_ = builderForValue.build(); + onChanged(); + } else { + tableNameBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .TableName table_name = 1; + */ + public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) { + if (tableNameBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + tableName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) { + tableName_ = + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial(); + } else { + tableName_ = value; + } + onChanged(); + } else { + tableNameBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .TableName table_name = 1; + */ + public Builder clearTableName() { + if (tableNameBuilder_ == null) { + tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance(); + onChanged(); + } else { + tableNameBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * optional .TableName table_name = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getTableNameFieldBuilder().getBuilder(); + } + /** + * optional .TableName table_name = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() { + if (tableNameBuilder_ != null) { + return tableNameBuilder_.getMessageOrBuilder(); + } else { + return tableName_; + } + } + /** + * optional .TableName table_name = 1; + */ + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> + getTableNameFieldBuilder() { + if (tableNameBuilder_ == null) { + tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>( + tableName_, + getParentForChildren(), + isClean()); + tableName_ = null; + } + return tableNameBuilder_; + } + + // @@protoc_insertion_point(builder_scope:GetTableStateRequest) + } + + static { + defaultInstance = new GetTableStateRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:GetTableStateRequest) + } + + public interface GetTableStateResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional .Table table_state = 1; + /** + * optional .Table table_state = 1; + */ + boolean hasTableState(); + /** + * optional .Table table_state = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table getTableState(); + /** + * optional .Table table_state = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableOrBuilder getTableStateOrBuilder(); + } + /** + * Protobuf type {@code GetTableStateResponse} + */ + public static final class GetTableStateResponse extends + com.google.protobuf.GeneratedMessage + implements GetTableStateResponseOrBuilder { + // Use GetTableStateResponse.newBuilder() to construct. + private GetTableStateResponse(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private GetTableStateResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final GetTableStateResponse defaultInstance; + public static GetTableStateResponse getDefaultInstance() { + return defaultInstance; + } + + public GetTableStateResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private GetTableStateResponse( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = tableState_.toBuilder(); + } + tableState_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(tableState_); + tableState_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetTableStateResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetTableStateResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public GetTableStateResponse parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new GetTableStateResponse(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // optional .Table table_state = 1; + public static final int TABLE_STATE_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table tableState_; + /** + * optional .Table table_state = 1; + */ + public boolean hasTableState() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .Table table_state = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table getTableState() { + return tableState_; + } + /** + * optional .Table table_state = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableOrBuilder getTableStateOrBuilder() { + return tableState_; + } + + private void initFields() { + tableState_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (hasTableState()) { + if (!getTableState().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, tableState_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, tableState_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse) obj; + + boolean result = true; + result = result && (hasTableState() == other.hasTableState()); + if (hasTableState()) { + result = result && getTableState() + .equals(other.getTableState()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasTableState()) { + hash = (37 * hash) + TABLE_STATE_FIELD_NUMBER; + hash = (53 * hash) + getTableState().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + /** + * Protobuf type {@code GetTableStateResponse} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetTableStateResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetTableStateResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getTableStateFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (tableStateBuilder_ == null) { + tableState_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDefaultInstance(); + } else { + tableStateBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_GetTableStateResponse_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse build() { + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (tableStateBuilder_ == null) { + result.tableState_ = tableState_; + } else { + result.tableState_ = tableStateBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance()) return this; + if (other.hasTableState()) { + mergeTableState(other.getTableState()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (hasTableState()) { + if (!getTableState().isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // optional .Table table_state = 1; + private org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table tableState_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.Builder, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableOrBuilder> tableStateBuilder_; + /** + * optional .Table table_state = 1; + */ + public boolean hasTableState() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .Table table_state = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table getTableState() { + if (tableStateBuilder_ == null) { + return tableState_; + } else { + return tableStateBuilder_.getMessage(); + } + } + /** + * optional .Table table_state = 1; + */ + public Builder setTableState(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table value) { + if (tableStateBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + tableState_ = value; + onChanged(); + } else { + tableStateBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .Table table_state = 1; + */ + public Builder setTableState( + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.Builder builderForValue) { + if (tableStateBuilder_ == null) { + tableState_ = builderForValue.build(); + onChanged(); + } else { + tableStateBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .Table table_state = 1; + */ + public Builder mergeTableState(org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table value) { + if (tableStateBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + tableState_ != org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDefaultInstance()) { + tableState_ = + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.newBuilder(tableState_).mergeFrom(value).buildPartial(); + } else { + tableState_ = value; + } + onChanged(); + } else { + tableStateBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + /** + * optional .Table table_state = 1; + */ + public Builder clearTableState() { + if (tableStateBuilder_ == null) { + tableState_ = org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.getDefaultInstance(); + onChanged(); + } else { + tableStateBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * optional .Table table_state = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.Builder getTableStateBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getTableStateFieldBuilder().getBuilder(); + } + /** + * optional .Table table_state = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableOrBuilder getTableStateOrBuilder() { + if (tableStateBuilder_ != null) { + return tableStateBuilder_.getMessageOrBuilder(); + } else { + return tableState_; + } + } + /** + * optional .Table table_state = 1; + */ + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.Builder, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableOrBuilder> + getTableStateFieldBuilder() { + if (tableStateBuilder_ == null) { + tableStateBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.Builder, org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.TableOrBuilder>( + tableState_, + getParentForChildren(), + isClean()); + tableState_ = null; + } + return tableStateBuilder_; + } + + // @@protoc_insertion_point(builder_scope:GetTableStateResponse) + } + + static { + defaultInstance = new GetTableStateResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:GetTableStateResponse) + } + /** * Protobuf service {@code MasterService} */ @@ -41176,6 +43185,30 @@ public final class MasterProtos { org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request, com.google.protobuf.RpcCallback done); + /** + * rpc GetMetaRegionLocation(.GetMetaLocationRequest) returns (.GetMetaLocationResponse); + * + *
+       ** Returns location of hbase:meta. Currently assumes 1 region, hence 1 region server 
+       * 
+ */ + public abstract void getMetaRegionLocation( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest request, + com.google.protobuf.RpcCallback done); + + /** + * rpc GetTableState(.GetTableStateRequest) returns (.GetTableStateResponse); + * + *
+       ** Returns state of the table requested 
+       * 
+ */ + public abstract void getTableState( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest request, + com.google.protobuf.RpcCallback done); + } public static com.google.protobuf.Service newReflectiveService( @@ -41525,6 +43558,22 @@ public final class MasterProtos { impl.listTableNamesByNamespace(controller, request, done); } + @java.lang.Override + public void getMetaRegionLocation( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest request, + com.google.protobuf.RpcCallback done) { + impl.getMetaRegionLocation(controller, request, done); + } + + @java.lang.Override + public void getTableState( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest request, + com.google.protobuf.RpcCallback done) { + impl.getTableState(controller, request, done); + } + }; } @@ -41633,6 +43682,10 @@ public final class MasterProtos { return impl.listTableDescriptorsByNamespace(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest)request); case 42: return impl.listTableNamesByNamespace(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest)request); + case 43: + return impl.getMetaRegionLocation(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest)request); + case 44: + return impl.getTableState(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest)request); default: throw new java.lang.AssertionError("Can't get here."); } @@ -41733,6 +43786,10 @@ public final class MasterProtos { return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest.getDefaultInstance(); case 42: return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest.getDefaultInstance(); + case 43: + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest.getDefaultInstance(); + case 44: + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); } @@ -41833,6 +43890,10 @@ public final class MasterProtos { return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse.getDefaultInstance(); case 42: return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance(); + case 43: + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse.getDefaultInstance(); + case 44: + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); } @@ -42383,6 +44444,30 @@ public final class MasterProtos { org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request, com.google.protobuf.RpcCallback done); + /** + * rpc GetMetaRegionLocation(.GetMetaLocationRequest) returns (.GetMetaLocationResponse); + * + *
+     ** Returns location of hbase:meta. Currently assumes 1 region, hence 1 region server 
+     * 
+ */ + public abstract void getMetaRegionLocation( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest request, + com.google.protobuf.RpcCallback done); + + /** + * rpc GetTableState(.GetTableStateRequest) returns (.GetTableStateResponse); + * + *
+     ** Returns state of the table requested 
+     * 
+ */ + public abstract void getTableState( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest request, + com.google.protobuf.RpcCallback done); + public static final com.google.protobuf.Descriptors.ServiceDescriptor getDescriptor() { @@ -42620,6 +44705,16 @@ public final class MasterProtos { com.google.protobuf.RpcUtil.specializeCallback( done)); return; + case 43: + this.getMetaRegionLocation(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 44: + this.getTableState(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; default: throw new java.lang.AssertionError("Can't get here."); } @@ -42720,6 +44815,10 @@ public final class MasterProtos { return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest.getDefaultInstance(); case 42: return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest.getDefaultInstance(); + case 43: + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest.getDefaultInstance(); + case 44: + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); } @@ -42820,6 +44919,10 @@ public final class MasterProtos { return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse.getDefaultInstance(); case 42: return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance(); + case 43: + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse.getDefaultInstance(); + case 44: + return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); } @@ -43485,6 +45588,36 @@ public final class MasterProtos { org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance())); } + + public void getMetaRegionLocation( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(43), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse.class, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse.getDefaultInstance())); + } + + public void getTableState( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(44), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.class, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance())); + } } public static BlockingInterface newBlockingStub( @@ -43707,6 +45840,16 @@ public final class MasterProtos { com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request) throws com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse getMetaRegionLocation( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest request) + throws com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse getTableState( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest request) + throws com.google.protobuf.ServiceException; } private static final class BlockingStub implements BlockingInterface { @@ -44231,6 +46374,30 @@ public final class MasterProtos { org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance()); } + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse getMetaRegionLocation( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(43), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetMetaLocationResponse.getDefaultInstance()); + } + + + public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse getTableState( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(44), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableStateResponse.getDefaultInstance()); + } + } // @@protoc_insertion_point(class_scope:MasterService) @@ -44646,6 +46813,26 @@ public final class MasterProtos { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_IsProcedureDoneResponse_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_GetMetaLocationRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_GetMetaLocationRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_GetMetaLocationResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_GetMetaLocationResponse_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_GetTableStateRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_GetTableStateRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_GetTableStateResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_GetTableStateResponse_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -44656,193 +46843,202 @@ public final class MasterProtos { static { java.lang.String[] descriptorData = { "\n\014Master.proto\032\013HBase.proto\032\014Client.prot" + - "o\032\023ClusterStatus.proto\"`\n\020AddColumnReque" + - "st\022\036\n\ntable_name\030\001 \002(\0132\n.TableName\022,\n\017co" + - "lumn_families\030\002 \002(\0132\023.ColumnFamilySchema" + - "\"\023\n\021AddColumnResponse\"J\n\023DeleteColumnReq" + - "uest\022\036\n\ntable_name\030\001 \002(\0132\n.TableName\022\023\n\013" + - "column_name\030\002 \002(\014\"\026\n\024DeleteColumnRespons" + - "e\"c\n\023ModifyColumnRequest\022\036\n\ntable_name\030\001" + - " \002(\0132\n.TableName\022,\n\017column_families\030\002 \002(" + - "\0132\023.ColumnFamilySchema\"\026\n\024ModifyColumnRe", - "sponse\"\\\n\021MoveRegionRequest\022 \n\006region\030\001 " + - "\002(\0132\020.RegionSpecifier\022%\n\020dest_server_nam" + - "e\030\002 \001(\0132\013.ServerName\"\024\n\022MoveRegionRespon" + - "se\"\200\001\n\035DispatchMergingRegionsRequest\022\"\n\010" + - "region_a\030\001 \002(\0132\020.RegionSpecifier\022\"\n\010regi" + - "on_b\030\002 \002(\0132\020.RegionSpecifier\022\027\n\010forcible" + - "\030\003 \001(\010:\005false\" \n\036DispatchMergingRegionsR" + - "esponse\"7\n\023AssignRegionRequest\022 \n\006region" + - "\030\001 \002(\0132\020.RegionSpecifier\"\026\n\024AssignRegion" + - "Response\"O\n\025UnassignRegionRequest\022 \n\006reg", - "ion\030\001 \002(\0132\020.RegionSpecifier\022\024\n\005force\030\002 \001" + - "(\010:\005false\"\030\n\026UnassignRegionResponse\"8\n\024O" + - "fflineRegionRequest\022 \n\006region\030\001 \002(\0132\020.Re" + - "gionSpecifier\"\027\n\025OfflineRegionResponse\"L" + - "\n\022CreateTableRequest\022\"\n\014table_schema\030\001 \002" + - "(\0132\014.TableSchema\022\022\n\nsplit_keys\030\002 \003(\014\"\025\n\023" + - "CreateTableResponse\"4\n\022DeleteTableReques" + - "t\022\036\n\ntable_name\030\001 \002(\0132\n.TableName\"\025\n\023Del" + - "eteTableResponse\"T\n\024TruncateTableRequest" + - "\022\035\n\ttableName\030\001 \002(\0132\n.TableName\022\035\n\016prese", - "rveSplits\030\002 \001(\010:\005false\"\027\n\025TruncateTableR" + - "esponse\"4\n\022EnableTableRequest\022\036\n\ntable_n" + - "ame\030\001 \002(\0132\n.TableName\"\025\n\023EnableTableResp" + - "onse\"5\n\023DisableTableRequest\022\036\n\ntable_nam" + - "e\030\001 \002(\0132\n.TableName\"\026\n\024DisableTableRespo" + - "nse\"X\n\022ModifyTableRequest\022\036\n\ntable_name\030" + - "\001 \002(\0132\n.TableName\022\"\n\014table_schema\030\002 \002(\0132" + - "\014.TableSchema\"\025\n\023ModifyTableResponse\"K\n\026" + - "CreateNamespaceRequest\0221\n\023namespaceDescr" + - "iptor\030\001 \002(\0132\024.NamespaceDescriptor\"\031\n\027Cre", - "ateNamespaceResponse\"/\n\026DeleteNamespaceR" + - "equest\022\025\n\rnamespaceName\030\001 \002(\t\"\031\n\027DeleteN" + - "amespaceResponse\"K\n\026ModifyNamespaceReque" + - "st\0221\n\023namespaceDescriptor\030\001 \002(\0132\024.Namesp" + - "aceDescriptor\"\031\n\027ModifyNamespaceResponse" + - "\"6\n\035GetNamespaceDescriptorRequest\022\025\n\rnam" + - "espaceName\030\001 \002(\t\"S\n\036GetNamespaceDescript" + - "orResponse\0221\n\023namespaceDescriptor\030\001 \002(\0132" + - "\024.NamespaceDescriptor\"!\n\037ListNamespaceDe" + - "scriptorsRequest\"U\n ListNamespaceDescrip", - "torsResponse\0221\n\023namespaceDescriptor\030\001 \003(" + - "\0132\024.NamespaceDescriptor\"?\n&ListTableDesc" + - "riptorsByNamespaceRequest\022\025\n\rnamespaceNa" + - "me\030\001 \002(\t\"L\n\'ListTableDescriptorsByNamesp" + - "aceResponse\022!\n\013tableSchema\030\001 \003(\0132\014.Table" + - "Schema\"9\n ListTableNamesByNamespaceReque" + - "st\022\025\n\rnamespaceName\030\001 \002(\t\"B\n!ListTableNa" + - "mesByNamespaceResponse\022\035\n\ttableName\030\001 \003(" + - "\0132\n.TableName\"\021\n\017ShutdownRequest\"\022\n\020Shut" + - "downResponse\"\023\n\021StopMasterRequest\"\024\n\022Sto", - "pMasterResponse\"\020\n\016BalanceRequest\"\'\n\017Bal" + - "anceResponse\022\024\n\014balancer_ran\030\001 \002(\010\"<\n\031Se" + - "tBalancerRunningRequest\022\n\n\002on\030\001 \002(\010\022\023\n\013s" + - "ynchronous\030\002 \001(\010\"8\n\032SetBalancerRunningRe" + - "sponse\022\032\n\022prev_balance_value\030\001 \001(\010\"\027\n\025Ru" + - "nCatalogScanRequest\"-\n\026RunCatalogScanRes" + - "ponse\022\023\n\013scan_result\030\001 \001(\005\"-\n\033EnableCata" + - "logJanitorRequest\022\016\n\006enable\030\001 \002(\010\"2\n\034Ena" + - "bleCatalogJanitorResponse\022\022\n\nprev_value\030" + - "\001 \001(\010\" \n\036IsCatalogJanitorEnabledRequest\"", - "0\n\037IsCatalogJanitorEnabledResponse\022\r\n\005va" + - "lue\030\001 \002(\010\"9\n\017SnapshotRequest\022&\n\010snapshot" + - "\030\001 \002(\0132\024.SnapshotDescription\",\n\020Snapshot" + - "Response\022\030\n\020expected_timeout\030\001 \002(\003\"\036\n\034Ge" + - "tCompletedSnapshotsRequest\"H\n\035GetComplet" + - "edSnapshotsResponse\022\'\n\tsnapshots\030\001 \003(\0132\024" + - ".SnapshotDescription\"?\n\025DeleteSnapshotRe" + + "o\032\023ClusterStatus.proto\032\017ZooKeeper.proto\"" + + "`\n\020AddColumnRequest\022\036\n\ntable_name\030\001 \002(\0132" + + "\n.TableName\022,\n\017column_families\030\002 \002(\0132\023.C" + + "olumnFamilySchema\"\023\n\021AddColumnResponse\"J" + + "\n\023DeleteColumnRequest\022\036\n\ntable_name\030\001 \002(" + + "\0132\n.TableName\022\023\n\013column_name\030\002 \002(\014\"\026\n\024De" + + "leteColumnResponse\"c\n\023ModifyColumnReques" + + "t\022\036\n\ntable_name\030\001 \002(\0132\n.TableName\022,\n\017col" + + "umn_families\030\002 \002(\0132\023.ColumnFamilySchema\"", + "\026\n\024ModifyColumnResponse\"\\\n\021MoveRegionReq" + + "uest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\022%" + + "\n\020dest_server_name\030\002 \001(\0132\013.ServerName\"\024\n" + + "\022MoveRegionResponse\"\200\001\n\035DispatchMergingR" + + "egionsRequest\022\"\n\010region_a\030\001 \002(\0132\020.Region" + + "Specifier\022\"\n\010region_b\030\002 \002(\0132\020.RegionSpec" + + "ifier\022\027\n\010forcible\030\003 \001(\010:\005false\" \n\036Dispat" + + "chMergingRegionsResponse\"7\n\023AssignRegion" + + "Request\022 \n\006region\030\001 \002(\0132\020.RegionSpecifie" + + "r\"\026\n\024AssignRegionResponse\"O\n\025UnassignReg", + "ionRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpeci" + + "fier\022\024\n\005force\030\002 \001(\010:\005false\"\030\n\026UnassignRe" + + "gionResponse\"8\n\024OfflineRegionRequest\022 \n\006" + + "region\030\001 \002(\0132\020.RegionSpecifier\"\027\n\025Offlin" + + "eRegionResponse\"L\n\022CreateTableRequest\022\"\n" + + "\014table_schema\030\001 \002(\0132\014.TableSchema\022\022\n\nspl" + + "it_keys\030\002 \003(\014\"\025\n\023CreateTableResponse\"4\n\022" + + "DeleteTableRequest\022\036\n\ntable_name\030\001 \002(\0132\n" + + ".TableName\"\025\n\023DeleteTableResponse\"T\n\024Tru" + + "ncateTableRequest\022\035\n\ttableName\030\001 \002(\0132\n.T", + "ableName\022\035\n\016preserveSplits\030\002 \001(\010:\005false\"" + + "\027\n\025TruncateTableResponse\"4\n\022EnableTableR" + + "equest\022\036\n\ntable_name\030\001 \002(\0132\n.TableName\"\025" + + "\n\023EnableTableResponse\"5\n\023DisableTableReq" + + "uest\022\036\n\ntable_name\030\001 \002(\0132\n.TableName\"\026\n\024" + + "DisableTableResponse\"X\n\022ModifyTableReque" + + "st\022\036\n\ntable_name\030\001 \002(\0132\n.TableName\022\"\n\014ta" + + "ble_schema\030\002 \002(\0132\014.TableSchema\"\025\n\023Modify" + + "TableResponse\"K\n\026CreateNamespaceRequest\022" + + "1\n\023namespaceDescriptor\030\001 \002(\0132\024.Namespace", + "Descriptor\"\031\n\027CreateNamespaceResponse\"/\n" + + "\026DeleteNamespaceRequest\022\025\n\rnamespaceName" + + "\030\001 \002(\t\"\031\n\027DeleteNamespaceResponse\"K\n\026Mod" + + "ifyNamespaceRequest\0221\n\023namespaceDescript" + + "or\030\001 \002(\0132\024.NamespaceDescriptor\"\031\n\027Modify" + + "NamespaceResponse\"6\n\035GetNamespaceDescrip" + + "torRequest\022\025\n\rnamespaceName\030\001 \002(\t\"S\n\036Get" + + "NamespaceDescriptorResponse\0221\n\023namespace" + + "Descriptor\030\001 \002(\0132\024.NamespaceDescriptor\"!" + + "\n\037ListNamespaceDescriptorsRequest\"U\n Lis", + "tNamespaceDescriptorsResponse\0221\n\023namespa" + + "ceDescriptor\030\001 \003(\0132\024.NamespaceDescriptor" + + "\"?\n&ListTableDescriptorsByNamespaceReque" + + "st\022\025\n\rnamespaceName\030\001 \002(\t\"L\n\'ListTableDe" + + "scriptorsByNamespaceResponse\022!\n\013tableSch" + + "ema\030\001 \003(\0132\014.TableSchema\"9\n ListTableName" + + "sByNamespaceRequest\022\025\n\rnamespaceName\030\001 \002" + + "(\t\"B\n!ListTableNamesByNamespaceResponse\022" + + "\035\n\ttableName\030\001 \003(\0132\n.TableName\"\021\n\017Shutdo" + + "wnRequest\"\022\n\020ShutdownResponse\"\023\n\021StopMas", + "terRequest\"\024\n\022StopMasterResponse\"\020\n\016Bala" + + "nceRequest\"\'\n\017BalanceResponse\022\024\n\014balance" + + "r_ran\030\001 \002(\010\"<\n\031SetBalancerRunningRequest" + + "\022\n\n\002on\030\001 \002(\010\022\023\n\013synchronous\030\002 \001(\010\"8\n\032Set" + + "BalancerRunningResponse\022\032\n\022prev_balance_" + + "value\030\001 \001(\010\"\027\n\025RunCatalogScanRequest\"-\n\026" + + "RunCatalogScanResponse\022\023\n\013scan_result\030\001 " + + "\001(\005\"-\n\033EnableCatalogJanitorRequest\022\016\n\006en" + + "able\030\001 \002(\010\"2\n\034EnableCatalogJanitorRespon" + + "se\022\022\n\nprev_value\030\001 \001(\010\" \n\036IsCatalogJanit", + "orEnabledRequest\"0\n\037IsCatalogJanitorEnab" + + "ledResponse\022\r\n\005value\030\001 \002(\010\"9\n\017SnapshotRe" + "quest\022&\n\010snapshot\030\001 \002(\0132\024.SnapshotDescri" + - "ption\"\030\n\026DeleteSnapshotResponse\"@\n\026Resto" + - "reSnapshotRequest\022&\n\010snapshot\030\001 \002(\0132\024.Sn", - "apshotDescription\"\031\n\027RestoreSnapshotResp" + - "onse\"?\n\025IsSnapshotDoneRequest\022&\n\010snapsho" + - "t\030\001 \001(\0132\024.SnapshotDescription\"U\n\026IsSnaps" + - "hotDoneResponse\022\023\n\004done\030\001 \001(\010:\005false\022&\n\010" + - "snapshot\030\002 \001(\0132\024.SnapshotDescription\"F\n\034" + - "IsRestoreSnapshotDoneRequest\022&\n\010snapshot" + - "\030\001 \001(\0132\024.SnapshotDescription\"4\n\035IsRestor" + - "eSnapshotDoneResponse\022\023\n\004done\030\001 \001(\010:\005fal" + - "se\"=\n\033GetSchemaAlterStatusRequest\022\036\n\ntab" + - "le_name\030\001 \002(\0132\n.TableName\"T\n\034GetSchemaAl", - "terStatusResponse\022\035\n\025yet_to_update_regio" + - "ns\030\001 \001(\r\022\025\n\rtotal_regions\030\002 \001(\r\"=\n\032GetTa" + - "bleDescriptorsRequest\022\037\n\013table_names\030\001 \003" + - "(\0132\n.TableName\"A\n\033GetTableDescriptorsRes" + - "ponse\022\"\n\014table_schema\030\001 \003(\0132\014.TableSchem" + - "a\"\026\n\024GetTableNamesRequest\"8\n\025GetTableNam" + - "esResponse\022\037\n\013table_names\030\001 \003(\0132\n.TableN" + - "ame\"\031\n\027GetClusterStatusRequest\"B\n\030GetClu" + - "sterStatusResponse\022&\n\016cluster_status\030\001 \002" + - "(\0132\016.ClusterStatus\"\030\n\026IsMasterRunningReq", - "uest\"4\n\027IsMasterRunningResponse\022\031\n\021is_ma" + - "ster_running\030\001 \002(\010\"@\n\024ExecProcedureReque" + - "st\022(\n\tprocedure\030\001 \002(\0132\025.ProcedureDescrip" + - "tion\"F\n\025ExecProcedureResponse\022\030\n\020expecte" + - "d_timeout\030\001 \001(\003\022\023\n\013return_data\030\002 \001(\014\"B\n\026" + - "IsProcedureDoneRequest\022(\n\tprocedure\030\001 \001(" + - "\0132\025.ProcedureDescription\"W\n\027IsProcedureD" + - "oneResponse\022\023\n\004done\030\001 \001(\010:\005false\022\'\n\010snap" + - "shot\030\002 \001(\0132\025.ProcedureDescription2\365\027\n\rMa" + - "sterService\022S\n\024GetSchemaAlterStatus\022\034.Ge", - "tSchemaAlterStatusRequest\032\035.GetSchemaAlt" + - "erStatusResponse\022P\n\023GetTableDescriptors\022" + - "\033.GetTableDescriptorsRequest\032\034.GetTableD" + - "escriptorsResponse\022>\n\rGetTableNames\022\025.Ge" + - "tTableNamesRequest\032\026.GetTableNamesRespon" + - "se\022G\n\020GetClusterStatus\022\030.GetClusterStatu" + - "sRequest\032\031.GetClusterStatusResponse\022D\n\017I" + - "sMasterRunning\022\027.IsMasterRunningRequest\032" + - "\030.IsMasterRunningResponse\0222\n\tAddColumn\022\021" + - ".AddColumnRequest\032\022.AddColumnResponse\022;\n", - "\014DeleteColumn\022\024.DeleteColumnRequest\032\025.De" + - "leteColumnResponse\022;\n\014ModifyColumn\022\024.Mod" + - "ifyColumnRequest\032\025.ModifyColumnResponse\022" + - "5\n\nMoveRegion\022\022.MoveRegionRequest\032\023.Move" + - "RegionResponse\022Y\n\026DispatchMergingRegions" + - "\022\036.DispatchMergingRegionsRequest\032\037.Dispa" + - "tchMergingRegionsResponse\022;\n\014AssignRegio" + - "n\022\024.AssignRegionRequest\032\025.AssignRegionRe" + - "sponse\022A\n\016UnassignRegion\022\026.UnassignRegio" + - "nRequest\032\027.UnassignRegionResponse\022>\n\rOff", - "lineRegion\022\025.OfflineRegionRequest\032\026.Offl" + - "ineRegionResponse\0228\n\013DeleteTable\022\023.Delet" + - "eTableRequest\032\024.DeleteTableResponse\022>\n\rt" + - "runcateTable\022\025.TruncateTableRequest\032\026.Tr" + - "uncateTableResponse\0228\n\013EnableTable\022\023.Ena" + - "bleTableRequest\032\024.EnableTableResponse\022;\n" + - "\014DisableTable\022\024.DisableTableRequest\032\025.Di" + - "sableTableResponse\0228\n\013ModifyTable\022\023.Modi" + - "fyTableRequest\032\024.ModifyTableResponse\0228\n\013" + - "CreateTable\022\023.CreateTableRequest\032\024.Creat", - "eTableResponse\022/\n\010Shutdown\022\020.ShutdownReq" + - "uest\032\021.ShutdownResponse\0225\n\nStopMaster\022\022." + - "StopMasterRequest\032\023.StopMasterResponse\022," + - "\n\007Balance\022\017.BalanceRequest\032\020.BalanceResp" + - "onse\022M\n\022SetBalancerRunning\022\032.SetBalancer" + - "RunningRequest\032\033.SetBalancerRunningRespo" + - "nse\022A\n\016RunCatalogScan\022\026.RunCatalogScanRe" + - "quest\032\027.RunCatalogScanResponse\022S\n\024Enable" + - "CatalogJanitor\022\034.EnableCatalogJanitorReq" + - "uest\032\035.EnableCatalogJanitorResponse\022\\\n\027I", - "sCatalogJanitorEnabled\022\037.IsCatalogJanito" + - "rEnabledRequest\032 .IsCatalogJanitorEnable" + - "dResponse\022L\n\021ExecMasterService\022\032.Coproce" + - "ssorServiceRequest\032\033.CoprocessorServiceR" + - "esponse\022/\n\010Snapshot\022\020.SnapshotRequest\032\021." + - "SnapshotResponse\022V\n\025GetCompletedSnapshot" + - "s\022\035.GetCompletedSnapshotsRequest\032\036.GetCo" + - "mpletedSnapshotsResponse\022A\n\016DeleteSnapsh" + - "ot\022\026.DeleteSnapshotRequest\032\027.DeleteSnaps" + - "hotResponse\022A\n\016IsSnapshotDone\022\026.IsSnapsh", - "otDoneRequest\032\027.IsSnapshotDoneResponse\022D" + - "\n\017RestoreSnapshot\022\027.RestoreSnapshotReque" + - "st\032\030.RestoreSnapshotResponse\022V\n\025IsRestor" + - "eSnapshotDone\022\035.IsRestoreSnapshotDoneReq" + - "uest\032\036.IsRestoreSnapshotDoneResponse\022>\n\r" + - "ExecProcedure\022\025.ExecProcedureRequest\032\026.E" + - "xecProcedureResponse\022E\n\024ExecProcedureWit" + - "hRet\022\025.ExecProcedureRequest\032\026.ExecProced" + - "ureResponse\022D\n\017IsProcedureDone\022\027.IsProce" + - "dureDoneRequest\032\030.IsProcedureDoneRespons", - "e\022D\n\017ModifyNamespace\022\027.ModifyNamespaceRe" + - "quest\032\030.ModifyNamespaceResponse\022D\n\017Creat" + - "eNamespace\022\027.CreateNamespaceRequest\032\030.Cr" + - "eateNamespaceResponse\022D\n\017DeleteNamespace" + - "\022\027.DeleteNamespaceRequest\032\030.DeleteNamesp" + - "aceResponse\022Y\n\026GetNamespaceDescriptor\022\036." + - "GetNamespaceDescriptorRequest\032\037.GetNames" + - "paceDescriptorResponse\022_\n\030ListNamespaceD" + - "escriptors\022 .ListNamespaceDescriptorsReq" + - "uest\032!.ListNamespaceDescriptorsResponse\022", - "t\n\037ListTableDescriptorsByNamespace\022\'.Lis" + - "tTableDescriptorsByNamespaceRequest\032(.Li" + - "stTableDescriptorsByNamespaceResponse\022b\n" + - "\031ListTableNamesByNamespace\022!.ListTableNa" + - "mesByNamespaceRequest\032\".ListTableNamesBy" + - "NamespaceResponseBB\n*org.apache.hadoop.h" + - "base.protobuf.generatedB\014MasterProtosH\001\210" + - "\001\001\240\001\001" + "ption\",\n\020SnapshotResponse\022\030\n\020expected_ti" + + "meout\030\001 \002(\003\"\036\n\034GetCompletedSnapshotsRequ" + + "est\"H\n\035GetCompletedSnapshotsResponse\022\'\n\t" + + "snapshots\030\001 \003(\0132\024.SnapshotDescription\"?\n" + + "\025DeleteSnapshotRequest\022&\n\010snapshot\030\001 \002(\013" + + "2\024.SnapshotDescription\"\030\n\026DeleteSnapshot" + + "Response\"@\n\026RestoreSnapshotRequest\022&\n\010sn", + "apshot\030\001 \002(\0132\024.SnapshotDescription\"\031\n\027Re" + + "storeSnapshotResponse\"?\n\025IsSnapshotDoneR" + + "equest\022&\n\010snapshot\030\001 \001(\0132\024.SnapshotDescr" + + "iption\"U\n\026IsSnapshotDoneResponse\022\023\n\004done" + + "\030\001 \001(\010:\005false\022&\n\010snapshot\030\002 \001(\0132\024.Snapsh" + + "otDescription\"F\n\034IsRestoreSnapshotDoneRe" + + "quest\022&\n\010snapshot\030\001 \001(\0132\024.SnapshotDescri" + + "ption\"4\n\035IsRestoreSnapshotDoneResponse\022\023" + + "\n\004done\030\001 \001(\010:\005false\"=\n\033GetSchemaAlterSta" + + "tusRequest\022\036\n\ntable_name\030\001 \002(\0132\n.TableNa", + "me\"T\n\034GetSchemaAlterStatusResponse\022\035\n\025ye" + + "t_to_update_regions\030\001 \001(\r\022\025\n\rtotal_regio" + + "ns\030\002 \001(\r\"=\n\032GetTableDescriptorsRequest\022\037" + + "\n\013table_names\030\001 \003(\0132\n.TableName\"A\n\033GetTa" + + "bleDescriptorsResponse\022\"\n\014table_schema\030\001" + + " \003(\0132\014.TableSchema\"\026\n\024GetTableNamesReque" + + "st\"8\n\025GetTableNamesResponse\022\037\n\013table_nam" + + "es\030\001 \003(\0132\n.TableName\"\031\n\027GetClusterStatus" + + "Request\"B\n\030GetClusterStatusResponse\022&\n\016c" + + "luster_status\030\001 \002(\0132\016.ClusterStatus\"\030\n\026I", + "sMasterRunningRequest\"4\n\027IsMasterRunning" + + "Response\022\031\n\021is_master_running\030\001 \002(\010\"@\n\024E" + + "xecProcedureRequest\022(\n\tprocedure\030\001 \002(\0132\025" + + ".ProcedureDescription\"F\n\025ExecProcedureRe" + + "sponse\022\030\n\020expected_timeout\030\001 \001(\003\022\023\n\013retu" + + "rn_data\030\002 \001(\014\"B\n\026IsProcedureDoneRequest\022" + + "(\n\tprocedure\030\001 \001(\0132\025.ProcedureDescriptio" + + "n\"W\n\027IsProcedureDoneResponse\022\023\n\004done\030\001 \001" + + "(\010:\005false\022\'\n\010snapshot\030\002 \001(\0132\025.ProcedureD" + + "escription\"\030\n\026GetMetaLocationRequest\"=\n\027", + "GetMetaLocationResponse\022\"\n\rmeta_location" + + "\030\001 \001(\0132\013.ServerName\"6\n\024GetTableStateRequ" + + "est\022\036\n\ntable_name\030\001 \001(\0132\n.TableName\"4\n\025G" + + "etTableStateResponse\022\033\n\013table_state\030\001 \001(" + + "\0132\006.Table2\201\031\n\rMasterService\022S\n\024GetSchema" + + "AlterStatus\022\034.GetSchemaAlterStatusReques" + + "t\032\035.GetSchemaAlterStatusResponse\022P\n\023GetT" + + "ableDescriptors\022\033.GetTableDescriptorsReq" + + "uest\032\034.GetTableDescriptorsResponse\022>\n\rGe" + + "tTableNames\022\025.GetTableNamesRequest\032\026.Get", + "TableNamesResponse\022G\n\020GetClusterStatus\022\030" + + ".GetClusterStatusRequest\032\031.GetClusterSta" + + "tusResponse\022D\n\017IsMasterRunning\022\027.IsMaste" + + "rRunningRequest\032\030.IsMasterRunningRespons" + + "e\0222\n\tAddColumn\022\021.AddColumnRequest\032\022.AddC" + + "olumnResponse\022;\n\014DeleteColumn\022\024.DeleteCo" + + "lumnRequest\032\025.DeleteColumnResponse\022;\n\014Mo" + + "difyColumn\022\024.ModifyColumnRequest\032\025.Modif" + + "yColumnResponse\0225\n\nMoveRegion\022\022.MoveRegi" + + "onRequest\032\023.MoveRegionResponse\022Y\n\026Dispat", + "chMergingRegions\022\036.DispatchMergingRegion" + + "sRequest\032\037.DispatchMergingRegionsRespons" + + "e\022;\n\014AssignRegion\022\024.AssignRegionRequest\032" + + "\025.AssignRegionResponse\022A\n\016UnassignRegion" + + "\022\026.UnassignRegionRequest\032\027.UnassignRegio" + + "nResponse\022>\n\rOfflineRegion\022\025.OfflineRegi" + + "onRequest\032\026.OfflineRegionResponse\0228\n\013Del" + + "eteTable\022\023.DeleteTableRequest\032\024.DeleteTa" + + "bleResponse\022>\n\rtruncateTable\022\025.TruncateT" + + "ableRequest\032\026.TruncateTableResponse\0228\n\013E", + "nableTable\022\023.EnableTableRequest\032\024.Enable" + + "TableResponse\022;\n\014DisableTable\022\024.DisableT" + + "ableRequest\032\025.DisableTableResponse\0228\n\013Mo" + + "difyTable\022\023.ModifyTableRequest\032\024.ModifyT" + + "ableResponse\0228\n\013CreateTable\022\023.CreateTabl" + + "eRequest\032\024.CreateTableResponse\022/\n\010Shutdo" + + "wn\022\020.ShutdownRequest\032\021.ShutdownResponse\022" + + "5\n\nStopMaster\022\022.StopMasterRequest\032\023.Stop" + + "MasterResponse\022,\n\007Balance\022\017.BalanceReque" + + "st\032\020.BalanceResponse\022M\n\022SetBalancerRunni", + "ng\022\032.SetBalancerRunningRequest\032\033.SetBala" + + "ncerRunningResponse\022A\n\016RunCatalogScan\022\026." + + "RunCatalogScanRequest\032\027.RunCatalogScanRe" + + "sponse\022S\n\024EnableCatalogJanitor\022\034.EnableC" + + "atalogJanitorRequest\032\035.EnableCatalogJani" + + "torResponse\022\\\n\027IsCatalogJanitorEnabled\022\037" + + ".IsCatalogJanitorEnabledRequest\032 .IsCata" + + "logJanitorEnabledResponse\022L\n\021ExecMasterS" + + "ervice\022\032.CoprocessorServiceRequest\032\033.Cop" + + "rocessorServiceResponse\022/\n\010Snapshot\022\020.Sn", + "apshotRequest\032\021.SnapshotResponse\022V\n\025GetC" + + "ompletedSnapshots\022\035.GetCompletedSnapshot" + + "sRequest\032\036.GetCompletedSnapshotsResponse" + + "\022A\n\016DeleteSnapshot\022\026.DeleteSnapshotReque" + + "st\032\027.DeleteSnapshotResponse\022A\n\016IsSnapsho" + + "tDone\022\026.IsSnapshotDoneRequest\032\027.IsSnapsh" + + "otDoneResponse\022D\n\017RestoreSnapshot\022\027.Rest" + + "oreSnapshotRequest\032\030.RestoreSnapshotResp" + + "onse\022V\n\025IsRestoreSnapshotDone\022\035.IsRestor" + + "eSnapshotDoneRequest\032\036.IsRestoreSnapshot", + "DoneResponse\022>\n\rExecProcedure\022\025.ExecProc" + + "edureRequest\032\026.ExecProcedureResponse\022E\n\024" + + "ExecProcedureWithRet\022\025.ExecProcedureRequ" + + "est\032\026.ExecProcedureResponse\022D\n\017IsProcedu" + + "reDone\022\027.IsProcedureDoneRequest\032\030.IsProc" + + "edureDoneResponse\022D\n\017ModifyNamespace\022\027.M" + + "odifyNamespaceRequest\032\030.ModifyNamespaceR" + + "esponse\022D\n\017CreateNamespace\022\027.CreateNames" + + "paceRequest\032\030.CreateNamespaceResponse\022D\n" + + "\017DeleteNamespace\022\027.DeleteNamespaceReques", + "t\032\030.DeleteNamespaceResponse\022Y\n\026GetNamesp" + + "aceDescriptor\022\036.GetNamespaceDescriptorRe" + + "quest\032\037.GetNamespaceDescriptorResponse\022_" + + "\n\030ListNamespaceDescriptors\022 .ListNamespa" + + "ceDescriptorsRequest\032!.ListNamespaceDesc" + + "riptorsResponse\022t\n\037ListTableDescriptorsB" + + "yNamespace\022\'.ListTableDescriptorsByNames" + + "paceRequest\032(.ListTableDescriptorsByName" + + "spaceResponse\022b\n\031ListTableNamesByNamespa" + + "ce\022!.ListTableNamesByNamespaceRequest\032\".", + "ListTableNamesByNamespaceResponse\022J\n\025Get" + + "MetaRegionLocation\022\027.GetMetaLocationRequ" + + "est\032\030.GetMetaLocationResponse\022>\n\rGetTabl" + + "eState\022\025.GetTableStateRequest\032\026.GetTable" + + "StateResponseBB\n*org.apache.hadoop.hbase" + + ".protobuf.generatedB\014MasterProtosH\001\210\001\001\240\001" + + "\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -45341,6 +47537,30 @@ public final class MasterProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_IsProcedureDoneResponse_descriptor, new java.lang.String[] { "Done", "Snapshot", }); + internal_static_GetMetaLocationRequest_descriptor = + getDescriptor().getMessageTypes().get(82); + internal_static_GetMetaLocationRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_GetMetaLocationRequest_descriptor, + new java.lang.String[] { }); + internal_static_GetMetaLocationResponse_descriptor = + getDescriptor().getMessageTypes().get(83); + internal_static_GetMetaLocationResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_GetMetaLocationResponse_descriptor, + new java.lang.String[] { "MetaLocation", }); + internal_static_GetTableStateRequest_descriptor = + getDescriptor().getMessageTypes().get(84); + internal_static_GetTableStateRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_GetTableStateRequest_descriptor, + new java.lang.String[] { "TableName", }); + internal_static_GetTableStateResponse_descriptor = + getDescriptor().getMessageTypes().get(85); + internal_static_GetTableStateResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_GetTableStateResponse_descriptor, + new java.lang.String[] { "TableState", }); return null; } }; @@ -45350,6 +47570,7 @@ public final class MasterProtos { org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor(), org.apache.hadoop.hbase.protobuf.generated.ClientProtos.getDescriptor(), org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.getDescriptor(), + org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.getDescriptor(), }, assigner); } diff --git hbase-protocol/src/main/protobuf/Master.proto hbase-protocol/src/main/protobuf/Master.proto index 94ea860..6c773af 100644 --- hbase-protocol/src/main/protobuf/Master.proto +++ hbase-protocol/src/main/protobuf/Master.proto @@ -28,6 +28,7 @@ option optimize_for = SPEED; import "HBase.proto"; import "Client.proto"; import "ClusterStatus.proto"; +import "ZooKeeper.proto"; /* Column-level protobufs */ @@ -358,6 +359,21 @@ message IsProcedureDoneResponse { optional ProcedureDescription snapshot = 2; } +message GetMetaLocationRequest { +} + +message GetMetaLocationResponse { + optional ServerName meta_location = 1; +} + +message GetTableStateRequest { + optional TableName table_name = 1; +} + +message GetTableStateResponse { + optional Table table_state = 1; +} + service MasterService { /** Used by the client to get the number of regions that have received the updated schema */ rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest) @@ -565,4 +581,12 @@ service MasterService { /** returns a list of tables for a given namespace*/ rpc ListTableNamesByNamespace(ListTableNamesByNamespaceRequest) returns(ListTableNamesByNamespaceResponse); + + /** Returns location of hbase:meta. Currently assumes 1 region, hence 1 region server */ + rpc GetMetaRegionLocation(GetMetaLocationRequest) + returns(GetMetaLocationResponse); + + /** Returns state of the table requested */ + rpc GetTableState(GetTableStateRequest) + returns(GetTableStateResponse); } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/TableStateManager.java hbase-server/src/main/java/org/apache/hadoop/hbase/TableStateManager.java index 56cd4ae..a628ce7 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/TableStateManager.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/TableStateManager.java @@ -112,4 +112,10 @@ public interface TableStateManager { void checkAndRemoveTableState(TableName tableName, ZooKeeperProtos.Table.State states, boolean deletePermanentState) throws CoordinatedStateException; + + /** + * Gets table state. + * @param tableName table we're checking + */ + public ZooKeeperProtos.Table.State getTableState(final TableName tableName); } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java index a512f83..51ffbb0 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java @@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -144,7 +143,7 @@ class CoprocessorHConnection implements ClusterConnection { public Admin getAdmin() throws IOException { return delegate.getAdmin(); } @Override - public boolean isMasterRunning() throws MasterNotRunningException, ZooKeeperConnectionException { + public boolean isMasterRunning() throws MasterNotRunningException { return delegate.isMasterRunning(); } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index f8e99f1..f8d7fdf 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -467,6 +467,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server { isActiveMaster = true; + // If the port was dynamically assigned from ephemeral range (mini cluster), + // we want to update configuration + this.conf.setInt(HConstants.MASTER_PORT, serverName.getPort()); + /* * We are active master now... go initialize components we need to run. * Note, there may be dross in zk from previous runs; it'll get addressed diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index d4f3d6d..1255109 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -1269,4 +1269,27 @@ public class MasterRpcServices extends RSRpcServices throw new ServiceException(ioe); } } + + @Override + public MasterProtos.GetMetaLocationResponse getMetaRegionLocation(RpcController controller, + MasterProtos.GetMetaLocationRequest request) throws ServiceException { + ServerName metaLocation = master.getMetaTableLocator(). + getMetaRegionLocation(master.getZooKeeper()); + + return MasterProtos.GetMetaLocationResponse. + newBuilder().setMetaLocation(ProtobufUtil.toServerName(metaLocation)).build(); + } + + @Override + public MasterProtos.GetTableStateResponse getTableState(RpcController controller, + MasterProtos.GetTableStateRequest request) throws ServiceException { + TableName tName = ProtobufUtil.toTableName(request.getTableName()); + + ZooKeeperProtos.Table.State state = master.getAssignmentManager().getTableStateManager(). + getTableState(tName); + + ZooKeeperProtos.Table table = ZooKeeperProtos.Table.newBuilder().setState(state).build(); + + return MasterProtos.GetTableStateResponse.newBuilder().setTableState(table).build(); + } } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ClusterIdResource.java hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ClusterIdResource.java new file mode 100644 index 0000000..616c8c3 --- /dev/null +++ hbase-server/src/main/java/org/apache/hadoop/hbase/rest/ClusterIdResource.java @@ -0,0 +1,81 @@ +/* + * + * 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.rest; + +import java.io.IOException; + +import javax.ws.rs.GET; +import javax.ws.rs.Produces; +import javax.ws.rs.core.CacheControl; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.UriInfo; +import javax.ws.rs.core.Response.ResponseBuilder; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hbase.rest.model.ClusterIdModel; + +/** + * Returns {@link org.apache.hadoop.hbase.ClusterId} via HTTP. + */ +@InterfaceAudience.Private +public class ClusterIdResource extends ResourceBase { + private static final Log LOG = LogFactory.getLog(ClusterIdResource.class); + + static CacheControl cacheControl; + static { + cacheControl = new CacheControl(); + cacheControl.setNoCache(true); + cacheControl.setNoTransform(false); + } + + /** + * Constructor + * @throws IOException + */ + public ClusterIdResource() throws IOException { + super(); + } + + @GET + @Produces({MIMETYPE_TEXT, MIMETYPE_JSON}) + public Response get(final @Context UriInfo uriInfo) { + if (LOG.isDebugEnabled()) { + LOG.debug("GET " + uriInfo.getAbsolutePath()); + } + servlet.getMetrics().incrementRequests(1); + try { + ClusterIdModel model = new ClusterIdModel(); + model.setClusterId(servlet.getAdmin().getClusterStatus().getClusterId()); + ResponseBuilder response = Response.ok(model); + response.cacheControl(cacheControl); + servlet.getMetrics().incrementSucessfulGetRequests(1); + return response.build(); + } catch (IOException e) { + servlet.getMetrics().incrementFailedGetRequests(1); + return Response.status(Response.Status.SERVICE_UNAVAILABLE) + .type(MIMETYPE_TEXT).entity("Unavailable" + CRLF) + .build(); + } + } +} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java index 0bae886..1c17ab6 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/rest/RootResource.java @@ -98,6 +98,11 @@ public class RootResource extends ResourceBase { return new VersionResource(); } + @Path("clusterId") + public ClusterIdResource getClusterIdResource() throws IOException { + return new ClusterIdResource(); + } + @Path("{table}") public TableResource getTableResource( final @PathParam("table") String table) throws IOException { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java deleted file mode 100644 index 77feb24..0000000 --- hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java +++ /dev/null @@ -1,506 +0,0 @@ -/* - * - * 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.rest.client; - -import java.io.IOException; -import java.io.InputStream; -import java.util.Collections; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.commons.httpclient.Header; -import org.apache.commons.httpclient.HttpClient; -import org.apache.commons.httpclient.HttpMethod; -import org.apache.commons.httpclient.HttpVersion; -import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager; -import org.apache.commons.httpclient.URI; -import org.apache.commons.httpclient.methods.ByteArrayRequestEntity; -import org.apache.commons.httpclient.methods.DeleteMethod; -import org.apache.commons.httpclient.methods.GetMethod; -import org.apache.commons.httpclient.methods.HeadMethod; -import org.apache.commons.httpclient.methods.PostMethod; -import org.apache.commons.httpclient.methods.PutMethod; -import org.apache.commons.httpclient.params.HttpClientParams; -import org.apache.commons.httpclient.params.HttpConnectionManagerParams; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; - -/** - * A wrapper around HttpClient which provides some useful function and - * semantics for interacting with the REST gateway. - */ -@InterfaceAudience.Public -@InterfaceStability.Stable -public class Client { - public static final Header[] EMPTY_HEADER_ARRAY = new Header[0]; - - private static final Log LOG = LogFactory.getLog(Client.class); - - private HttpClient httpClient; - private Cluster cluster; - - private Map extraHeaders; - - /** - * Default Constructor - */ - public Client() { - this(null); - } - - /** - * Constructor - * @param cluster the cluster definition - */ - public Client(Cluster cluster) { - this.cluster = cluster; - MultiThreadedHttpConnectionManager manager = - new MultiThreadedHttpConnectionManager(); - HttpConnectionManagerParams managerParams = manager.getParams(); - managerParams.setConnectionTimeout(2000); // 2 s - managerParams.setDefaultMaxConnectionsPerHost(10); - managerParams.setMaxTotalConnections(100); - extraHeaders = new ConcurrentHashMap(); - this.httpClient = new HttpClient(manager); - HttpClientParams clientParams = httpClient.getParams(); - clientParams.setVersion(HttpVersion.HTTP_1_1); - } - - /** - * Shut down the client. Close any open persistent connections. - */ - public void shutdown() { - MultiThreadedHttpConnectionManager manager = - (MultiThreadedHttpConnectionManager) httpClient.getHttpConnectionManager(); - manager.shutdown(); - } - - /** - * @return the wrapped HttpClient - */ - public HttpClient getHttpClient() { - return httpClient; - } - - /** - * Add extra headers. These extra headers will be applied to all http - * methods before they are removed. If any header is not used any more, - * client needs to remove it explicitly. - */ - public void addExtraHeader(final String name, final String value) { - extraHeaders.put(name, value); - } - - /** - * Get an extra header value. - */ - public String getExtraHeader(final String name) { - return extraHeaders.get(name); - } - - /** - * Get all extra headers (read-only). - */ - public Map getExtraHeaders() { - return Collections.unmodifiableMap(extraHeaders); - } - - /** - * Remove an extra header. - */ - public void removeExtraHeader(final String name) { - extraHeaders.remove(name); - } - - /** - * Execute a transaction method given only the path. Will select at random - * one of the members of the supplied cluster definition and iterate through - * the list until a transaction can be successfully completed. The - * definition of success here is a complete HTTP transaction, irrespective - * of result code. - * @param cluster the cluster definition - * @param method the transaction method - * @param headers HTTP header values to send - * @param path the properly urlencoded path - * @return the HTTP response code - * @throws IOException - */ - public int executePathOnly(Cluster cluster, HttpMethod method, - Header[] headers, String path) throws IOException { - IOException lastException; - if (cluster.nodes.size() < 1) { - throw new IOException("Cluster is empty"); - } - int start = (int)Math.round((cluster.nodes.size() - 1) * Math.random()); - int i = start; - do { - cluster.lastHost = cluster.nodes.get(i); - try { - StringBuilder sb = new StringBuilder(); - sb.append("http://"); - sb.append(cluster.lastHost); - sb.append(path); - URI uri = new URI(sb.toString(), true); - return executeURI(method, headers, uri.toString()); - } catch (IOException e) { - lastException = e; - } - } while (++i != start && i < cluster.nodes.size()); - throw lastException; - } - - /** - * Execute a transaction method given a complete URI. - * @param method the transaction method - * @param headers HTTP header values to send - * @param uri a properly urlencoded URI - * @return the HTTP response code - * @throws IOException - */ - public int executeURI(HttpMethod method, Header[] headers, String uri) - throws IOException { - method.setURI(new URI(uri, true)); - for (Map.Entry e: extraHeaders.entrySet()) { - method.addRequestHeader(e.getKey(), e.getValue()); - } - if (headers != null) { - for (Header header: headers) { - method.addRequestHeader(header); - } - } - long startTime = System.currentTimeMillis(); - int code = httpClient.executeMethod(method); - long endTime = System.currentTimeMillis(); - if (LOG.isDebugEnabled()) { - LOG.debug(method.getName() + " " + uri + " " + code + " " + - method.getStatusText() + " in " + (endTime - startTime) + " ms"); - } - return code; - } - - /** - * Execute a transaction method. Will call either executePathOnly - * or executeURI depending on whether a path only is supplied in - * 'path', or if a complete URI is passed instead, respectively. - * @param cluster the cluster definition - * @param method the HTTP method - * @param headers HTTP header values to send - * @param path the properly urlencoded path or URI - * @return the HTTP response code - * @throws IOException - */ - public int execute(Cluster cluster, HttpMethod method, Header[] headers, - String path) throws IOException { - if (path.startsWith("/")) { - return executePathOnly(cluster, method, headers, path); - } - return executeURI(method, headers, path); - } - - /** - * @return the cluster definition - */ - public Cluster getCluster() { - return cluster; - } - - /** - * @param cluster the cluster definition - */ - public void setCluster(Cluster cluster) { - this.cluster = cluster; - } - - /** - * Send a HEAD request - * @param path the path or URI - * @return a Response object with response detail - * @throws IOException - */ - public Response head(String path) throws IOException { - return head(cluster, path, null); - } - - /** - * Send a HEAD request - * @param cluster the cluster definition - * @param path the path or URI - * @param headers the HTTP headers to include in the request - * @return a Response object with response detail - * @throws IOException - */ - public Response head(Cluster cluster, String path, Header[] headers) - throws IOException { - HeadMethod method = new HeadMethod(); - try { - int code = execute(cluster, method, null, path); - headers = method.getResponseHeaders(); - return new Response(code, headers, null); - } finally { - method.releaseConnection(); - } - } - - /** - * Send a GET request - * @param path the path or URI - * @return a Response object with response detail - * @throws IOException - */ - public Response get(String path) throws IOException { - return get(cluster, path); - } - - /** - * Send a GET request - * @param cluster the cluster definition - * @param path the path or URI - * @return a Response object with response detail - * @throws IOException - */ - public Response get(Cluster cluster, String path) throws IOException { - return get(cluster, path, EMPTY_HEADER_ARRAY); - } - - /** - * Send a GET request - * @param path the path or URI - * @param accept Accept header value - * @return a Response object with response detail - * @throws IOException - */ - public Response get(String path, String accept) throws IOException { - return get(cluster, path, accept); - } - - /** - * Send a GET request - * @param cluster the cluster definition - * @param path the path or URI - * @param accept Accept header value - * @return a Response object with response detail - * @throws IOException - */ - public Response get(Cluster cluster, String path, String accept) - throws IOException { - Header[] headers = new Header[1]; - headers[0] = new Header("Accept", accept); - return get(cluster, path, headers); - } - - /** - * Send a GET request - * @param path the path or URI - * @param headers the HTTP headers to include in the request, - * Accept must be supplied - * @return a Response object with response detail - * @throws IOException - */ - public Response get(String path, Header[] headers) throws IOException { - return get(cluster, path, headers); - } - - /** - * Send a GET request - * @param c the cluster definition - * @param path the path or URI - * @param headers the HTTP headers to include in the request - * @return a Response object with response detail - * @throws IOException - */ - public Response get(Cluster c, String path, Header[] headers) - throws IOException { - GetMethod method = new GetMethod(); - try { - int code = execute(c, method, headers, path); - headers = method.getResponseHeaders(); - byte[] body = method.getResponseBody(); - InputStream in = method.getResponseBodyAsStream(); - return new Response(code, headers, body, in); - } finally { - method.releaseConnection(); - } - } - - /** - * Send a PUT request - * @param path the path or URI - * @param contentType the content MIME type - * @param content the content bytes - * @return a Response object with response detail - * @throws IOException - */ - public Response put(String path, String contentType, byte[] content) - throws IOException { - return put(cluster, path, contentType, content); - } - - /** - * Send a PUT request - * @param cluster the cluster definition - * @param path the path or URI - * @param contentType the content MIME type - * @param content the content bytes - * @return a Response object with response detail - * @throws IOException - */ - public Response put(Cluster cluster, String path, String contentType, - byte[] content) throws IOException { - Header[] headers = new Header[1]; - headers[0] = new Header("Content-Type", contentType); - return put(cluster, path, headers, content); - } - - /** - * Send a PUT request - * @param path the path or URI - * @param headers the HTTP headers to include, Content-Type must be - * supplied - * @param content the content bytes - * @return a Response object with response detail - * @throws IOException - */ - public Response put(String path, Header[] headers, byte[] content) - throws IOException { - return put(cluster, path, headers, content); - } - - /** - * Send a PUT request - * @param cluster the cluster definition - * @param path the path or URI - * @param headers the HTTP headers to include, Content-Type must be - * supplied - * @param content the content bytes - * @return a Response object with response detail - * @throws IOException - */ - public Response put(Cluster cluster, String path, Header[] headers, - byte[] content) throws IOException { - PutMethod method = new PutMethod(); - try { - method.setRequestEntity(new ByteArrayRequestEntity(content)); - int code = execute(cluster, method, headers, path); - headers = method.getResponseHeaders(); - content = method.getResponseBody(); - return new Response(code, headers, content); - } finally { - method.releaseConnection(); - } - } - - /** - * Send a POST request - * @param path the path or URI - * @param contentType the content MIME type - * @param content the content bytes - * @return a Response object with response detail - * @throws IOException - */ - public Response post(String path, String contentType, byte[] content) - throws IOException { - return post(cluster, path, contentType, content); - } - - /** - * Send a POST request - * @param cluster the cluster definition - * @param path the path or URI - * @param contentType the content MIME type - * @param content the content bytes - * @return a Response object with response detail - * @throws IOException - */ - public Response post(Cluster cluster, String path, String contentType, - byte[] content) throws IOException { - Header[] headers = new Header[1]; - headers[0] = new Header("Content-Type", contentType); - return post(cluster, path, headers, content); - } - - /** - * Send a POST request - * @param path the path or URI - * @param headers the HTTP headers to include, Content-Type must be - * supplied - * @param content the content bytes - * @return a Response object with response detail - * @throws IOException - */ - public Response post(String path, Header[] headers, byte[] content) - throws IOException { - return post(cluster, path, headers, content); - } - - /** - * Send a POST request - * @param cluster the cluster definition - * @param path the path or URI - * @param headers the HTTP headers to include, Content-Type must be - * supplied - * @param content the content bytes - * @return a Response object with response detail - * @throws IOException - */ - public Response post(Cluster cluster, String path, Header[] headers, - byte[] content) throws IOException { - PostMethod method = new PostMethod(); - try { - method.setRequestEntity(new ByteArrayRequestEntity(content)); - int code = execute(cluster, method, headers, path); - headers = method.getResponseHeaders(); - content = method.getResponseBody(); - return new Response(code, headers, content); - } finally { - method.releaseConnection(); - } - } - - /** - * Send a DELETE request - * @param path the path or URI - * @return a Response object with response detail - * @throws IOException - */ - public Response delete(String path) throws IOException { - return delete(cluster, path); - } - - /** - * Send a DELETE request - * @param cluster the cluster definition - * @param path the path or URI - * @return a Response object with response detail - * @throws IOException - */ - public Response delete(Cluster cluster, String path) throws IOException { - DeleteMethod method = new DeleteMethod(); - try { - int code = execute(cluster, method, null, path); - Header[] headers = method.getResponseHeaders(); - byte[] content = method.getResponseBody(); - return new Response(code, headers, content); - } finally { - method.releaseConnection(); - } - } -} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java deleted file mode 100644 index f9fbdc2..0000000 --- hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/Cluster.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * - * 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.rest.client; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; - -/** - * A list of 'host:port' addresses of HTTP servers operating as a single - * entity, for example multiple redundant web service gateways. - */ -@InterfaceAudience.Public -@InterfaceStability.Stable -public class Cluster { - protected List nodes = - Collections.synchronizedList(new ArrayList()); - protected String lastHost; - - /** - * Constructor - */ - public Cluster() {} - - /** - * Constructor - * @param nodes a list of service locations, in 'host:port' format - */ - public Cluster(List nodes) { - nodes.addAll(nodes); - } - - /** - * @return true if no locations have been added, false otherwise - */ - public boolean isEmpty() { - return nodes.isEmpty(); - } - - /** - * Add a node to the cluster - * @param node the service location in 'host:port' format - */ - public Cluster add(String node) { - nodes.add(node); - return this; - } - - /** - * Add a node to the cluster - * @param name host name - * @param port service port - */ - public Cluster add(String name, int port) { - StringBuilder sb = new StringBuilder(); - sb.append(name); - sb.append(':'); - sb.append(port); - return add(sb.toString()); - } - - /** - * Remove a node from the cluster - * @param node the service location in 'host:port' format - */ - public Cluster remove(String node) { - nodes.remove(node); - return this; - } - - /** - * Remove a node from the cluster - * @param name host name - * @param port service port - */ - public Cluster remove(String name, int port) { - StringBuilder sb = new StringBuilder(); - sb.append(name); - sb.append(':'); - sb.append(port); - return remove(sb.toString()); - } -} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java deleted file mode 100644 index c2810fe..0000000 --- hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java +++ /dev/null @@ -1,155 +0,0 @@ -/* - * - * 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.rest.client; - -import java.io.InputStream; - -import org.apache.commons.httpclient.Header; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; - -/** - * The HTTP result code, response headers, and body of a HTTP response. - */ -@InterfaceAudience.Public -@InterfaceStability.Stable -public class Response { - private int code; - private Header[] headers; - private byte[] body; - private InputStream stream; - - /** - * Constructor - * @param code the HTTP response code - */ - public Response(int code) { - this(code, null, null); - } - - /** - * Constructor - * @param code the HTTP response code - * @param headers the HTTP response headers - */ - public Response(int code, Header[] headers) { - this(code, headers, null); - } - - /** - * Constructor - * @param code the HTTP response code - * @param headers the HTTP response headers - * @param body the response body, can be null - */ - public Response(int code, Header[] headers, byte[] body) { - this.code = code; - this.headers = headers; - this.body = body; - } - - /** - * Constructor - * @param code the HTTP response code - * @param headers headers the HTTP response headers - * @param body the response body, can be null - * @param in Inputstream if the response had one. - */ - public Response(int code, Header[] headers, byte[] body, InputStream in) { - this.code = code; - this.headers = headers; - this.body = body; - this.stream = in; - } - - /** - * @return the HTTP response code - */ - public int getCode() { - return code; - } - - /** - * Gets the input stream instance. - * - * @return an instance of InputStream class. - */ - public InputStream getStream(){ - return this.stream; - } - - /** - * @return the HTTP response headers - */ - public Header[] getHeaders() { - return headers; - } - - public String getHeader(String key) { - for (Header header: headers) { - if (header.getName().equalsIgnoreCase(key)) { - return header.getValue(); - } - } - return null; - } - - /** - * @return the value of the Location header - */ - public String getLocation() { - return getHeader("Location"); - } - - /** - * @return true if a response body was sent - */ - public boolean hasBody() { - return body != null; - } - - /** - * @return the HTTP response body - */ - public byte[] getBody() { - return body; - } - - /** - * @param code the HTTP response code - */ - public void setCode(int code) { - this.code = code; - } - - /** - * @param headers the HTTP response headers - */ - public void setHeaders(Header[] headers) { - this.headers = headers; - } - - /** - * @param body the response body - */ - public void setBody(byte[] body) { - this.body = body; - } -} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/ClusterIdModel.java hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/ClusterIdModel.java new file mode 100644 index 0000000..6e32dcd --- /dev/null +++ hbase-server/src/main/java/org/apache/hadoop/hbase/rest/model/ClusterIdModel.java @@ -0,0 +1,74 @@ +/* + * + * 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.rest.model; + +import org.codehaus.jackson.annotate.JsonValue; + +import java.io.Serializable; + +import javax.xml.bind.annotation.XmlValue; + +import org.apache.hadoop.classification.InterfaceAudience; + +/** + * Simple representation of {@link org.apache.hadoop.hbase.ClusterId}. + *
+ * <complexType name="ClusterIdModel">
+ *   <attribute name="clusterId" type="string"></attribute>
+ * </complexType>
+ * 
+ */ +@InterfaceAudience.Private +public class ClusterIdModel implements Serializable { + private static final long serialVersionUID = 1L; + + private String clusterId; + + /** + * @return {@link org.apache.hadoop.hbase.ClusterId} of the cluster + */ + @XmlValue + public String getClusterId() { + return clusterId; + } + + /** + * @param clusterId {@link org.apache.hadoop.hbase.ClusterId} of the cluster + */ + public void setClusterId(String clusterId) { + this.clusterId = clusterId; + } + + /* (non-Javadoc) + * @see java.lang.Object#toString() + */ + @JsonValue + @Override + public String toString() { + return clusterId; + } + + //needed for jackson deserialization + private static ClusterIdModel valueOf(String value) { + ClusterIdModel clusterIdModel = new ClusterIdModel(); + clusterIdModel.setClusterId(value); + return clusterIdModel; + } +} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableStateManager.java hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableStateManager.java index 1aff12f..51a360b 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableStateManager.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTableStateManager.java @@ -261,6 +261,13 @@ public class ZKTableStateManager implements TableStateManager { } } + @Override + public ZooKeeperProtos.Table.State getTableState(TableName tableName) { + synchronized (this.cache) { + return this.cache.get(tableName); + } + } + /** * Gets a list of all the tables of specified states in zookeeper. * @return Set of tables of specified states, empty Set if none diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 53a3b93..6964db2 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -897,6 +897,13 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { this.hbaseCluster = new MiniHBaseCluster(c, numMasters, numSlaves, masterClass, regionserverClass); // Don't leave here till we've done a successful scan of the hbase:meta + + // Since mini HBase cluster binds to random TCP port, need to capture the actual + // port bound here and put back in configuration. + int masterPort = ((MiniHBaseCluster) hbaseCluster).getMaster().getServerName().getPort(); + this.conf.setInt(HConstants.MASTER_PORT, masterPort); + c.setInt(HConstants.MASTER_PORT, masterPort); + HTable t = new HTable(c, TableName.META_TABLE_NAME); ResultScanner s = t.getScanner(new Scan()); while (s.next() != null) { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java index 5edb69f..40a9f09 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java @@ -214,6 +214,14 @@ public class MiniHBaseCluster extends HBaseCluster { hbaseCluster = new LocalHBaseCluster(conf, nMasterNodes, 0, masterClass, regionserverClass); + while (hbaseCluster.getActiveMaster() == null) { + Thread.sleep(100); + } + + int actualMasterPort = hbaseCluster.getActiveMaster().getMasterRpcServices(). + getSocketAddress().getPort(); + conf.setInt(HConstants.MASTER_PORT, actualMasterPort); + // manually add the regionservers as other users for (int i=0; i