HBASE-12072 Standardize retry handling for master operations

This commit is contained in:
Enis Soztutar 2014-11-26 12:09:44 -08:00
parent 54627ea6dc
commit 02963b202a
23 changed files with 704 additions and 389 deletions

View File

@ -397,12 +397,10 @@ public class MetaTableAccessor {
TableName tableName, final boolean excludeOfflinedSplitParents)
throws IOException {
List<Pair<HRegionInfo, ServerName>> result;
try {
result = getTableRegionsAndLocations(connection, tableName,
excludeOfflinedSplitParents);
} catch (InterruptedException e) {
throw (InterruptedIOException)new InterruptedIOException().initCause(e);
}
result = getTableRegionsAndLocations(connection, tableName,
excludeOfflinedSplitParents);
return getListOfHRegionInfos(result);
}
@ -465,11 +463,10 @@ public class MetaTableAccessor {
* @param tableName table we're looking for
* @return Return list of regioninfos and server.
* @throws IOException
* @throws InterruptedException
*/
public static List<Pair<HRegionInfo, ServerName>>
getTableRegionsAndLocations(Connection connection, TableName tableName)
throws IOException, InterruptedException {
throws IOException {
return getTableRegionsAndLocations(connection, tableName, true);
}
@ -479,11 +476,10 @@ public class MetaTableAccessor {
* @param tableName table to work with
* @return Return list of regioninfos and server addresses.
* @throws IOException
* @throws InterruptedException
*/
public static List<Pair<HRegionInfo, ServerName>> getTableRegionsAndLocations(
Connection connection, final TableName tableName,
final boolean excludeOfflinedSplitParents) throws IOException, InterruptedException {
final boolean excludeOfflinedSplitParents) throws IOException {
if (tableName.equals(TableName.META_TABLE_NAME)) {
throw new IOException("This method can't be used to locate meta regions;"
+ " use MetaTableLocator instead");

View File

@ -20,13 +20,14 @@ package org.apache.hadoop.hbase;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
/**
* Thrown when we are asked to operate on a region we know nothing about.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class UnknownRegionException extends RegionException {
public class UnknownRegionException extends DoNotRetryRegionException {
private static final long serialVersionUID = 1968858760475205392L;
public UnknownRegionException(String regionName) {

View File

@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase.client;
import java.io.Closeable;
import java.io.IOException;
import java.util.List;
@ -27,7 +28,6 @@ import java.util.regex.Pattern;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
@ -55,8 +54,6 @@ import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
import org.apache.hadoop.hbase.util.Pair;
import com.google.protobuf.ServiceException;
/**
* The administrative API for HBase. Obtain an instance from an {@link Connection#getAdmin()} and
* call {@link #close()} afterwards.
@ -84,13 +81,6 @@ public interface Admin extends Abortable, Closeable {
*/
Connection getConnection();
/**
* @return - true if the master server is running. Throws an exception otherwise.
* @throws ZooKeeperConnectionException
* @throws MasterNotRunningException
*/
boolean isMasterRunning() throws MasterNotRunningException, ZooKeeperConnectionException;
/**
* @param tableName Table to check.
* @return True if table exists already.
@ -99,9 +89,7 @@ public interface Admin extends Abortable, Closeable {
boolean tableExists(final TableName tableName) throws IOException;
/**
* List all the userspace tables. In other words, scan the hbase:meta table. If we wanted this to
* be really fast, we could implement a special catalog table that just contains table names and
* their descriptors. Right now, it only exists as part of the hbase:meta table's region info.
* List all the userspace tables.
*
* @return - returns an array of HTableDescriptors
* @throws IOException if a remote or network exception occurs
@ -500,36 +488,32 @@ public interface Admin extends Abortable, Closeable {
*
* @param tableName table to flush
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
void flush(final TableName tableName) throws IOException, InterruptedException;
void flush(final TableName tableName) throws IOException;
/**
* Flush an individual region. Synchronous operation.
*
* @param regionName region to flush
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
void flushRegion(final byte[] regionName) throws IOException, InterruptedException;
void flushRegion(final byte[] regionName) throws IOException;
/**
* Compact a table. Asynchronous operation.
*
* @param tableName table to compact
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
void compact(final TableName tableName) throws IOException, InterruptedException;
void compact(final TableName tableName) throws IOException;
/**
* Compact an individual region. Asynchronous operation.
*
* @param regionName region to compact
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
void compactRegion(final byte[] regionName) throws IOException, InterruptedException;
void compactRegion(final byte[] regionName) throws IOException;
/**
* Compact a column family within a table. Asynchronous operation.
@ -537,10 +521,9 @@ public interface Admin extends Abortable, Closeable {
* @param tableName table to compact
* @param columnFamily column family within a table
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
void compact(final TableName tableName, final byte[] columnFamily)
throws IOException, InterruptedException;
throws IOException;
/**
* Compact a column family within a region. Asynchronous operation.
@ -548,28 +531,25 @@ public interface Admin extends Abortable, Closeable {
* @param regionName region to compact
* @param columnFamily column family within a region
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
void compactRegion(final byte[] regionName, final byte[] columnFamily)
throws IOException, InterruptedException;
throws IOException;
/**
* Major compact a table. Asynchronous operation.
*
* @param tableName table to major compact
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
void majorCompact(TableName tableName) throws IOException, InterruptedException;
void majorCompact(TableName tableName) throws IOException;
/**
* Major compact a table or an individual region. Asynchronous operation.
*
* @param regionName region to major compact
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
void majorCompactRegion(final byte[] regionName) throws IOException, InterruptedException;
void majorCompactRegion(final byte[] regionName) throws IOException;
/**
* Major compact a column family within a table. Asynchronous operation.
@ -577,10 +557,9 @@ public interface Admin extends Abortable, Closeable {
* @param tableName table to major compact
* @param columnFamily column family within a table
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
void majorCompact(TableName tableName, final byte[] columnFamily)
throws IOException, InterruptedException;
throws IOException;
/**
* Major compact a column family within region. Asynchronous operation.
@ -588,10 +567,9 @@ public interface Admin extends Abortable, Closeable {
* @param regionName egion to major compact
* @param columnFamily column family within a region
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
void majorCompactRegion(final byte[] regionName, final byte[] columnFamily)
throws IOException, InterruptedException;
throws IOException;
/**
* Compact all regions on the region server
@ -615,20 +593,15 @@ public interface Admin extends Abortable, Closeable {
* Here is an example: <code> host187.example.com,60020,1289493121758</code>
* @throws UnknownRegionException Thrown if we can't find a region named
* <code>encodedRegionName</code>
* @throws ZooKeeperConnectionException
* @throws MasterNotRunningException
*/
void move(final byte[] encodedRegionName, final byte[] destServerName)
throws HBaseIOException, MasterNotRunningException, ZooKeeperConnectionException;
throws IOException;
/**
* @param regionName Region name to assign.
* @throws MasterNotRunningException
* @throws ZooKeeperConnectionException
* @throws IOException
*/
void assign(final byte[] regionName)
throws MasterNotRunningException, ZooKeeperConnectionException, IOException;
throws IOException;
/**
* Unassign a region from current hosting regionserver. Region will then be assigned to a
@ -638,12 +611,9 @@ public interface Admin extends Abortable, Closeable {
* @param regionName Region to unassign. Will clear any existing RegionPlan if one found.
* @param force If true, force unassign (Will remove region from regions-in-transition too if
* present. If results in double assignment use hbck -fix to resolve. To be used by experts).
* @throws MasterNotRunningException
* @throws ZooKeeperConnectionException
* @throws IOException
*/
void unassign(final byte[] regionName, final boolean force)
throws MasterNotRunningException, ZooKeeperConnectionException, IOException;
throws IOException;
/**
* Offline specified region from master's in-memory state. It will not attempt to reassign the
@ -660,12 +630,11 @@ public interface Admin extends Abortable, Closeable {
/**
* Turn the load balancer on or off.
*
* @param on If true, enable balancer. If false, disable balancer.
* @param synchronous If true, it waits until current balance() call, if outstanding, to return.
* @return Previous balancer value
*/
boolean setBalancerRunning(final boolean on, final boolean synchronous)
throws MasterNotRunningException, ZooKeeperConnectionException;
throws IOException;
/**
* Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the
@ -673,35 +642,28 @@ public interface Admin extends Abortable, Closeable {
*
* @return True if balancer ran, false otherwise.
*/
boolean balancer()
throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException;
boolean balancer() throws IOException;
/**
* Enable/Disable the catalog janitor
*
* @param enable if true enables the catalog janitor
* @return the previous state
* @throws ServiceException
* @throws MasterNotRunningException
*/
boolean enableCatalogJanitor(boolean enable) throws ServiceException, MasterNotRunningException;
boolean enableCatalogJanitor(boolean enable) throws IOException;
/**
* Ask for a scan of the catalog table
*
* @return the number of entries cleaned
* @throws ServiceException
* @throws MasterNotRunningException
*/
int runCatalogScan() throws ServiceException, MasterNotRunningException;
int runCatalogScan() throws IOException;
/**
* Query on the catalog janitor state (Enabled/Disabled?)
*
* @throws ServiceException
* @throws org.apache.hadoop.hbase.MasterNotRunningException
*/
boolean isCatalogJanitorEnabled() throws ServiceException, MasterNotRunningException;
boolean isCatalogJanitorEnabled() throws IOException;
/**
* Merge two regions. Asynchronous operation.
@ -720,18 +682,16 @@ public interface Admin extends Abortable, Closeable {
*
* @param tableName table to split
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
void split(final TableName tableName) throws IOException, InterruptedException;
void split(final TableName tableName) throws IOException;
/**
* Split an individual region. Asynchronous operation.
*
* @param regionName region to split
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
void splitRegion(final byte[] regionName) throws IOException, InterruptedException;
void splitRegion(final byte[] regionName) throws IOException;
/**
* Split a table. Asynchronous operation.
@ -739,10 +699,9 @@ public interface Admin extends Abortable, Closeable {
* @param tableName table to split
* @param splitPoint the explicit position to split on
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException interrupt exception occurred
*/
void split(final TableName tableName, final byte[] splitPoint)
throws IOException, InterruptedException;
throws IOException;
/**
* Split an individual region. Asynchronous operation.
@ -750,10 +709,9 @@ public interface Admin extends Abortable, Closeable {
* @param regionName region to split
* @param splitPoint the explicit position to split on
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException interrupt exception occurred
*/
void splitRegion(final byte[] regionName, final byte[] splitPoint)
throws IOException, InterruptedException;
throws IOException;
/**
* Modify an existing table, more IRB friendly version. Asynchronous operation. This means that
@ -763,7 +721,8 @@ public interface Admin extends Abortable, Closeable {
* @param htd modified description of the table
* @throws IOException if a remote or network exception occurs
*/
void modifyTable(final TableName tableName, final HTableDescriptor htd) throws IOException;
void modifyTable(final TableName tableName, final HTableDescriptor htd)
throws IOException;
/**
* Shuts down the HBase cluster
@ -806,7 +765,8 @@ public interface Admin extends Abortable, Closeable {
* @param descriptor descriptor which describes the new namespace
* @throws IOException
*/
void createNamespace(final NamespaceDescriptor descriptor) throws IOException;
void createNamespace(final NamespaceDescriptor descriptor)
throws IOException;
/**
* Modify an existing namespace
@ -814,7 +774,8 @@ public interface Admin extends Abortable, Closeable {
* @param descriptor descriptor which describes the new namespace
* @throws IOException
*/
void modifyNamespace(final NamespaceDescriptor descriptor) throws IOException;
void modifyNamespace(final NamespaceDescriptor descriptor)
throws IOException;
/**
* Delete an existing namespace. Only empty namespaces (no tables) can be removed.
@ -831,7 +792,8 @@ public interface Admin extends Abortable, Closeable {
* @return A descriptor
* @throws IOException
*/
NamespaceDescriptor getNamespaceDescriptor(final String name) throws IOException;
NamespaceDescriptor getNamespaceDescriptor(final String name)
throws IOException;
/**
* List available namespace descriptors
@ -839,7 +801,8 @@ public interface Admin extends Abortable, Closeable {
* @return List of descriptors
* @throws IOException
*/
NamespaceDescriptor[] listNamespaceDescriptors() throws IOException;
NamespaceDescriptor[] listNamespaceDescriptors()
throws IOException;
/**
* Get list of table descriptors by namespace
@ -848,7 +811,8 @@ public interface Admin extends Abortable, Closeable {
* @return A descriptor
* @throws IOException
*/
HTableDescriptor[] listTableDescriptorsByNamespace(final String name) throws IOException;
HTableDescriptor[] listTableDescriptorsByNamespace(final String name)
throws IOException;
/**
* Get list of table names by namespace
@ -857,7 +821,8 @@ public interface Admin extends Abortable, Closeable {
* @return The list of table names in the namespace
* @throws IOException
*/
TableName[] listTableNamesByNamespace(final String name) throws IOException;
TableName[] listTableNamesByNamespace(final String name)
throws IOException;
/**
* Get the regions of a given table.
@ -866,7 +831,8 @@ public interface Admin extends Abortable, Closeable {
* @return List of {@link HRegionInfo}.
* @throws IOException
*/
List<HRegionInfo> getTableRegions(final TableName tableName) throws IOException;
List<HRegionInfo> getTableRegions(final TableName tableName)
throws IOException;
@Override
void close() throws IOException;
@ -878,7 +844,8 @@ public interface Admin extends Abortable, Closeable {
* @return HTD[] the tableDescriptor
* @throws IOException if a remote or network exception occurs
*/
HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames) throws IOException;
HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames)
throws IOException;
/**
* Get tableDescriptors
@ -887,7 +854,8 @@ public interface Admin extends Abortable, Closeable {
* @return HTD[] the tableDescriptor
* @throws IOException if a remote or network exception occurs
*/
HTableDescriptor[] getTableDescriptors(List<String> names) throws IOException;
HTableDescriptor[] getTableDescriptors(List<String> names)
throws IOException;
/**
* Roll the log writer. I.e. for filesystem based write ahead logs, start writing to a new file.
@ -907,7 +875,7 @@ public interface Admin extends Abortable, Closeable {
* @return an array of master coprocessors
* @see org.apache.hadoop.hbase.ClusterStatus#getMasterCoprocessors()
*/
String[] getMasterCoprocessors();
String[] getMasterCoprocessors() throws IOException;
/**
* Get the current compaction state of a table. It could be in a major compaction, a minor
@ -916,10 +884,9 @@ public interface Admin extends Abortable, Closeable {
* @param tableName table to examine
* @return the current compaction state
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
AdminProtos.GetRegionInfoResponse.CompactionState getCompactionState(final TableName tableName)
throws IOException, InterruptedException;
throws IOException;
/**
* Get the current compaction state of region. It could be in a major compaction, a minor
@ -928,10 +895,9 @@ public interface Admin extends Abortable, Closeable {
* @param regionName region to examine
* @return the current compaction state
* @throws IOException if a remote or network exception occurs
* @throws InterruptedException
*/
AdminProtos.GetRegionInfoResponse.CompactionState getCompactionStateForRegion(
final byte[] regionName) throws IOException, InterruptedException;
final byte[] regionName) throws IOException;
/**
* Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be
@ -1115,7 +1081,7 @@ public interface Admin extends Abortable, Closeable {
* @throws IllegalArgumentException if the specified table has not a valid name
*/
void cloneSnapshot(final byte[] snapshotName, final TableName tableName)
throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException;
throws IOException, TableExistsException, RestoreSnapshotException;
/**
* Create a new table by cloning the snapshot content.
@ -1128,7 +1094,7 @@ public interface Admin extends Abortable, Closeable {
* @throws IllegalArgumentException if the specified table has not a valid name
*/
void cloneSnapshot(final String snapshotName, final TableName tableName)
throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException;
throws IOException, TableExistsException, RestoreSnapshotException;
/**
* Execute a distributed procedure on a cluster.

View File

@ -39,8 +39,10 @@ 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
* @deprecated this has been deprecated without a replacement */
@Override
@Deprecated
boolean isMasterRunning()
throws MasterNotRunningException, ZooKeeperConnectionException;

View File

@ -44,6 +44,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Chore;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
@ -850,6 +851,7 @@ class ConnectionManager {
* @throws MasterNotRunningException - if the master is not running
* @throws ZooKeeperConnectionException
*/
@Deprecated
@Override
public boolean isMasterRunning()
throws MasterNotRunningException, ZooKeeperConnectionException {
@ -1450,18 +1452,14 @@ class ConnectionManager {
* @return A stub to do <code>intf</code> against the master
* @throws MasterNotRunningException
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings (value="SWL_SLEEP_WITH_LOCK_HELD")
Object makeStub() throws MasterNotRunningException {
Object makeStub() throws IOException {
// The lock must be at the beginning to prevent multiple master creations
// (and leaks) in a multithread context
synchronized (masterAndZKLock) {
Exception exceptionCaught = null;
Object stub = null;
int tries = 0;
while (!closed && stub == null) {
tries++;
if (!closed) {
try {
stub = makeStubNoRetries();
return makeStubNoRetries();
} catch (IOException e) {
exceptionCaught = e;
} catch (KeeperException e) {
@ -1470,34 +1468,10 @@ class ConnectionManager {
exceptionCaught = e;
}
if (exceptionCaught != null)
// It failed. If it's not the last try, we're going to wait a little
if (tries < numTries && !ExceptionUtil.isInterrupt(exceptionCaught)) {
// tries at this point is 1 or more; decrement to start from 0.
long pauseTime = ConnectionUtils.getPauseTime(pause, tries - 1);
LOG.info("getMaster attempt " + tries + " of " + numTries +
" failed; retrying after sleep of " + pauseTime + ", exception=" +
exceptionCaught);
try {
Thread.sleep(pauseTime);
} catch (InterruptedException e) {
throw new MasterNotRunningException(
"Thread was interrupted while trying to connect to master.", e);
}
} else {
// Enough tries, we stop now
LOG.info("getMaster attempt " + tries + " of " + numTries +
" failed; no more retrying.", exceptionCaught);
throw new MasterNotRunningException(exceptionCaught);
}
throw new MasterNotRunningException(exceptionCaught);
} else {
throw new DoNotRetryIOException("Connection was closed while trying to get master");
}
if (stub == null) {
// implies this.closed true
throw new MasterNotRunningException("Connection was closed while trying to get master");
}
return stub;
}
}
}
@ -1513,8 +1487,7 @@ class ConnectionManager {
}
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings("SWL_SLEEP_WITH_LOCK_HELD")
MasterService.BlockingInterface makeStub() throws MasterNotRunningException {
MasterService.BlockingInterface makeStub() throws IOException {
return (MasterService.BlockingInterface)super.makeStub();
}
@ -1720,7 +1693,14 @@ class ConnectionManager {
synchronized (masterAndZKLock) {
if (!isKeepAliveMasterConnectedAndRunning(this.masterServiceState)) {
MasterServiceStubMaker stubMaker = new MasterServiceStubMaker();
this.masterServiceState.stub = stubMaker.makeStub();
try {
this.masterServiceState.stub = stubMaker.makeStub();
} catch (MasterNotRunningException ex) {
throw ex;
} catch (IOException e) {
// rethrow as MasterNotRunningException so that we can keep the method sig
throw new MasterNotRunningException(e);
}
}
resetMasterServiceState(this.masterServiceState);
}
@ -2392,6 +2372,10 @@ class ConnectionManager {
close();
}
/**
* @deprecated Use {@link Admin#listTables()} instead
*/
@Deprecated
@Override
public HTableDescriptor[] listTables() throws IOException {
MasterKeepAliveConnection master = getKeepAliveMasterService();
@ -2406,6 +2390,10 @@ class ConnectionManager {
}
}
/**
* @deprecated Use {@link Admin#listTableNames()} instead
*/
@Deprecated
@Override
public HTableDescriptor[] listTables(String regex) throws IOException {
MasterKeepAliveConnection master = getKeepAliveMasterService();
@ -2430,6 +2418,10 @@ class ConnectionManager {
return result;
}
/**
* @deprecated Use {@link Admin#listTableNames()} instead
*/
@Deprecated
@Override
public TableName[] listTableNames() throws IOException {
MasterKeepAliveConnection master = getKeepAliveMasterService();
@ -2444,6 +2436,10 @@ class ConnectionManager {
}
}
/**
* @deprecated Use {@link Admin#getTableDescriptorsByTableName(List)} instead
*/
@Deprecated
@Override
public HTableDescriptor[] getHTableDescriptorsByTableName(
List<TableName> tableNames) throws IOException {
@ -2460,6 +2456,10 @@ class ConnectionManager {
}
}
/**
* @deprecated Use {@link Admin#getTableDescriptorsByTableName(List)} instead
*/
@Deprecated
@Override
public HTableDescriptor[] getHTableDescriptors(
List<String> names) throws IOException {
@ -2481,7 +2481,9 @@ class ConnectionManager {
* @param tableName table name
* @throws IOException if the connection to master fails or if the table
* is not found.
* @deprecated Use {@link Admin#getTableDescriptor(TableName)} instead
*/
@Deprecated
@Override
public HTableDescriptor getHTableDescriptor(final TableName tableName)
throws IOException {
@ -2503,6 +2505,10 @@ class ConnectionManager {
throw new TableNotFoundException(tableName.getNameAsString());
}
/**
* @deprecated Use {@link Admin#getTableDescriptor(TableName)} instead
*/
@Deprecated
@Override
public HTableDescriptor getHTableDescriptor(final byte[] tableName)
throws IOException {

View File

@ -0,0 +1,42 @@
/**
* 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.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Similar to RegionException, but disables retries.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class DoNotRetryRegionException extends DoNotRetryIOException {
private static final long serialVersionUID = 6907047686199321701L;
public DoNotRetryRegionException() {
super();
}
public DoNotRetryRegionException(String s) {
super(s);
}
}

View File

@ -18,8 +18,6 @@
*/
package org.apache.hadoop.hbase.client;
import java.io.Closeable;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.net.SocketTimeoutException;
@ -39,8 +37,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.ClusterStatus;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
@ -65,7 +63,6 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.exceptions.MergeRegionException;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.ipc.MasterCoprocessorRpcChannel;
import org.apache.hadoop.hbase.ipc.RegionServerCoprocessorRpcChannel;
@ -106,6 +103,9 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnaps
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest;
@ -267,8 +267,9 @@ public class HBaseAdmin implements Admin {
* otherwise.
* @throws ZooKeeperConnectionException
* @throws MasterNotRunningException
* @deprecated this has been deprecated without a replacement
*/
@Override
@Deprecated
public boolean isMasterRunning()
throws MasterNotRunningException, ZooKeeperConnectionException {
return connection.isMasterRunning();
@ -306,7 +307,14 @@ public class HBaseAdmin implements Admin {
*/
@Override
public HTableDescriptor[] listTables() throws IOException {
return this.connection.listTables();
return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
@Override
public HTableDescriptor[] call(int callTimeout) throws ServiceException {
GetTableDescriptorsRequest req =
RequestConverter.buildGetTableDescriptorsRequest((List<TableName>)null);
return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
}
});
}
/**
@ -339,10 +347,16 @@ public class HBaseAdmin implements Admin {
* List all of the names of userspace tables.
* @return String[] table names
* @throws IOException if a remote or network exception occurs
* @deprecated Use {@link Admin#listTableNames()} instead
*/
@Deprecated
public String[] getTableNames() throws IOException {
return this.connection.getTableNames();
TableName[] tableNames = listTableNames();
String result[] = new String[tableNames.length];
for (int i = 0; i < tableNames.length; i++) {
result[i] = tableNames[i].getNameAsString();
}
return result;
}
/**
@ -350,11 +364,12 @@ public class HBaseAdmin implements Admin {
* @param pattern The regular expression to match against
* @return String[] table names
* @throws IOException if a remote or network exception occurs
* @deprecated Use {@link Admin#listTables(Pattern)} instead.
*/
@Deprecated
public String[] getTableNames(Pattern pattern) throws IOException {
List<String> matched = new ArrayList<String>();
for (String name: this.connection.getTableNames()) {
for (String name: getTableNames()) {
if (pattern.matcher(name).matches()) {
matched.add(name);
}
@ -367,6 +382,7 @@ public class HBaseAdmin implements Admin {
* @param regex The regular expression to match against
* @return String[] table names
* @throws IOException if a remote or network exception occurs
* @deprecated Use {@link Admin#listTables(Pattern)} instead.
*/
@Deprecated
public String[] getTableNames(String regex) throws IOException {
@ -380,7 +396,14 @@ public class HBaseAdmin implements Admin {
*/
@Override
public TableName[] listTableNames() throws IOException {
return this.connection.listTableNames();
return executeCallable(new MasterCallable<TableName[]>(getConnection()) {
@Override
public TableName[] call(int callTimeout) throws ServiceException {
return ProtobufUtil.getTableNameArray(master.getTableNames(null,
GetTableNamesRequest.newBuilder().build())
.getTableNamesList());
}
});
}
/**
@ -393,7 +416,28 @@ public class HBaseAdmin implements Admin {
@Override
public HTableDescriptor getTableDescriptor(final TableName tableName)
throws TableNotFoundException, IOException {
return this.connection.getHTableDescriptor(tableName);
if (tableName == null) return null;
if (tableName.equals(TableName.META_TABLE_NAME)) {
return HTableDescriptor.META_TABLEDESC;
}
HTableDescriptor htd = executeCallable(new MasterCallable<HTableDescriptor>(getConnection()) {
@Override
public HTableDescriptor call(int callTimeout) throws ServiceException {
GetTableDescriptorsResponse htds;
GetTableDescriptorsRequest req =
RequestConverter.buildGetTableDescriptorsRequest(tableName);
htds = master.getTableDescriptors(null, req);
if (!htds.getTableSchemaList().isEmpty()) {
return HTableDescriptor.convert(htds.getTableSchemaList().get(0));
}
return null;
}
});
if (htd != null) {
return htd;
}
throw new TableNotFoundException(tableName.getNameAsString());
}
public HTableDescriptor getTableDescriptor(final byte[] tableName)
@ -1449,7 +1493,7 @@ public class HBaseAdmin implements Admin {
* {@inheritDoc}
*/
@Override
public void flush(final TableName tableName) throws IOException, InterruptedException {
public void flush(final TableName tableName) throws IOException {
checkTableExists(tableName);
if (isTableDisabled(tableName)) {
LOG.info("Table is disabled: " + tableName.getNameAsString());
@ -1463,7 +1507,7 @@ public class HBaseAdmin implements Admin {
* {@inheritDoc}
*/
@Override
public void flushRegion(final byte[] regionName) throws IOException, InterruptedException {
public void flushRegion(final byte[] regionName) throws IOException {
Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
if (regionServerPair == null) {
throw new IllegalArgumentException("Unknown regionname: " + Bytes.toStringBinary(regionName));
@ -1516,7 +1560,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public void compact(final TableName tableName)
throws IOException, InterruptedException {
throws IOException {
compact(tableName, null, false);
}
@ -1525,7 +1569,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public void compactRegion(final byte[] regionName)
throws IOException, InterruptedException {
throws IOException {
compactRegion(regionName, null, false);
}
@ -1535,7 +1579,7 @@ public class HBaseAdmin implements Admin {
*/
@Deprecated
public void compact(final String tableNameOrRegionName)
throws IOException, InterruptedException {
throws IOException {
compact(Bytes.toBytes(tableNameOrRegionName));
}
@ -1545,7 +1589,7 @@ public class HBaseAdmin implements Admin {
*/
@Deprecated
public void compact(final byte[] tableNameOrRegionName)
throws IOException, InterruptedException {
throws IOException {
try {
compactRegion(tableNameOrRegionName, null, false);
} catch (IllegalArgumentException e) {
@ -1558,7 +1602,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public void compact(final TableName tableName, final byte[] columnFamily)
throws IOException, InterruptedException {
throws IOException {
compact(tableName, columnFamily, false);
}
@ -1567,7 +1611,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public void compactRegion(final byte[] regionName, final byte[] columnFamily)
throws IOException, InterruptedException {
throws IOException {
compactRegion(regionName, columnFamily, false);
}
@ -1577,7 +1621,7 @@ public class HBaseAdmin implements Admin {
*/
@Deprecated
public void compact(String tableOrRegionName, String columnFamily)
throws IOException, InterruptedException {
throws IOException {
compact(Bytes.toBytes(tableOrRegionName), Bytes.toBytes(columnFamily));
}
@ -1587,7 +1631,7 @@ public class HBaseAdmin implements Admin {
*/
@Deprecated
public void compact(final byte[] tableNameOrRegionName, final byte[] columnFamily)
throws IOException, InterruptedException {
throws IOException {
try {
compactRegion(tableNameOrRegionName, columnFamily, false);
} catch (IllegalArgumentException e) {
@ -1612,7 +1656,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public void majorCompact(final TableName tableName)
throws IOException, InterruptedException {
throws IOException {
compact(tableName, null, true);
}
@ -1621,7 +1665,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public void majorCompactRegion(final byte[] regionName)
throws IOException, InterruptedException {
throws IOException {
compactRegion(regionName, null, true);
}
@ -1631,7 +1675,7 @@ public class HBaseAdmin implements Admin {
*/
@Deprecated
public void majorCompact(final String tableNameOrRegionName)
throws IOException, InterruptedException {
throws IOException {
majorCompact(Bytes.toBytes(tableNameOrRegionName));
}
@ -1641,7 +1685,7 @@ public class HBaseAdmin implements Admin {
*/
@Deprecated
public void majorCompact(final byte[] tableNameOrRegionName)
throws IOException, InterruptedException {
throws IOException {
try {
compactRegion(tableNameOrRegionName, null, true);
} catch (IllegalArgumentException e) {
@ -1655,7 +1699,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public void majorCompact(final TableName tableName, final byte[] columnFamily)
throws IOException, InterruptedException {
throws IOException {
compact(tableName, columnFamily, true);
}
@ -1664,7 +1708,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public void majorCompactRegion(final byte[] regionName, final byte[] columnFamily)
throws IOException, InterruptedException {
throws IOException {
compactRegion(regionName, columnFamily, true);
}
@ -1674,7 +1718,7 @@ public class HBaseAdmin implements Admin {
*/
@Deprecated
public void majorCompact(final String tableNameOrRegionName, final String columnFamily)
throws IOException, InterruptedException {
throws IOException {
majorCompact(Bytes.toBytes(tableNameOrRegionName), Bytes.toBytes(columnFamily));
}
@ -1684,7 +1728,7 @@ public class HBaseAdmin implements Admin {
*/
@Deprecated
public void majorCompact(final byte[] tableNameOrRegionName, final byte[] columnFamily)
throws IOException, InterruptedException {
throws IOException {
try {
compactRegion(tableNameOrRegionName, columnFamily, true);
} catch (IllegalArgumentException e) {
@ -1704,7 +1748,7 @@ public class HBaseAdmin implements Admin {
* @throws InterruptedException
*/
private void compact(final TableName tableName, final byte[] columnFamily,final boolean major)
throws IOException, InterruptedException {
throws IOException {
ZooKeeperWatcher zookeeper = null;
try {
checkTableExists(tableName);
@ -1747,7 +1791,7 @@ public class HBaseAdmin implements Admin {
* @throws InterruptedException
*/
private void compactRegion(final byte[] regionName, final byte[] columnFamily,final boolean major)
throws IOException, InterruptedException {
throws IOException {
Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
if (regionServerPair == null) {
throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
@ -1783,28 +1827,25 @@ public class HBaseAdmin implements Admin {
* <code> host187.example.com,60020,1289493121758</code>
* @throws UnknownRegionException Thrown if we can't find a region named
* <code>encodedRegionName</code>
* @throws ZooKeeperConnectionException
* @throws MasterNotRunningException
*/
@Override
public void move(final byte [] encodedRegionName, final byte [] destServerName)
throws HBaseIOException, MasterNotRunningException, ZooKeeperConnectionException {
MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
try {
MoveRegionRequest request =
RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
stub.moveRegion(null, request);
} catch (ServiceException se) {
IOException ioe = ProtobufUtil.getRemoteException(se);
if (ioe instanceof HBaseIOException) {
throw (HBaseIOException)ioe;
throws IOException {
executeCallable(new MasterCallable<Void>(getConnection()) {
@Override
public Void call(int callTimeout) throws ServiceException {
try {
MoveRegionRequest request =
RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
master.moveRegion(null, request);
} catch (DeserializationException de) {
LOG.error("Could not parse destination server name: " + de);
throw new ServiceException(new DoNotRetryIOException(de));
}
return null;
}
LOG.error("Unexpected exception: " + se + " from calling HMaster.moveRegion");
} catch (DeserializationException de) {
LOG.error("Could not parse destination server name: " + de);
} finally {
stub.close();
}
});
}
/**
@ -1873,14 +1914,13 @@ public class HBaseAdmin implements Admin {
@Override
public void offline(final byte [] regionName)
throws IOException {
MasterKeepAliveConnection master = connection.getKeepAliveMasterService();
try {
master.offlineRegion(null,RequestConverter.buildOfflineRegionRequest(regionName));
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
} finally {
master.close();
}
executeCallable(new MasterCallable<Void>(getConnection()) {
@Override
public Void call(int callTimeout) throws ServiceException {
master.offlineRegion(null,RequestConverter.buildOfflineRegionRequest(regionName));
return null;
}
});
}
/**
@ -1891,27 +1931,15 @@ public class HBaseAdmin implements Admin {
*/
@Override
public boolean setBalancerRunning(final boolean on, final boolean synchronous)
throws MasterNotRunningException, ZooKeeperConnectionException {
MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
try {
SetBalancerRunningRequest req =
RequestConverter.buildSetBalancerRunningRequest(on, synchronous);
return stub.setBalancerRunning(null, req).getPrevBalanceValue();
} catch (ServiceException se) {
IOException ioe = ProtobufUtil.getRemoteException(se);
if (ioe instanceof MasterNotRunningException) {
throw (MasterNotRunningException)ioe;
throws IOException {
return executeCallable(new MasterCallable<Boolean>(getConnection()) {
@Override
public Boolean call(int callTimeout) throws ServiceException {
SetBalancerRunningRequest req =
RequestConverter.buildSetBalancerRunningRequest(on, synchronous);
return master.setBalancerRunning(null, req).getPrevBalanceValue();
}
if (ioe instanceof ZooKeeperConnectionException) {
throw (ZooKeeperConnectionException)ioe;
}
// Throwing MasterNotRunningException even though not really valid in order to not
// break interface by adding additional exception type.
throw new MasterNotRunningException("Unexpected exception when calling balanceSwitch",se);
} finally {
stub.close();
}
});
}
/**
@ -1921,66 +1949,62 @@ public class HBaseAdmin implements Admin {
* @return True if balancer ran, false otherwise.
*/
@Override
public boolean balancer()
throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException {
MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
try {
return stub.balance(null, RequestConverter.buildBalanceRequest()).getBalancerRan();
} finally {
stub.close();
}
public boolean balancer() throws IOException {
return executeCallable(new MasterCallable<Boolean>(getConnection()) {
@Override
public Boolean call(int callTimeout) throws ServiceException {
return master.balance(null, RequestConverter.buildBalanceRequest()).getBalancerRan();
}
});
}
/**
* Enable/Disable the catalog janitor
* @param enable if true enables the catalog janitor
* @return the previous state
* @throws ServiceException
* @throws MasterNotRunningException
*/
@Override
public boolean enableCatalogJanitor(boolean enable)
throws ServiceException, MasterNotRunningException {
MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
try {
return stub.enableCatalogJanitor(null,
RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue();
} finally {
stub.close();
}
public boolean enableCatalogJanitor(final boolean enable)
throws IOException {
return executeCallable(new MasterCallable<Boolean>(getConnection()) {
@Override
public Boolean call(int callTimeout) throws ServiceException {
return master.enableCatalogJanitor(null,
RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue();
}
});
}
/**
* Ask for a scan of the catalog table
* @return the number of entries cleaned
* @throws ServiceException
* @throws MasterNotRunningException
*/
@Override
public int runCatalogScan() throws ServiceException, MasterNotRunningException {
MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
try {
return stub.runCatalogScan(null,
RequestConverter.buildCatalogScanRequest()).getScanResult();
} finally {
stub.close();
}
public int runCatalogScan() throws IOException {
return executeCallable(new MasterCallable<Integer>(getConnection()) {
@Override
public Integer call(int callTimeout) throws ServiceException {
return master.runCatalogScan(null,
RequestConverter.buildCatalogScanRequest()).getScanResult();
}
});
}
/**
* Query on the catalog janitor state (Enabled/Disabled?)
* @throws ServiceException
* @throws org.apache.hadoop.hbase.MasterNotRunningException
*/
@Override
public boolean isCatalogJanitorEnabled() throws ServiceException, MasterNotRunningException {
MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
try {
return stub.isCatalogJanitorEnabled(null,
RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue();
} finally {
stub.close();
}
public boolean isCatalogJanitorEnabled() throws IOException {
return executeCallable(new MasterCallable<Boolean>(getConnection()) {
@Override
public Boolean call(int callTimeout) throws ServiceException {
return master.isCatalogJanitorEnabled(null,
RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue();
}
});
}
/**
@ -1995,28 +2019,21 @@ public class HBaseAdmin implements Admin {
public void mergeRegions(final byte[] encodedNameOfRegionA,
final byte[] encodedNameOfRegionB, final boolean forcible)
throws IOException {
MasterKeepAliveConnection master = connection
.getKeepAliveMasterService();
try {
DispatchMergingRegionsRequest request = RequestConverter
.buildDispatchMergingRegionsRequest(encodedNameOfRegionA,
encodedNameOfRegionB, forcible);
master.dispatchMergingRegions(null, request);
} catch (ServiceException se) {
IOException ioe = ProtobufUtil.getRemoteException(se);
if (ioe instanceof UnknownRegionException) {
throw (UnknownRegionException) ioe;
executeCallable(new MasterCallable<Void>(getConnection()) {
@Override
public Void call(int callTimeout) throws ServiceException {
try {
DispatchMergingRegionsRequest request = RequestConverter
.buildDispatchMergingRegionsRequest(encodedNameOfRegionA,
encodedNameOfRegionB, forcible);
master.dispatchMergingRegions(null, request);
} catch (DeserializationException de) {
LOG.error("Could not parse destination server name: " + de);
}
return null;
}
if (ioe instanceof MergeRegionException) {
throw (MergeRegionException) ioe;
}
LOG.error("Unexpected exception: " + se
+ " from calling HMaster.dispatchMergingRegions");
} catch (DeserializationException de) {
LOG.error("Could not parse destination server name: " + de);
} finally {
master.close();
}
});
}
/**
@ -2024,7 +2041,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public void split(final TableName tableName)
throws IOException, InterruptedException {
throws IOException {
split(tableName, null);
}
@ -2033,7 +2050,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public void splitRegion(final byte[] regionName)
throws IOException, InterruptedException {
throws IOException {
splitRegion(regionName, null);
}
@ -2062,7 +2079,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public void split(final TableName tableName, final byte [] splitPoint)
throws IOException, InterruptedException {
throws IOException {
ZooKeeperWatcher zookeeper = null;
try {
checkTableExists(tableName);
@ -2097,7 +2114,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public void splitRegion(final byte[] regionName, final byte [] splitPoint)
throws IOException, InterruptedException {
throws IOException {
Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
if (regionServerPair == null) {
throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
@ -2114,7 +2131,7 @@ public class HBaseAdmin implements Admin {
*/
@Deprecated
public void split(final String tableNameOrRegionName,
final String splitPoint) throws IOException, InterruptedException {
final String splitPoint) throws IOException {
split(Bytes.toBytes(tableNameOrRegionName), Bytes.toBytes(splitPoint));
}
@ -2124,7 +2141,7 @@ public class HBaseAdmin implements Admin {
*/
@Deprecated
public void split(final byte[] tableNameOrRegionName,
final byte [] splitPoint) throws IOException, InterruptedException {
final byte [] splitPoint) throws IOException {
try {
splitRegion(tableNameOrRegionName, splitPoint);
} catch (IllegalArgumentException e) {
@ -2139,6 +2156,7 @@ public class HBaseAdmin implements Admin {
Bytes.compareTo(hri.getStartKey(), splitPoint) == 0) {
throw new IOException("should not give a splitkey which equals to startkey!");
}
// TODO: This is not executed via retries
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
ProtobufUtil.split(admin, hri, splitPoint);
}
@ -2568,9 +2586,16 @@ public class HBaseAdmin implements Admin {
* @throws IOException if a remote or network exception occurs
*/
@Override
public HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames)
public HTableDescriptor[] getTableDescriptorsByTableName(final List<TableName> tableNames)
throws IOException {
return this.connection.getHTableDescriptorsByTableName(tableNames);
return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
@Override
public HTableDescriptor[] call(int callTimeout) throws Exception {
GetTableDescriptorsRequest req =
RequestConverter.buildGetTableDescriptorsRequest(tableNames);
return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
}
});
}
/**
@ -2681,7 +2706,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public CompactionState getCompactionState(final TableName tableName)
throws IOException, InterruptedException {
throws IOException {
CompactionState state = CompactionState.NONE;
ZooKeeperWatcher zookeeper =
new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
@ -2751,7 +2776,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public CompactionState getCompactionStateForRegion(final byte[] regionName)
throws IOException, InterruptedException {
throws IOException {
try {
Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
if (regionServerPair == null) {
@ -3136,12 +3161,7 @@ public class HBaseAdmin implements Admin {
// The table does not exists, switch to clone.
if (!tableExists(tableName)) {
try {
cloneSnapshot(snapshotName, tableName);
} catch (InterruptedException e) {
throw new InterruptedIOException("Interrupted when restoring a nonexistent table: " +
e.getMessage());
}
cloneSnapshot(snapshotName, tableName);
return;
}
@ -3208,7 +3228,7 @@ public class HBaseAdmin implements Admin {
* @throws IllegalArgumentException if the specified table has not a valid name
*/
public void cloneSnapshot(final byte[] snapshotName, final byte[] tableName)
throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
throws IOException, TableExistsException, RestoreSnapshotException {
cloneSnapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName));
}
@ -3224,7 +3244,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public void cloneSnapshot(final byte[] snapshotName, final TableName tableName)
throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
throws IOException, TableExistsException, RestoreSnapshotException {
cloneSnapshot(Bytes.toString(snapshotName), tableName);
}
@ -3257,7 +3277,7 @@ public class HBaseAdmin implements Admin {
*/
@Override
public void cloneSnapshot(final String snapshotName, final TableName tableName)
throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
throws IOException, TableExistsException, RestoreSnapshotException {
if (tableExists(tableName)) {
throw new TableExistsException(tableName);
}
@ -3621,45 +3641,6 @@ public class HBaseAdmin implements Admin {
return QuotaRetriever.open(conf, filter);
}
/**
* Parent of {@link MasterCallable} and {@link MasterCallable}.
* Has common methods.
* @param <V>
*/
abstract static class MasterCallable<V> implements RetryingCallable<V>, Closeable {
protected HConnection connection;
protected MasterKeepAliveConnection master;
public MasterCallable(final HConnection connection) {
this.connection = connection;
}
@Override
public void prepare(boolean reload) throws IOException {
this.master = this.connection.getKeepAliveMasterService();
}
@Override
public void close() throws IOException {
// The above prepare could fail but this would still be called though masterAdmin is null
if (this.master != null) this.master.close();
}
@Override
public void throwable(Throwable t, boolean retrying) {
}
@Override
public String getExceptionMessageAdditionalDetail() {
return "";
}
@Override
public long sleep(long pause, int tries) {
return ConnectionUtils.getPauseTime(pause, tries);
}
}
private <V> V executeCallable(MasterCallable<V> callable) throws IOException {
RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller();
try {

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.ExecutorService;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
@ -252,13 +251,11 @@ public interface HConnection extends Connection {
/**
* List all the userspace tables. In other words, scan the hbase:meta table.
*
* If we wanted this to be really fast, we could implement a special
* catalog table that just contains table names and their descriptors.
* Right now, it only exists as part of the hbase:meta table's region info.
*
* @return - returns an array of HTableDescriptors
* @throws IOException if a remote or network exception occurs
* @deprecated Use {@link Admin#listTables()} instead.
*/
@Deprecated
HTableDescriptor[] listTables() throws IOException;
/**
@ -272,9 +269,16 @@ public interface HConnection extends Connection {
// successor function, returning TableName, listTableNames in later versions
// because Java polymorphism doesn't consider return value types
/**
* @deprecated Use {@link Admin#listTableNames()} instead.
*/
@Deprecated
String[] getTableNames() throws IOException;
/**
* @deprecated Use {@link Admin#listTables()} instead.
*/
@Deprecated
TableName[] listTableNames() throws IOException;
/**
@ -282,6 +286,7 @@ public interface HConnection extends Connection {
* @return table metadata
* @throws IOException if a remote or network exception occurs
*/
@Deprecated
HTableDescriptor getHTableDescriptor(TableName tableName)
throws IOException;
@ -551,7 +556,9 @@ public interface HConnection extends Connection {
* @param tableNames List of table names
* @return HTD[] table metadata
* @throws IOException if a remote or network exception occurs
* @deprecated Use {@link Admin#getTableDescriptor(TableName)} instead.
*/
@Deprecated
HTableDescriptor[] getHTableDescriptorsByTableName(List<TableName> tableNames) throws IOException;
@Deprecated

View File

@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
@ -68,6 +69,8 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
@ -564,8 +567,39 @@ public class HTable implements HTableInterface, RegionLocator {
*/
@Override
public HTableDescriptor getTableDescriptor() throws IOException {
return new UnmodifyableHTableDescriptor(
this.connection.getHTableDescriptor(this.tableName));
// TODO: This is the same as HBaseAdmin.getTableDescriptor(). Only keep one.
if (tableName == null) return null;
if (tableName.equals(TableName.META_TABLE_NAME)) {
return HTableDescriptor.META_TABLEDESC;
}
HTableDescriptor htd = executeMasterCallable(
new MasterCallable<HTableDescriptor>(getConnection()) {
@Override
public HTableDescriptor call(int callTimeout) throws ServiceException {
GetTableDescriptorsResponse htds;
GetTableDescriptorsRequest req =
RequestConverter.buildGetTableDescriptorsRequest(tableName);
htds = master.getTableDescriptors(null, req);
if (!htds.getTableSchemaList().isEmpty()) {
return HTableDescriptor.convert(htds.getTableSchemaList().get(0));
}
return null;
}
});
if (htd != null) {
return new UnmodifyableHTableDescriptor(htd);
}
throw new TableNotFoundException(tableName.getNameAsString());
}
private <V> V executeMasterCallable(MasterCallable<V> callable) throws IOException {
RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller();
try {
return caller.callWithRetries(callable, operationTimeout);
} finally {
callable.close();
}
}
/**

View File

@ -0,0 +1,60 @@
/**
* 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.Closeable;
import java.io.IOException;
/**
* A RetryingCallable for master operations.
* @param <V> return type
*/
abstract class MasterCallable<V> implements RetryingCallable<V>, Closeable {
protected HConnection connection;
protected MasterKeepAliveConnection master;
public MasterCallable(final HConnection connection) {
this.connection = connection;
}
@Override
public void prepare(boolean reload) throws IOException {
this.master = this.connection.getKeepAliveMasterService();
}
@Override
public void close() throws IOException {
// The above prepare could fail but this would still be called though masterAdmin is null
if (this.master != null) this.master.close();
}
@Override
public void throwable(Throwable t, boolean retrying) {
}
@Override
public String getExceptionMessageAdditionalDetail() {
return "";
}
@Override
public long sleep(long pause, int tries) {
return ConnectionUtils.getPauseTime(pause, tries);
}
}

View File

@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class NoServerForRegionException extends RegionException {
public class NoServerForRegionException extends DoNotRetryRegionException {
private static final long serialVersionUID = 1L << 11 - 1L;
/** default constructor */

View File

@ -18,7 +18,7 @@
*/
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.hbase.RegionException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class RowTooBigException extends RegionException {
public class RowTooBigException extends DoNotRetryRegionException {
public RowTooBigException(String message) {
super(message);

View File

@ -40,6 +40,8 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.replication.ReplicationException;
@ -467,7 +469,14 @@ public class ReplicationAdmin implements Closeable {
*/
public List<HashMap<String, String>> listReplicated() throws IOException {
List<HashMap<String, String>> replicationColFams = new ArrayList<HashMap<String, String>>();
HTableDescriptor[] tables = this.connection.listTables();
Admin admin = new HBaseAdmin(this.connection.getConfiguration());
HTableDescriptor[] tables;
try {
tables = admin.listTables();
} finally {
if (admin!= null) admin.close();
}
for (HTableDescriptor table : tables) {
HColumnDescriptor[] columns = table.getColumnFamilies();

View File

@ -17,16 +17,18 @@
*/
package org.apache.hadoop.hbase.exceptions;
import org.apache.hadoop.hbase.RegionException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
/**
* Thrown when something is wrong in trying to merge two regions.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class MergeRegionException extends RegionException {
public class MergeRegionException extends DoNotRetryRegionException {
private static final long serialVersionUID = 4970899110066124122L;

View File

@ -126,20 +126,15 @@ public class MasterFlushTableProcedureManager extends MasterProcedureManager {
// Each region server will get its own online regions for the table.
// We may still miss regions that need to be flushed.
List<Pair<HRegionInfo, ServerName>> regionsAndLocations;
try {
if (TableName.META_TABLE_NAME.equals(tableName)) {
regionsAndLocations = new MetaTableLocator().getMetaRegionsAndLocations(
master.getZooKeeper());
} else {
regionsAndLocations = MetaTableAccessor.getTableRegionsAndLocations(
master.getConnection(), tableName, false);
}
} catch (InterruptedException e1) {
String msg = "Failed to get regions for '" + desc.getInstance() + "'";
LOG.error(msg);
throw new IOException(msg, e1);
if (TableName.META_TABLE_NAME.equals(tableName)) {
regionsAndLocations = new MetaTableLocator().getMetaRegionsAndLocations(
master.getZooKeeper());
} else {
regionsAndLocations = MetaTableAccessor.getTableRegionsAndLocations(
master.getConnection(), tableName, false);
}
Set<String> regionServers = new HashSet<String>(regionsAndLocations.size());
for (Pair<HRegionInfo, ServerName> region : regionsAndLocations) {
if (region != null && region.getFirst() != null && region.getSecond() != null) {

View File

@ -91,7 +91,7 @@ public class HBaseFsckRepair {
* @throws KeeperException
*/
public static void fixUnassigned(Admin admin, HRegionInfo region)
throws IOException, KeeperException {
throws IOException, KeeperException, InterruptedException {
HRegionInfo actualRegion = new HRegionInfo(region);
// Force ZK node to OFFLINE so master assigns
@ -111,7 +111,7 @@ public class HBaseFsckRepair {
* in comparators that is addressed by HBASE-5563.
*/
private static void forceOfflineInZK(Admin admin, final HRegionInfo region)
throws ZooKeeperConnectionException, KeeperException, IOException {
throws ZooKeeperConnectionException, KeeperException, IOException, InterruptedException {
admin.assign(region.getRegionName());
}

View File

@ -277,12 +277,8 @@ public class TestFromClientSide {
TEST_UTIL.waitFor(6000, new Waiter.Predicate<IOException>() {
@Override
public boolean evaluate() throws IOException {
try {
return TEST_UTIL.getHBaseAdmin().getCompactionState(TABLENAME) ==
AdminProtos.GetRegionInfoResponse.CompactionState.NONE;
} catch (InterruptedException e) {
throw new IOException(e);
}
return TEST_UTIL.getHBaseAdmin().getCompactionState(TABLENAME) ==
AdminProtos.GetRegionInfoResponse.CompactionState.NONE;
}
});

View File

@ -18,8 +18,11 @@
package org.apache.hadoop.hbase.client;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.util.ArrayList;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.testclassification.ClientTests;
@ -32,10 +35,22 @@ import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.PleaseHoldException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.mortbay.log.Log;
import com.google.protobuf.RpcController;
@ -46,10 +61,10 @@ public class TestHBaseAdminNoCluster {
/**
* Verify that PleaseHoldException gets retried.
* HBASE-8764
* @throws IOException
* @throws ZooKeeperConnectionException
* @throws MasterNotRunningException
* @throws ServiceException
* @throws IOException
* @throws ZooKeeperConnectionException
* @throws MasterNotRunningException
* @throws ServiceException
*/
@Test
public void testMasterMonitorCallableRetries()
@ -88,4 +103,219 @@ public class TestHBaseAdminNoCluster {
if (connection != null) connection.close();
}
}
@Test
public void testMasterOperationsRetries() throws Exception {
// Admin.listTables()
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.listTables();
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
Mockito.verify(masterAdmin, Mockito.atLeast(count))
.getTableDescriptors((RpcController)Mockito.any(),
(GetTableDescriptorsRequest)Mockito.any());
}
});
// Admin.listTableNames()
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.listTableNames();
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
Mockito.verify(masterAdmin, Mockito.atLeast(count))
.getTableNames((RpcController)Mockito.any(),
(GetTableNamesRequest)Mockito.any());
}
});
// Admin.getTableDescriptor()
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.getTableDescriptor(TableName.valueOf("getTableDescriptor"));
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
Mockito.verify(masterAdmin, Mockito.atLeast(count))
.getTableDescriptors((RpcController)Mockito.any(),
(GetTableDescriptorsRequest)Mockito.any());
}
});
// Admin.getTableDescriptorsByTableName()
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.getTableDescriptorsByTableName(new ArrayList<TableName>());
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
Mockito.verify(masterAdmin, Mockito.atLeast(count))
.getTableDescriptors((RpcController)Mockito.any(),
(GetTableDescriptorsRequest)Mockito.any());
}
});
// Admin.move()
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.move(new byte[0], null);
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
Mockito.verify(masterAdmin, Mockito.atLeast(count))
.moveRegion((RpcController)Mockito.any(),
(MoveRegionRequest)Mockito.any());
}
});
// Admin.offline()
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.offline(new byte[0]);
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
Mockito.verify(masterAdmin, Mockito.atLeast(count))
.offlineRegion((RpcController)Mockito.any(),
(OfflineRegionRequest)Mockito.any());
}
});
// Admin.setBalancerRunning()
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.setBalancerRunning(true, true);
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
Mockito.verify(masterAdmin, Mockito.atLeast(count))
.setBalancerRunning((RpcController)Mockito.any(),
(SetBalancerRunningRequest)Mockito.any());
}
});
// Admin.balancer()
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.balancer();
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
Mockito.verify(masterAdmin, Mockito.atLeast(count))
.balance((RpcController)Mockito.any(),
(BalanceRequest)Mockito.any());
}
});
// Admin.enabledCatalogJanitor()
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.enableCatalogJanitor(true);
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
Mockito.verify(masterAdmin, Mockito.atLeast(count))
.enableCatalogJanitor((RpcController)Mockito.any(),
(EnableCatalogJanitorRequest)Mockito.any());
}
});
// Admin.runCatalogScan()
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.runCatalogScan();
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
Mockito.verify(masterAdmin, Mockito.atLeast(count))
.runCatalogScan((RpcController)Mockito.any(),
(RunCatalogScanRequest)Mockito.any());
}
});
// Admin.isCatalogJanitorEnabled()
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.isCatalogJanitorEnabled();
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
Mockito.verify(masterAdmin, Mockito.atLeast(count))
.isCatalogJanitorEnabled((RpcController)Mockito.any(),
(IsCatalogJanitorEnabledRequest)Mockito.any());
}
});
// Admin.mergeRegions()
testMasterOperationIsRetried(new MethodCaller() {
@Override
public void call(Admin admin) throws Exception {
admin.mergeRegions(new byte[0], new byte[0], true);
}
@Override
public void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception {
Mockito.verify(masterAdmin, Mockito.atLeast(count))
.dispatchMergingRegions((RpcController)Mockito.any(),
(DispatchMergingRegionsRequest)Mockito.any());
}
});
}
private static interface MethodCaller {
void call(Admin admin) throws Exception;
void verify(MasterKeepAliveConnection masterAdmin, int count) throws Exception;
}
private void testMasterOperationIsRetried(MethodCaller caller) throws Exception {
Configuration configuration = HBaseConfiguration.create();
// Set the pause and retry count way down.
configuration.setLong(HConstants.HBASE_CLIENT_PAUSE, 1);
final int count = 10;
configuration.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, count);
ClusterConnection connection = mock(ClusterConnection.class);
when(connection.getConfiguration()).thenReturn(configuration);
MasterKeepAliveConnection masterAdmin =
Mockito.mock(MasterKeepAliveConnection.class, new Answer() {
@Override
public Object answer(InvocationOnMock invocation) throws Throwable {
if (invocation.getMethod().getName().equals("close")) {
return null;
}
throw new MasterNotRunningException(); // all methods will throw an exception
}
});
Mockito.when(connection.getKeepAliveMasterService()).thenReturn(masterAdmin);
Admin admin = null;
try {
admin = new HBaseAdmin(connection);
try {
caller.call(admin); // invoke the HBaseAdmin method
fail();
} catch (RetriesExhaustedException e) {
Log.info("Expected fail", e);
}
// Assert we were called 'count' times.
caller.verify(masterAdmin, count);
} finally {
if (admin != null) {admin.close();}
}
}
}

View File

@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.TableNotFoundException;
@ -186,16 +185,6 @@ public class TestReplicasClient {
TestRegionServerNoMaster.stopMasterAndAssignMeta(HTU);
Configuration c = new Configuration(HTU.getConfiguration());
c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
Admin ha = new HBaseAdmin(c);
for (boolean masterRuns = true; masterRuns; ) {
Thread.sleep(100);
try {
masterRuns = false;
masterRuns = ha.isMasterRunning();
} catch (MasterNotRunningException ignored) {
}
}
ha.close();
LOG.info("Master has stopped");
}

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.util.StringUtils;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -64,6 +65,8 @@ public class TestMaster {
@BeforeClass
public static void beforeAllTests() throws Exception {
// we will retry operations when PleaseHoldException is thrown
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
// Start a cluster of two regionservers.
TEST_UTIL.startMiniCluster(2);
admin = TEST_UTIL.getHBaseAdmin();
@ -173,7 +176,7 @@ public class TestMaster {
admin.move(tableRegions.get(0).getEncodedNameAsBytes(), null);
fail("Region should not be moved since master is not initialized");
} catch (IOException ioe) {
assertTrue(ioe instanceof PleaseHoldException);
assertTrue(StringUtils.stringifyException(ioe).contains("PleaseHoldException"));
} finally {
master.initialized = true;
TEST_UTIL.deleteTable(tableName);

View File

@ -68,6 +68,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.util.StringUtils;
import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -293,8 +294,9 @@ public class TestRegionMergeTransactionOnCluster {
admin.mergeRegions(a.getEncodedNameAsBytes(), b.getEncodedNameAsBytes(), false);
fail("Offline regions should not be able to merge");
} catch (IOException ie) {
System.out.println(ie);
assertTrue("Exception should mention regions not online",
ie.getMessage().contains("regions not online")
StringUtils.stringifyException(ie).contains("regions not online")
&& ie instanceof MergeRegionException);
}
try {
@ -303,7 +305,7 @@ public class TestRegionMergeTransactionOnCluster {
fail("A region should not be able to merge with itself, even forcifully");
} catch (IOException ie) {
assertTrue("Exception should mention regions not online",
ie.getMessage().contains("region to itself")
StringUtils.stringifyException(ie).contains("region to itself")
&& ie instanceof MergeRegionException);
}
try {

View File

@ -40,7 +40,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
@ -227,33 +226,33 @@ public class TestSplitTransactionOnCluster {
byte[] cf = Bytes.toBytes("cf");
htd.addFamily(new HColumnDescriptor(cf));
admin.createTable(htd);
for (int i = 0; cluster.getRegions(tableName).size() == 0 && i < 100; i++) {
Thread.sleep(100);
}
assertEquals(1, cluster.getRegions(tableName).size());
HRegion region = cluster.getRegions(tableName).get(0);
Store store = region.getStore(cf);
int regionServerIndex = cluster.getServerWith(region.getRegionName());
HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
Table t = new HTable(conf, tableName);
// insert data
insertData(tableName, admin, t);
insertData(tableName, admin, t);
int fileNum = store.getStorefiles().size();
// 0, Compaction Request
store.triggerMajorCompaction();
CompactionContext cc = store.requestCompaction();
assertNotNull(cc);
// 1, A timeout split
// 1.1 close region
// 1, A timeout split
// 1.1 close region
assertEquals(2, region.close(false).get(cf).size());
// 1.2 rollback and Region initialize again
region.initialize();
// 2, Run Compaction cc
assertFalse(region.compact(cc, store));
assertTrue(fileNum > store.getStorefiles().size());
@ -264,7 +263,7 @@ public class TestSplitTransactionOnCluster {
st.execute(regionServer, regionServer);
assertEquals(2, cluster.getRegions(tableName).size());
}
public static class FailingSplitRegionObserver extends BaseRegionObserver {
static volatile CountDownLatch latch = new CountDownLatch(1);
@Override
@ -1059,7 +1058,7 @@ public class TestSplitTransactionOnCluster {
*/
private int ensureTableRegionNotOnSameServerAsMeta(final Admin admin,
final HRegionInfo hri)
throws HBaseIOException, MasterNotRunningException,
throws IOException, MasterNotRunningException,
ZooKeeperConnectionException, InterruptedException {
// Now make sure that the table region is not on same server as that hosting
// hbase:meta We don't want hbase:meta replay polluting our test when we later crash

View File

@ -686,8 +686,6 @@ public class ThriftServerRunner implements Runnable {
public void compact(ByteBuffer tableNameOrRegionName) throws IOError {
try{
getHBaseAdmin().compact(getBytes(tableNameOrRegionName));
} catch (InterruptedException e) {
throw new IOError(e.getMessage());
} catch (IOException e) {
LOG.warn(e.getMessage(), e);
throw new IOError(e.getMessage());
@ -698,9 +696,6 @@ public class ThriftServerRunner implements Runnable {
public void majorCompact(ByteBuffer tableNameOrRegionName) throws IOError {
try{
getHBaseAdmin().majorCompact(getBytes(tableNameOrRegionName));
} catch (InterruptedException e) {
LOG.warn(e.getMessage(), e);
throw new IOError(e.getMessage());
} catch (IOException e) {
LOG.warn(e.getMessage(), e);
throw new IOError(e.getMessage());