HBASE-5443 Convert the client protocol of HRegionInterface to PB

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1325937 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2012-04-13 20:28:21 +00:00
parent 740ed62e50
commit 4d04f85308
40 changed files with 9874 additions and 6992 deletions

View File

@ -31,18 +31,15 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.ipc.RemoteException;
/**
* Reads region and assignment information from <code>.META.</code>.
@ -252,19 +249,6 @@ public class MetaReader {
}
}
/**
* Reads the location of META from ROOT.
* @param metaServer connection to server hosting ROOT
* @return location of META in ROOT where location, or null if not available
* @throws IOException
* @deprecated Does not retry; use #getMetaRegionLocation(CatalogTracker)
*/
public static ServerName readMetaLocation(HRegionInterface metaServer)
throws IOException {
return readLocation(metaServer, CatalogTracker.ROOT_REGION_NAME,
CatalogTracker.META_REGION_NAME);
}
/**
* Gets the location of <code>.META.</code> region by reading content of
* <code>-ROOT-</code>.
@ -292,50 +276,6 @@ public class MetaReader {
return (pair == null || pair.getSecond() == null)? null: pair.getSecond();
}
// TODO: Remove when deprecated dependencies are removed.
private static ServerName readLocation(HRegionInterface metaServer,
byte [] catalogRegionName, byte [] regionName)
throws IOException {
Result r = null;
try {
r = metaServer.get(catalogRegionName,
new Get(regionName).
addColumn(HConstants.CATALOG_FAMILY,
HConstants.SERVER_QUALIFIER).
addColumn(HConstants.CATALOG_FAMILY,
HConstants.STARTCODE_QUALIFIER));
} catch (java.net.SocketTimeoutException e) {
// Treat this exception + message as unavailable catalog table. Catch it
// and fall through to return a null
} catch (java.net.SocketException e) {
// Treat this exception + message as unavailable catalog table. Catch it
// and fall through to return a null
} catch (RemoteException re) {
IOException ioe = re.unwrapRemoteException();
if (ioe instanceof NotServingRegionException) {
// Treat this NSRE as unavailable table. Catch and fall through to
// return null below
} else if (ioe.getMessage().contains("Server not running")) {
// Treat as unavailable table.
} else {
throw re;
}
} catch (IOException e) {
if (e.getCause() != null && e.getCause() instanceof IOException &&
e.getCause().getMessage() != null &&
e.getCause().getMessage().contains("Connection reset by peer")) {
// Treat this exception + message as unavailable catalog table. Catch it
// and fall through to return a null
} else {
throw e;
}
}
if (r == null || r.isEmpty()) {
return null;
}
return getServerNameFromCatalogResult(r);
}
/**
* Gets the region info and assignment for the specified region.
* @param catalogTracker
@ -654,35 +594,6 @@ public class MetaReader {
fullScan(catalogTracker, v);
}
/**
* Fully scan a given region, on a given server starting with given row.
* @param hRegionInterface region server
* @param visitor visitor
* @param regionName name of region
* @param startrow start row
* @throws IOException
* @deprecated Does not retry; use fullScan xxx instead.
x
*/
public static void fullScan(HRegionInterface hRegionInterface,
Visitor visitor, final byte[] regionName,
byte[] startrow) throws IOException {
if (hRegionInterface == null) return;
Scan scan = new Scan();
if (startrow != null) scan.setStartRow(startrow);
scan.addFamily(HConstants.CATALOG_FAMILY);
long scannerid = hRegionInterface.openScanner(regionName, scan);
try {
Result data;
while((data = hRegionInterface.next(scannerid)) != null) {
if (!data.isEmpty()) visitor.visit(data);
}
} finally {
hRegionInterface.close(scannerid);
}
return;
}
/**
* Performs a full scan of a catalog table.
* @param catalogTracker

View File

@ -307,6 +307,7 @@ public class ClientScanner extends AbstractClientScanner {
}
// Clear region
this.currentRegion = null;
callable = null;
continue;
}
long currentTime = System.currentTimeMillis();

View File

@ -19,6 +19,17 @@
*/
package org.apache.hadoop.hbase.client;
import java.io.Closeable;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.net.SocketTimeoutException;
import java.util.Arrays;
import java.util.LinkedList;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -45,6 +56,12 @@ import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.ipc.HMasterInterface;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ClientProtocol;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
@ -54,16 +71,7 @@ import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.StringUtils;
import org.apache.zookeeper.KeeperException;
import java.io.Closeable;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.net.SocketTimeoutException;
import java.util.Arrays;
import java.util.LinkedList;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.regex.Pattern;
import com.google.protobuf.ServiceException;
/**
* Provides an interface to manage HBase database table metadata + general
@ -497,23 +505,27 @@ public class HBaseAdmin implements Abortable, Closeable {
});
// Wait until all regions deleted
HRegionInterface server =
connection.getHRegionConnection(firstMetaServer.getHostname(), firstMetaServer.getPort());
ClientProtocol server =
connection.getClient(firstMetaServer.getHostname(), firstMetaServer.getPort());
for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
long scannerId = -1L;
try {
Scan scan = MetaReader.getScanForTableName(tableName);
scan.addColumn(HConstants.CATALOG_FAMILY,
HConstants.REGIONINFO_QUALIFIER);
scannerId = server.openScanner(
firstMetaServer.getRegionInfo().getRegionName(), scan);
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
ScanRequest request = RequestConverter.buildScanRequest(
firstMetaServer.getRegionInfo().getRegionName(), scan, 1, true);
Result[] values = null;
// Get a batch at a time.
Result values = server.next(scannerId);
try {
ScanResponse response = server.scan(null, request);
values = ResponseConverter.getResults(response);
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
// let us wait until .META. table is updated and
// HMaster removes the table from its HTableDescriptors
if (values == null) {
if (values == null || values.length == 0) {
boolean tableExists = false;
HTableDescriptor[] htds;
MasterKeepAliveConnection master = connection.getKeepAliveMaster();
@ -542,14 +554,6 @@ public class HBaseAdmin implements Abortable, Closeable {
throw ex;
}
}
} finally {
if (scannerId != -1L) {
try {
server.close(scannerId);
} catch (IOException ex) {
LOG.warn(ex);
}
}
}
try {
Thread.sleep(getPauseTime(tries));

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
import org.apache.hadoop.hbase.ipc.HMasterInterface;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.protobuf.ClientProtocol;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
/**
@ -220,6 +221,19 @@ public interface HConnection extends Abortable, Closeable {
public HRegionInterface getHRegionConnection(final String hostname, final int port)
throws IOException;
/**
* Establishes a connection to the region server at the specified address, and return
* a region client protocol.
*
* @param hostname RegionServer hostname
* @param port RegionServer port
* @return ClientProtocol proxy for RegionServer
* @throws IOException if a remote or network exception occurs
*
*/
public ClientProtocol getClient(final String hostname, final int port)
throws IOException;
/**
* Establishes a connection to the region server at the specified address.
* @param regionServer - the server to connect to

View File

@ -19,6 +19,33 @@
*/
package org.apache.hadoop.hbase.client;
import java.io.Closeable;
import java.io.IOException;
import java.lang.reflect.InvocationHandler;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.lang.reflect.Proxy;
import java.lang.reflect.UndeclaredThrowableException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CopyOnWriteArraySet;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -46,6 +73,13 @@ import org.apache.hadoop.hbase.ipc.ExecRPCInvoker;
import org.apache.hadoop.hbase.ipc.HBaseRPC;
import org.apache.hadoop.hbase.ipc.HMasterInterface;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.ipc.VersionedProtocol;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ClientProtocol;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
@ -61,32 +95,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.zookeeper.KeeperException;
import java.io.Closeable;
import java.io.IOException;
import java.lang.reflect.InvocationHandler;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.lang.reflect.Proxy;
import java.lang.reflect.UndeclaredThrowableException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CopyOnWriteArraySet;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import com.google.protobuf.ServiceException;
/**
* A non-instantiable class that manages {@link HConnection}s.
@ -146,6 +155,12 @@ public class HConnectionManager {
public static final int MAX_CACHED_HBASE_INSTANCES;
/** Parameter name for what client protocol to use. */
public static final String CLIENT_PROTOCOL_CLASS = "hbase.clientprotocol.class";
/** Default client protocol class name. */
public static final String DEFAULT_CLIENT_PROTOCOL_CLASS = ClientProtocol.class.getName();
private static final Log LOG = LogFactory.getLog(HConnectionManager.class);
static {
@ -493,6 +508,7 @@ public class HConnectionManager {
static class HConnectionImplementation implements HConnection, Closeable {
static final Log LOG = LogFactory.getLog(HConnectionImplementation.class);
private final Class<? extends HRegionInterface> serverInterfaceClass;
private final Class<? extends ClientProtocol> clientClass;
private final long pause;
private final int numRetries;
private final int maxRPCAttempts;
@ -521,8 +537,8 @@ public class HConnectionManager {
private final Configuration conf;
// Known region HServerAddress.toString() -> HRegionInterface
private final Map<String, HRegionInterface> servers =
new ConcurrentHashMap<String, HRegionInterface>();
private final ConcurrentHashMap<String, Map<String, VersionedProtocol>> servers =
new ConcurrentHashMap<String, Map<String, VersionedProtocol>>();
private final ConcurrentHashMap<String, String> connectionLock =
new ConcurrentHashMap<String, String>();
@ -570,6 +586,15 @@ public class HConnectionManager {
throw new UnsupportedOperationException(
"Unable to find region server interface " + serverClassName, e);
}
String clientClassName = conf.get(CLIENT_PROTOCOL_CLASS,
DEFAULT_CLIENT_PROTOCOL_CLASS);
try {
this.clientClass =
(Class<? extends ClientProtocol>) Class.forName(clientClassName);
} catch (ClassNotFoundException e) {
throw new UnsupportedOperationException(
"Unable to find client protocol " + clientClassName, e);
}
this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
this.numRetries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
@ -1328,19 +1353,29 @@ public class HConnectionManager {
return getHRegionConnection(hostname, port, false);
}
@Override
public ClientProtocol getClient(
final String hostname, final int port) throws IOException {
return (ClientProtocol)getProtocol(hostname, port,
clientClass, ClientProtocol.VERSION);
}
@Override
@Deprecated
public HRegionInterface getHRegionConnection(HServerAddress hsa,
boolean master)
throws IOException {
return getHRegionConnection(null, -1, hsa.getInetSocketAddress(), master);
String hostname = hsa.getInetSocketAddress().getHostName();
int port = hsa.getInetSocketAddress().getPort();
return getHRegionConnection(hostname, port, master);
}
@Override
public HRegionInterface getHRegionConnection(final String hostname,
final int port, final boolean master)
throws IOException {
return getHRegionConnection(hostname, port, null, master);
return (HRegionInterface)getProtocol(hostname, port,
serverInterfaceClass, HRegionInterface.VERSION);
}
/**
@ -1348,43 +1383,44 @@ public class HConnectionManager {
* can be but not both.
* @param hostname
* @param port
* @param isa
* @param master
* @param protocolClass
* @param version
* @return Proxy.
* @throws IOException
*/
HRegionInterface getHRegionConnection(final String hostname, final int port,
final InetSocketAddress isa, final boolean master)
throws IOException {
HRegionInterface server;
String rsName = null;
if (isa != null) {
rsName = Addressing.createHostAndPortStr(isa.getHostName(),
isa.getPort());
} else {
rsName = Addressing.createHostAndPortStr(hostname, port);
}
VersionedProtocol getProtocol(final String hostname,
final int port, final Class <? extends VersionedProtocol> protocolClass,
final long version) throws IOException {
String rsName = Addressing.createHostAndPortStr(hostname, port);
// See if we already have a connection (common case)
server = this.servers.get(rsName);
Map<String, VersionedProtocol> protocols = this.servers.get(rsName);
if (protocols == null) {
protocols = new HashMap<String, VersionedProtocol>();
Map<String, VersionedProtocol> existingProtocols =
this.servers.putIfAbsent(rsName, protocols);
if (existingProtocols != null) {
protocols = existingProtocols;
}
}
String protocol = protocolClass.getName();
VersionedProtocol server = protocols.get(protocol);
if (server == null) {
// create a unique lock for this RS (if necessary)
this.connectionLock.putIfAbsent(rsName, rsName);
// create a unique lock for this RS + protocol (if necessary)
String lockKey = protocol + "@" + rsName;
this.connectionLock.putIfAbsent(lockKey, lockKey);
// get the RS lock
synchronized (this.connectionLock.get(rsName)) {
synchronized (this.connectionLock.get(lockKey)) {
// do one more lookup in case we were stalled above
server = this.servers.get(rsName);
server = protocols.get(protocol);
if (server == null) {
try {
// Only create isa when we need to.
InetSocketAddress address = isa != null? isa:
new InetSocketAddress(hostname, port);
InetSocketAddress address = new InetSocketAddress(hostname, port);
// definitely a cache miss. establish an RPC for this RS
server = (HRegionInterface) HBaseRPC.waitForProxy(
serverInterfaceClass, HRegionInterface.VERSION,
address, this.conf,
server = HBaseRPC.waitForProxy(
protocolClass, version, address, this.conf,
this.maxRPCAttempts, this.rpcTimeout, this.rpcTimeout);
this.servers.put(Addressing.createHostAndPortStr(
address.getHostName(), address.getPort()), server);
protocols.put(protocol, server);
} catch (RemoteException e) {
LOG.warn("RemoteException connecting to RS", e);
// Throw what the RemoteException was carrying.
@ -1679,11 +1715,42 @@ public class HConnectionManager {
ServerCallable<MultiResponse> callable =
new ServerCallable<MultiResponse>(connection, tableName, null) {
public MultiResponse call() throws IOException {
return server.multi(multi);
try {
MultiResponse response = new MultiResponse();
for (Map.Entry<byte[], List<Action<R>>> e: multi.actions.entrySet()) {
byte[] regionName = e.getKey();
int rowMutations = 0;
List<Action<R>> actions = e.getValue();
for (Action<R> action: actions) {
Row row = action.getAction();
if (row instanceof RowMutations) {
MultiRequest request =
RequestConverter.buildMultiRequest(regionName, (RowMutations)row);
server.multi(null, request);
response.add(regionName, action.getOriginalIndex(), new Result());
rowMutations++;
}
}
if (actions.size() > rowMutations) {
MultiRequest request =
RequestConverter.buildMultiRequest(regionName, actions);
ClientProtos.MultiResponse
proto = server.multi(null, request);
List<Object> results = ResponseConverter.getResults(proto);
for (int i = 0, n = results.size(); i < n; i++) {
int originalIndex = actions.get(i).getOriginalIndex();
response.add(regionName, originalIndex, results.get(i));
}
}
}
return response;
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
@Override
public void connect(boolean reload) throws IOException {
server = connection.getHRegionConnection(
server = connection.getClient(
loc.getHostname(), loc.getPort());
}
};
@ -1885,7 +1952,7 @@ public class HConnectionManager {
}
}
} catch (ExecutionException e) {
LOG.warn("Failed all from " + loc, e);
LOG.debug("Failed all from " + loc, e);
}
}
@ -2077,8 +2144,10 @@ public class HConnectionManager {
delayedClosing.stop("Closing connection");
if (stopProxy) {
closeMaster();
for (HRegionInterface i : servers.values()) {
HBaseRPC.stopProxy(i);
for (Map<String, VersionedProtocol> i : servers.values()) {
for (VersionedProtocol server: i.values()) {
HBaseRPC.stopProxy(server);
}
}
}
closeZooKeeperWatcher();

View File

@ -53,13 +53,27 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
import org.apache.hadoop.hbase.ipc.ExecRPCInvoker;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition.CompareType;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.LockRowRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.LockRowResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
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.UnlockRowRequest;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Writables;
import com.google.protobuf.ServiceException;
/**
* <p>Used to communicate with a single HBase table.
*
@ -648,8 +662,15 @@ public class HTable implements HTableInterface {
throws IOException {
return new ServerCallable<Result>(connection, tableName, row, operationTimeout) {
public Result call() throws IOException {
return server.getClosestRowBefore(location.getRegionInfo().getRegionName(),
row, family);
try {
GetRequest request = RequestConverter.buildGetRequest(
location.getRegionInfo().getRegionName(), row, family, true);
GetResponse response = server.get(null, request);
if (!response.hasResult()) return null;
return ProtobufUtil.toResult(response.getResult());
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
}.withRetries();
}
@ -694,7 +715,14 @@ public class HTable implements HTableInterface {
public Result get(final Get get) throws IOException {
return new ServerCallable<Result>(connection, tableName, get.getRow(), operationTimeout) {
public Result call() throws IOException {
return server.get(location.getRegionInfo().getRegionName(), get);
try {
GetRequest request = RequestConverter.buildGetRequest(
location.getRegionInfo().getRegionName(), get);
GetResponse response = server.get(null, request);
return ProtobufUtil.toResult(response.getResult());
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
}.withRetries();
}
@ -746,13 +774,18 @@ public class HTable implements HTableInterface {
@Override
public void delete(final Delete delete)
throws IOException {
new ServerCallable<Void>(connection, tableName, delete.getRow(),
operationTimeout) {
public Void call() throws IOException {
server.delete(location.getRegionInfo().getRegionName(), delete);
return null;
}
}.withRetries();
new ServerCallable<Boolean>(connection, tableName, delete.getRow(), operationTimeout) {
public Boolean call() throws IOException {
try {
MutateRequest request = RequestConverter.buildMutateRequest(
location.getRegionInfo().getRegionName(), delete);
MutateResponse response = server.mutate(null, request);
return Boolean.valueOf(response.getProcessed());
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
}.withRetries();
}
/**
@ -821,7 +854,13 @@ public class HTable implements HTableInterface {
new ServerCallable<Void>(connection, tableName, rm.getRow(),
operationTimeout) {
public Void call() throws IOException {
server.mutateRow(location.getRegionInfo().getRegionName(), rm);
try {
MultiRequest request = RequestConverter.buildMultiRequest(
location.getRegionInfo().getRegionName(), rm);
server.multi(null, request);
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
return null;
}
}.withRetries();
@ -838,8 +877,15 @@ public class HTable implements HTableInterface {
}
return new ServerCallable<Result>(connection, tableName, append.getRow(), operationTimeout) {
public Result call() throws IOException {
return server.append(
try {
MutateRequest request = RequestConverter.buildMutateRequest(
location.getRegionInfo().getRegionName(), append);
MutateResponse response = server.mutate(null, request);
if (!response.hasResult()) return null;
return ProtobufUtil.toResult(response.getResult());
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
}.withRetries();
}
@ -855,8 +901,14 @@ public class HTable implements HTableInterface {
}
return new ServerCallable<Result>(connection, tableName, increment.getRow(), operationTimeout) {
public Result call() throws IOException {
return server.increment(
try {
MutateRequest request = RequestConverter.buildMutateRequest(
location.getRegionInfo().getRegionName(), increment);
MutateResponse response = server.mutate(null, request);
return ProtobufUtil.toResult(response.getResult());
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
}.withRetries();
}
@ -890,9 +942,16 @@ public class HTable implements HTableInterface {
}
return new ServerCallable<Long>(connection, tableName, row, operationTimeout) {
public Long call() throws IOException {
return server.incrementColumnValue(
try {
MutateRequest request = RequestConverter.buildMutateRequest(
location.getRegionInfo().getRegionName(), row, family,
qualifier, amount, writeToWAL);
MutateResponse response = server.mutate(null, request);
Result result = ProtobufUtil.toResult(response.getResult());
return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier)));
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
}.withRetries();
}
@ -907,8 +966,15 @@ public class HTable implements HTableInterface {
throws IOException {
return new ServerCallable<Boolean>(connection, tableName, row, operationTimeout) {
public Boolean call() throws IOException {
return server.checkAndPut(location.getRegionInfo().getRegionName(),
row, family, qualifier, value, put) ? Boolean.TRUE : Boolean.FALSE;
try {
MutateRequest request = RequestConverter.buildMutateRequest(
location.getRegionInfo().getRegionName(), row, family, qualifier,
new BinaryComparator(value), CompareType.EQUAL, put);
MutateResponse response = server.mutate(null, request);
return Boolean.valueOf(response.getProcessed());
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
}.withRetries();
}
@ -924,10 +990,15 @@ public class HTable implements HTableInterface {
throws IOException {
return new ServerCallable<Boolean>(connection, tableName, row, operationTimeout) {
public Boolean call() throws IOException {
return server.checkAndDelete(
location.getRegionInfo().getRegionName(),
row, family, qualifier, value, delete)
? Boolean.TRUE : Boolean.FALSE;
try {
MutateRequest request = RequestConverter.buildMutateRequest(
location.getRegionInfo().getRegionName(), row, family, qualifier,
new BinaryComparator(value), CompareType.EQUAL, delete);
MutateResponse response = server.mutate(null, request);
return Boolean.valueOf(response.getProcessed());
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
}.withRetries();
}
@ -939,8 +1010,14 @@ public class HTable implements HTableInterface {
public boolean exists(final Get get) throws IOException {
return new ServerCallable<Boolean>(connection, tableName, get.getRow(), operationTimeout) {
public Boolean call() throws IOException {
return server.
exists(location.getRegionInfo().getRegionName(), get);
try {
GetRequest request = RequestConverter.buildGetRequest(
location.getRegionInfo().getRegionName(), get, true);
GetResponse response = server.get(null, request);
return response.getExists();
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
}.withRetries();
}
@ -1026,9 +1103,14 @@ public class HTable implements HTableInterface {
throws IOException {
return new ServerCallable<RowLock>(connection, tableName, row, operationTimeout) {
public RowLock call() throws IOException {
long lockId =
server.lockRow(location.getRegionInfo().getRegionName(), row);
return new RowLock(row,lockId);
try {
LockRowRequest request = RequestConverter.buildLockRowRequest(
location.getRegionInfo().getRegionName(), row);
LockRowResponse response = server.lockRow(null, request);
return new RowLock(row, response.getLockId());
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
}.withRetries();
}
@ -1039,14 +1121,18 @@ public class HTable implements HTableInterface {
@Override
public void unlockRow(final RowLock rl)
throws IOException {
new ServerCallable<Void>(connection, tableName, rl.getRow(),
operationTimeout) {
public Void call() throws IOException {
server.unlockRow(location.getRegionInfo().getRegionName(), rl
.getLockId());
return null;
}
}.withRetries();
new ServerCallable<Boolean>(connection, tableName, rl.getRow(), operationTimeout) {
public Boolean call() throws IOException {
try {
UnlockRowRequest request = RequestConverter.buildUnlockRowRequest(
location.getRegionInfo().getRegionName(), rl.getLockId());
server.unlockRow(null, request);
return Boolean.TRUE;
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
}.withRetries();
}
/**

View File

@ -25,15 +25,22 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.DNS;
import com.google.protobuf.ServiceException;
/**
* Retries scanner operations such as create, next, etc.
* Used by {@link ResultScanner}s made by {@link HTable}.
@ -107,41 +114,58 @@ public class ScannerCallable extends ServerCallable<Result[]> {
* @see java.util.concurrent.Callable#call()
*/
public Result [] call() throws IOException {
if (scannerId != -1L && closed) {
close();
} else if (scannerId == -1L && !closed) {
this.scannerId = openScanner();
} else {
Result [] rrs = null;
try {
incRPCcallsMetrics();
rrs = server.next(scannerId, caching);
updateResultsMetrics(rrs);
} catch (IOException e) {
IOException ioe = null;
if (e instanceof RemoteException) {
ioe = RemoteExceptionHandler.decodeRemoteException((RemoteException)e);
}
if (ioe == null) throw new IOException(e);
if (ioe instanceof NotServingRegionException) {
// Throw a DNRE so that we break out of cycle of calling NSRE
// when what we need is to open scanner against new location.
// Attach NSRE to signal client that it needs to resetup scanner.
if (this.scanMetrics != null) {
this.scanMetrics.countOfNSRE.inc();
}
throw new DoNotRetryIOException("Reset scanner", ioe);
} else if (ioe instanceof RegionServerStoppedException) {
// Throw a DNRE so that we break out of cycle of calling RSSE
// when what we need is to open scanner against new location.
// Attach RSSE to signal client that it needs to resetup scanner.
throw new DoNotRetryIOException("Reset scanner", ioe);
} else {
// The outer layers will retry
throw ioe;
}
if (closed) {
if (scannerId != -1) {
close();
}
} else {
if (scannerId == -1L) {
this.scannerId = openScanner();
} else {
Result [] rrs = null;
try {
incRPCcallsMetrics();
ScanRequest request =
RequestConverter.buildScanRequest(scannerId, caching, false);
try {
ScanResponse response = server.scan(null, request);
rrs = ResponseConverter.getResults(response);
if (response.hasMoreResults()
&& !response.getMoreResults()) {
scannerId = -1L;
closed = true;
return null;
}
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
updateResultsMetrics(rrs);
} catch (IOException e) {
IOException ioe = null;
if (e instanceof RemoteException) {
ioe = RemoteExceptionHandler.decodeRemoteException((RemoteException)e);
}
if (ioe == null) throw new IOException(e);
if (ioe instanceof NotServingRegionException) {
// Throw a DNRE so that we break out of cycle of calling NSRE
// when what we need is to open scanner against new location.
// Attach NSRE to signal client that it needs to resetup scanner.
if (this.scanMetrics != null) {
this.scanMetrics.countOfNSRE.inc();
}
throw new DoNotRetryIOException("Reset scanner", ioe);
} else if (ioe instanceof RegionServerStoppedException) {
// Throw a DNRE so that we break out of cycle of calling RSSE
// when what we need is to open scanner against new location.
// Attach RSSE to signal client that it needs to resetup scanner.
throw new DoNotRetryIOException("Reset scanner", ioe);
} else {
// The outer layers will retry
throw ioe;
}
}
return rrs;
}
return rrs;
}
return null;
}
@ -161,10 +185,12 @@ public class ScannerCallable extends ServerCallable<Result[]> {
return;
}
for (Result rr : rrs) {
this.scanMetrics.countOfBytesInResults.inc(rr.getBytes().getLength());
if (isRegionServerRemote) {
this.scanMetrics.countOfBytesInRemoteResults.inc(
rr.getBytes().getLength());
if (rr.getBytes() != null) {
this.scanMetrics.countOfBytesInResults.inc(rr.getBytes().getLength());
if (isRegionServerRemote) {
this.scanMetrics.countOfBytesInRemoteResults.inc(
rr.getBytes().getLength());
}
}
}
}
@ -175,7 +201,13 @@ public class ScannerCallable extends ServerCallable<Result[]> {
}
try {
incRPCcallsMetrics();
this.server.close(this.scannerId);
ScanRequest request =
RequestConverter.buildScanRequest(this.scannerId, 0, true);
try {
server.scan(null, request);
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
} catch (IOException e) {
LOG.warn("Ignore, probably already closed", e);
}
@ -184,8 +216,16 @@ public class ScannerCallable extends ServerCallable<Result[]> {
protected long openScanner() throws IOException {
incRPCcallsMetrics();
return this.server.openScanner(this.location.getRegionInfo().getRegionName(),
this.scan);
ScanRequest request =
RequestConverter.buildScanRequest(
this.location.getRegionInfo().getRegionName(),
this.scan, 0, false);
try {
ScanResponse response = server.scan(null, request);
return response.getScannerId();
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}
protected Scan getScan() {

View File

@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ipc.HBaseRPC;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.protobuf.ClientProtocol;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.ipc.RemoteException;
@ -57,7 +57,7 @@ public abstract class ServerCallable<T> implements Callable<T> {
protected final byte [] tableName;
protected final byte [] row;
protected HRegionLocation location;
protected HRegionInterface server;
protected ClientProtocol server;
protected int callTimeout;
protected long startTime, endTime;
@ -84,8 +84,8 @@ public abstract class ServerCallable<T> implements Callable<T> {
*/
public void connect(final boolean reload) throws IOException {
this.location = connection.getRegionLocation(tableName, row, reload);
this.server = connection.getHRegionConnection(location.getHostname(),
location.getPort());
this.server = connection.getClient(location.getHostname(),
location.getPort());
}
/** @return the server name
@ -224,7 +224,7 @@ public abstract class ServerCallable<T> implements Callable<T> {
}
}
private static Throwable translateException(Throwable t) throws IOException {
protected static Throwable translateException(Throwable t) throws IOException {
if (t instanceof UndeclaredThrowableException) {
t = t.getCause();
}

View File

@ -19,13 +19,10 @@
*/
package org.apache.hadoop.hbase.filter;
import java.nio.ByteBuffer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.util.Bytes;
import java.nio.ByteBuffer;
import java.util.HashMap;
import org.apache.hadoop.hbase.filter.*;
/**
* ParseConstants holds a bunch of constants related to parsing Filter Strings

View File

@ -100,6 +100,7 @@ import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableUtils;
import com.google.protobuf.Message;
import com.google.protobuf.RpcController;
/**
* This is a customized version of the polymorphic hadoop
@ -268,6 +269,8 @@ public class HbaseObjectWritable implements Writable, WritableWithSize, Configur
GENERIC_ARRAY_CODE = code++;
addToMap(Array.class, GENERIC_ARRAY_CODE);
addToMap(RpcController.class, code++);
// make sure that this is the last statement in this static block
NEXT_CLASS_CODE = code;
}
@ -357,7 +360,7 @@ public class HbaseObjectWritable implements Writable, WritableWithSize, Configur
}
}
static Integer getClassCode(final Class<?> c)
public static Integer getClassCode(final Class<?> c)
throws IOException {
Integer code = CLASS_TO_CODE.get(c);
if (code == null ) {
@ -726,7 +729,7 @@ public class HbaseObjectWritable implements Writable, WritableWithSize, Configur
* @return the instantiated Message instance
* @throws IOException if an IO problem occurs
*/
private static Message tryInstantiateProtobuf(
public static Message tryInstantiateProtobuf(
Class<?> protoClass,
DataInput dataIn) throws IOException {

View File

@ -109,6 +109,14 @@ public class TimeRange implements Writable {
return maxStamp;
}
/**
* Check if it is for all time
* @return true if it is for all time
*/
public boolean isAllTime() {
return allTime;
}
/**
* Check if the specified timestamp is within this TimeRange.
* <p>

View File

@ -19,18 +19,23 @@
*/
package org.apache.hadoop.hbase.ipc;
import java.lang.reflect.InvocationHandler;
import java.lang.reflect.Method;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.*;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.ServerCallable;
import org.apache.hadoop.hbase.client.coprocessor.Exec;
import org.apache.hadoop.hbase.client.coprocessor.ExecResult;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorResponse;
import org.apache.hadoop.hbase.util.Bytes;
import java.lang.reflect.InvocationHandler;
import java.lang.reflect.Method;
/**
* Backs a {@link CoprocessorProtocol} subclass proxy and forwards method
* invocations for server execution. Note that internally this will issue a
@ -74,8 +79,13 @@ public class ExecRPCInvoker implements InvocationHandler {
ServerCallable<ExecResult> callable =
new ServerCallable<ExecResult>(connection, table, row) {
public ExecResult call() throws Exception {
return server.execCoprocessor(location.getRegionInfo().getRegionName(),
exec);
byte[] regionName = location.getRegionInfo().getRegionName();
ExecCoprocessorRequest request =
RequestConverter.buildExecCoprocessorRequest(regionName, exec);
ExecCoprocessorResponse response =
server.execCoprocessor(null, request);
Object value = ProtobufUtil.toObject(response.getValue());
return new ExecResult(regionName, value);
}
};
ExecResult result = callable.withRetries();

View File

@ -19,18 +19,22 @@
*/
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.io.HbaseObjectWritable;
import org.apache.hadoop.io.VersionMismatchException;
import org.apache.hadoop.io.VersionedWritable;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.lang.reflect.Field;
import java.lang.reflect.Method;
import java.util.HashMap;
import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.io.HbaseObjectWritable;
import org.apache.hadoop.hbase.protobuf.ClientProtocol;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.io.VersionMismatchException;
import org.apache.hadoop.io.VersionedWritable;
/** A method invocation, including the method name and its parameters.*/
@InterfaceAudience.Private
@ -43,6 +47,17 @@ public class Invocation extends VersionedWritable implements Configurable {
private long clientVersion;
private int clientMethodsHash;
// For generated protocol classes which don't have VERSION field,
// such as protobuf interfaces.
private static final Map<Class<?>, Long>
PROTOCOL_VERSION = new HashMap<Class<?>, Long>();
static {
PROTOCOL_VERSION.put(ClientService.BlockingInterface.class,
Long.valueOf(ClientProtocol.VERSION));
}
private static byte RPC_VERSION = 1;
public Invocation() {}
@ -51,22 +66,28 @@ public class Invocation extends VersionedWritable implements Configurable {
this.methodName = method.getName();
this.parameterClasses = method.getParameterTypes();
this.parameters = parameters;
if (method.getDeclaringClass().equals(VersionedProtocol.class)) {
Class<?> declaringClass = method.getDeclaringClass();
if (declaringClass.equals(VersionedProtocol.class)) {
//VersionedProtocol is exempted from version check.
clientVersion = 0;
clientMethodsHash = 0;
} else {
try {
Field versionField = method.getDeclaringClass().getField("VERSION");
versionField.setAccessible(true);
this.clientVersion = versionField.getLong(method.getDeclaringClass());
Long version = PROTOCOL_VERSION.get(declaringClass);
if (version != null) {
this.clientVersion = version.longValue();
} else {
Field versionField = declaringClass.getField("VERSION");
versionField.setAccessible(true);
this.clientVersion = versionField.getLong(declaringClass);
}
} catch (NoSuchFieldException ex) {
throw new RuntimeException("The " + method.getDeclaringClass(), ex);
throw new RuntimeException("The " + declaringClass, ex);
} catch (IllegalAccessException ex) {
throw new RuntimeException(ex);
}
this.clientMethodsHash = ProtocolSignature.getFingerprint(method
.getDeclaringClass().getMethods());
this.clientMethodsHash = ProtocolSignature.getFingerprint(
declaringClass.getMethods());
}
}

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.Operation;
import org.apache.hadoop.hbase.io.HbaseObjectWritable;
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Objects;
@ -52,6 +53,8 @@ import org.apache.hadoop.conf.*;
import org.codehaus.jackson.map.ObjectMapper;
import com.google.protobuf.ServiceException;
/** An RpcEngine implementation for Writable data. */
@InterfaceAudience.Private
class WritableRpcEngine implements RpcEngine {
@ -407,6 +410,9 @@ class WritableRpcEngine implements RpcEngine {
if (target instanceof IOException) {
throw (IOException)target;
}
if (target instanceof ServiceException) {
throw ProtobufUtil.getRemoteException((ServiceException)target);
}
IOException ioe = new IOException(target.toString());
ioe.setStackTrace(target.getStackTrace());
throw ioe;

View File

@ -68,10 +68,13 @@ import org.apache.hadoop.hbase.io.Reference.Range;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
@ -486,7 +489,11 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
LOG.debug("Going to connect to server " + location + " for row "
+ Bytes.toStringBinary(row));
byte[] regionName = location.getRegionInfo().getRegionName();
return server.bulkLoadHFiles(famPaths, regionName);
BulkLoadHFileRequest request =
RequestConverter.buildBulkLoadHFileRequest(famPaths, regionName);
BulkLoadHFileResponse response =
server.bulkLoadHFile(null, request);
return response.getLoaded();
}
};

View File

@ -0,0 +1,37 @@
/**
* 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.protobuf;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.ipc.VersionedProtocol;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.security.TokenInfo;
import org.apache.hadoop.security.KerberosInfo;
/**
* Protocol that a HBase client uses to communicate with a region server.
*/
@KerberosInfo(
serverPrincipal = "hbase.regionserver.kerberos.principal")
@TokenInfo("HBASE_AUTH_TOKEN")
@InterfaceAudience.Private
public interface AdminProtocol extends
AdminService.BlockingInterface, VersionedProtocol {
public static final long VERSION = 1L;
}

View File

@ -0,0 +1,39 @@
/**
* 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.protobuf;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.ipc.VersionedProtocol;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
import org.apache.hadoop.hbase.security.TokenInfo;
import org.apache.hadoop.security.KerberosInfo;
/**
* Protocol that a HBase client uses to communicate with a region server.
*/
@KerberosInfo(
serverPrincipal = "hbase.regionserver.kerberos.principal")
@TokenInfo("HBASE_AUTH_TOKEN")
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface ClientProtocol extends
ClientService.BlockingInterface, VersionedProtocol {
public static final long VERSION = 1L;
}

View File

@ -17,12 +17,89 @@
*/
package org.apache.hadoop.hbase.protobuf;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInput;
import java.io.DataInputStream;
import java.io.DataOutput;
import java.io.DataOutputStream;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.Method;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowLock;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.coprocessor.Exec;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.HbaseObjectWritable;
import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UUID;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry.WALEdit.FamilyScope;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry.WALKey;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.ByteString;
import com.google.protobuf.ServiceException;
/**
* Protobufs utility.
*/
public class ProtobufUtil {
public final class ProtobufUtil {
private ProtobufUtil() {
}
/**
* Primitive type to class mapping.
*/
private final static Map<String, Class<?>>
PRIMITIVES = new HashMap<String, Class<?>>();
static {
PRIMITIVES.put(Boolean.TYPE.getName(), Boolean.TYPE);
PRIMITIVES.put(Byte.TYPE.getName(), Byte.TYPE);
PRIMITIVES.put(Character.TYPE.getName(), Character.TYPE);
PRIMITIVES.put(Short.TYPE.getName(), Short.TYPE);
PRIMITIVES.put(Integer.TYPE.getName(), Integer.TYPE);
PRIMITIVES.put(Long.TYPE.getName(), Long.TYPE);
PRIMITIVES.put(Float.TYPE.getName(), Float.TYPE);
PRIMITIVES.put(Double.TYPE.getName(), Double.TYPE);
PRIMITIVES.put(Void.TYPE.getName(), Void.TYPE);
}
/**
* Magic we put ahead of a serialized protobuf message.
* For example, all znode content is protobuf messages with the below magic
@ -56,4 +133,592 @@ public class ProtobufUtil {
public static int lengthOfPBMagic() {
return PB_MAGIC.length;
}
/**
* Return the IOException thrown by the remote server wrapped in
* ServiceException as cause.
*
* @param se ServiceException that wraps IO exception thrown by the server
* @return Exception wrapped in ServiceException or
* a new IOException that wraps the unexpected ServiceException.
*/
public static IOException getRemoteException(ServiceException se) {
Throwable e = se.getCause();
if (e == null) {
return new IOException(se);
}
return e instanceof IOException ? (IOException) e : new IOException(se);
}
/**
* Convert a protocol buffer Exec to a client Exec
*
* @param proto the protocol buffer Exec to convert
* @return the converted client Exec
*/
@SuppressWarnings("unchecked")
public static Exec toExec(
final ClientProtos.Exec proto) throws IOException {
byte[] row = proto.getRow().toByteArray();
String protocolName = proto.getProtocolName();
String methodName = proto.getMethodName();
List<Object> parameters = new ArrayList<Object>();
Class<? extends CoprocessorProtocol> protocol = null;
Method method = null;
try {
List<Class<?>> types = new ArrayList<Class<?>>();
for (NameBytesPair parameter: proto.getParameterList()) {
String type = parameter.getName();
Class<?> declaredClass = PRIMITIVES.get(type);
if (declaredClass == null) {
declaredClass = Class.forName(parameter.getName());
}
parameters.add(toObject(parameter));
types.add(declaredClass);
}
Class<?> [] parameterTypes = new Class<?> [types.size()];
types.toArray(parameterTypes);
protocol = (Class<? extends CoprocessorProtocol>)
Class.forName(protocolName);
method = protocol.getMethod(methodName, parameterTypes);
} catch (NoSuchMethodException nsme) {
throw new IOException(nsme);
} catch (ClassNotFoundException cnfe) {
throw new IOException(cnfe);
}
Configuration conf = HBaseConfiguration.create();
for (NameStringPair p: proto.getPropertyList()) {
conf.set(p.getName(), p.getValue());
}
Object[] parameterObjects = new Object[parameters.size()];
parameters.toArray(parameterObjects);
return new Exec(conf, row, protocol,
method, parameterObjects);
}
/**
* Convert a ServerName to a protocol buffer ServerName
*
* @param serverName the ServerName to convert
* @return the converted protocol buffer ServerName
*/
public static HBaseProtos.ServerName
toServerName(final ServerName serverName) {
if (serverName == null) return null;
HBaseProtos.ServerName.Builder builder =
HBaseProtos.ServerName.newBuilder();
builder.setHostName(serverName.getHostname());
if (serverName.getPort() >= 0) {
builder.setPort(serverName.getPort());
}
if (serverName.getStartcode() >= 0) {
builder.setStartCode(serverName.getStartcode());
}
return builder.build();
}
/**
* Convert a RegionInfo to a HRegionInfo
*
* @param proto the RegionInfo to convert
* @return the converted HRegionInfo
*/
public static HRegionInfo
toRegionInfo(final RegionInfo proto) {
if (proto == null) return null;
byte[] tableName = proto.getTableName().toByteArray();
long regionId = proto.getRegionId();
byte[] startKey = null;
byte[] endKey = null;
if (proto.hasStartKey()) {
startKey = proto.getStartKey().toByteArray();
}
if (proto.hasEndKey()) {
endKey = proto.getEndKey().toByteArray();
}
return new HRegionInfo(tableName,
startKey, endKey, false, regionId);
}
/**
* Convert a HRegionInfo to a RegionInfo
*
* @param info the HRegionInfo to convert
* @return the converted RegionInfo
*/
public static RegionInfo
toRegionInfo(final HRegionInfo info) {
if (info == null) return null;
RegionInfo.Builder builder = RegionInfo.newBuilder();
builder.setTableName(ByteString.copyFrom(info.getTableName()));
builder.setRegionId(info.getRegionId());
if (info.getStartKey() != null) {
builder.setStartKey(ByteString.copyFrom(info.getStartKey()));
}
if (info.getEndKey() != null) {
builder.setEndKey(ByteString.copyFrom(info.getEndKey()));
}
return builder.build();
}
/**
* Convert a protocol buffer Get to a client Get
*
* @param get the protocol buffer Get to convert
* @return the converted client Get
* @throws IOException
*/
public static Get toGet(
final ClientProtos.Get proto) throws IOException {
if (proto == null) return null;
byte[] row = proto.getRow().toByteArray();
RowLock rowLock = null;
if (proto.hasLockId()) {
rowLock = new RowLock(proto.getLockId());
}
Get get = new Get(row, rowLock);
if (proto.hasCacheBlocks()) {
get.setCacheBlocks(proto.getCacheBlocks());
}
if (proto.hasMaxVersions()) {
get.setMaxVersions(proto.getMaxVersions());
}
if (proto.hasTimeRange()) {
HBaseProtos.TimeRange timeRange = proto.getTimeRange();
long minStamp = 0;
long maxStamp = Long.MAX_VALUE;
if (timeRange.hasFrom()) {
minStamp = timeRange.getFrom();
}
if (timeRange.hasTo()) {
maxStamp = timeRange.getTo();
}
get.setTimeRange(minStamp, maxStamp);
}
if (proto.hasFilter()) {
NameBytesPair filter = proto.getFilter();
get.setFilter((Filter)toObject(filter));
}
for (NameBytesPair attribute: proto.getAttributeList()) {
get.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
}
if (proto.getColumnCount() > 0) {
for (Column column: proto.getColumnList()) {
byte[] family = column.getFamily().toByteArray();
if (column.getQualifierCount() > 0) {
for (ByteString qualifier: column.getQualifierList()) {
get.addColumn(family, qualifier.toByteArray());
}
} else {
get.addFamily(family);
}
}
}
return get;
}
/**
* Convert a protocol buffer Mutate to a Put
*
* @param proto the protocol buffer Mutate to convert
* @return the converted client Put
* @throws DoNotRetryIOException
*/
public static Put toPut(
final Mutate proto) throws DoNotRetryIOException {
MutateType type = proto.getMutateType();
assert type == MutateType.PUT : type.name();
byte[] row = proto.getRow().toByteArray();
long timestamp = HConstants.LATEST_TIMESTAMP;
if (proto.hasTimestamp()) {
timestamp = proto.getTimestamp();
}
RowLock lock = null;
if (proto.hasLockId()) {
lock = new RowLock(proto.getLockId());
}
Put put = new Put(row, timestamp, lock);
put.setWriteToWAL(proto.getWriteToWAL());
for (NameBytesPair attribute: proto.getAttributeList()) {
put.setAttribute(attribute.getName(),
attribute.getValue().toByteArray());
}
for (ColumnValue column: proto.getColumnValueList()) {
byte[] family = column.getFamily().toByteArray();
for (QualifierValue qv: column.getQualifierValueList()) {
byte[] qualifier = qv.getQualifier().toByteArray();
if (!qv.hasValue()) {
throw new DoNotRetryIOException(
"Missing required field: qualifer value");
}
byte[] value = qv.getValue().toByteArray();
long ts = timestamp;
if (qv.hasTimestamp()) {
ts = qv.getTimestamp();
}
put.add(family, qualifier, ts, value);
}
}
return put;
}
/**
* Convert a protocol buffer Mutate to a Delete
*
* @param proto the protocol buffer Mutate to convert
* @return the converted client Delete
*/
public static Delete toDelete(final Mutate proto) {
MutateType type = proto.getMutateType();
assert type == MutateType.DELETE : type.name();
byte[] row = proto.getRow().toByteArray();
long timestamp = HConstants.LATEST_TIMESTAMP;
if (proto.hasTimestamp()) {
timestamp = proto.getTimestamp();
}
RowLock lock = null;
if (proto.hasLockId()) {
lock = new RowLock(proto.getLockId());
}
Delete delete = new Delete(row, timestamp, lock);
delete.setWriteToWAL(proto.getWriteToWAL());
for (NameBytesPair attribute: proto.getAttributeList()) {
delete.setAttribute(attribute.getName(),
attribute.getValue().toByteArray());
}
for (ColumnValue column: proto.getColumnValueList()) {
byte[] family = column.getFamily().toByteArray();
for (QualifierValue qv: column.getQualifierValueList()) {
DeleteType deleteType = qv.getDeleteType();
byte[] qualifier = null;
if (qv.hasQualifier()) {
qualifier = qv.getQualifier().toByteArray();
}
long ts = HConstants.LATEST_TIMESTAMP;
if (qv.hasTimestamp()) {
ts = qv.getTimestamp();
}
if (deleteType == DeleteType.DELETE_ONE_VERSION) {
delete.deleteColumn(family, qualifier, ts);
} else if (deleteType == DeleteType.DELETE_MULTIPLE_VERSIONS) {
delete.deleteColumns(family, qualifier, ts);
} else {
delete.deleteFamily(family, ts);
}
}
}
return delete;
}
/**
* Convert a protocol buffer Mutate to an Append
*
* @param proto the protocol buffer Mutate to convert
* @return the converted client Append
* @throws DoNotRetryIOException
*/
public static Append toAppend(
final Mutate proto) throws DoNotRetryIOException {
MutateType type = proto.getMutateType();
assert type == MutateType.APPEND : type.name();
byte[] row = proto.getRow().toByteArray();
Append append = new Append(row);
append.setWriteToWAL(proto.getWriteToWAL());
for (NameBytesPair attribute: proto.getAttributeList()) {
append.setAttribute(attribute.getName(),
attribute.getValue().toByteArray());
}
for (ColumnValue column: proto.getColumnValueList()) {
byte[] family = column.getFamily().toByteArray();
for (QualifierValue qv: column.getQualifierValueList()) {
byte[] qualifier = qv.getQualifier().toByteArray();
if (!qv.hasValue()) {
throw new DoNotRetryIOException(
"Missing required field: qualifer value");
}
byte[] value = qv.getValue().toByteArray();
append.add(family, qualifier, value);
}
}
return append;
}
/**
* Convert a protocol buffer Mutate to an Increment
*
* @param proto the protocol buffer Mutate to convert
* @return the converted client Increment
* @throws IOException
*/
public static Increment toIncrement(
final Mutate proto) throws IOException {
MutateType type = proto.getMutateType();
assert type == MutateType.INCREMENT : type.name();
RowLock lock = null;
if (proto.hasLockId()) {
lock = new RowLock(proto.getLockId());
}
byte[] row = proto.getRow().toByteArray();
Increment increment = new Increment(row, lock);
increment.setWriteToWAL(proto.getWriteToWAL());
if (proto.hasTimeRange()) {
HBaseProtos.TimeRange timeRange = proto.getTimeRange();
long minStamp = 0;
long maxStamp = Long.MAX_VALUE;
if (timeRange.hasFrom()) {
minStamp = timeRange.getFrom();
}
if (timeRange.hasTo()) {
maxStamp = timeRange.getTo();
}
increment.setTimeRange(minStamp, maxStamp);
}
for (ColumnValue column: proto.getColumnValueList()) {
byte[] family = column.getFamily().toByteArray();
for (QualifierValue qv: column.getQualifierValueList()) {
byte[] qualifier = qv.getQualifier().toByteArray();
if (!qv.hasValue()) {
throw new DoNotRetryIOException(
"Missing required field: qualifer value");
}
long value = Bytes.toLong(qv.getValue().toByteArray());
increment.addColumn(family, qualifier, value);
}
}
return increment;
}
/**
* Convert a protocol buffer Scan to a client Scan
*
* @param proto the protocol buffer Scan to convert
* @return the converted client Scan
* @throws IOException
*/
public static Scan toScan(
final ClientProtos.Scan proto) throws IOException {
byte [] startRow = HConstants.EMPTY_START_ROW;
byte [] stopRow = HConstants.EMPTY_END_ROW;
if (proto.hasStartRow()) {
startRow = proto.getStartRow().toByteArray();
}
if (proto.hasStopRow()) {
stopRow = proto.getStopRow().toByteArray();
}
Scan scan = new Scan(startRow, stopRow);
if (proto.hasCacheBlocks()) {
scan.setCacheBlocks(proto.getCacheBlocks());
}
if (proto.hasMaxVersions()) {
scan.setMaxVersions(proto.getMaxVersions());
}
if (proto.hasTimeRange()) {
HBaseProtos.TimeRange timeRange = proto.getTimeRange();
long minStamp = 0;
long maxStamp = Long.MAX_VALUE;
if (timeRange.hasFrom()) {
minStamp = timeRange.getFrom();
}
if (timeRange.hasTo()) {
maxStamp = timeRange.getTo();
}
scan.setTimeRange(minStamp, maxStamp);
}
if (proto.hasFilter()) {
NameBytesPair filter = proto.getFilter();
scan.setFilter((Filter)toObject(filter));
}
if (proto.hasBatchSize()) {
scan.setBatch(proto.getBatchSize());
}
for (NameBytesPair attribute: proto.getAttributeList()) {
scan.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
}
if (proto.getColumnCount() > 0) {
for (Column column: proto.getColumnList()) {
byte[] family = column.getFamily().toByteArray();
if (column.getQualifierCount() > 0) {
for (ByteString qualifier: column.getQualifierList()) {
scan.addColumn(family, qualifier.toByteArray());
}
} else {
scan.addFamily(family);
}
}
}
return scan;
}
/**
* Convert a client Result to a protocol buffer Result
*
* @param result the client Result to convert
* @return the converted protocol buffer Result
*/
public static ClientProtos.Result toResult(final Result result) {
ClientProtos.Result.Builder builder = ClientProtos.Result.newBuilder();
List<ByteString> protos = new ArrayList<ByteString>();
List<KeyValue> keyValues = result.list();
if (keyValues != null) {
for (KeyValue keyValue: keyValues) {
ByteString value = ByteString.copyFrom(keyValue.getBuffer(),
keyValue.getOffset(), keyValue.getLength());
protos.add(value);
}
}
builder.addAllKeyValueBytes(protos);
return builder.build();
}
/**
* Convert a protocol buffer Result to a client Result
*
* @param proto the protocol buffer Result to convert
* @return the converted client Result
*/
public static Result toResult(final ClientProtos.Result proto) {
List<ByteString> values = proto.getKeyValueBytesList();
List<KeyValue> keyValues = new ArrayList<KeyValue>(values.size());
for (ByteString value: values) {
keyValues.add(new KeyValue(value.toByteArray()));
}
return new Result(keyValues);
}
/**
* Get the HLog entries from a list of protocol buffer WALEntry
*
* @param protoList the list of protocol buffer WALEntry
* @return an array of HLog entries
*/
public static HLog.Entry[]
toHLogEntries(final List<WALEntry> protoList) {
List<HLog.Entry> entries = new ArrayList<HLog.Entry>();
for (WALEntry entry: protoList) {
WALKey walKey = entry.getWalKey();
java.util.UUID clusterId = HConstants.DEFAULT_CLUSTER_ID;
if (walKey.hasClusterId()) {
UUID protoUuid = walKey.getClusterId();
clusterId = new java.util.UUID(
protoUuid.getMostSigBits(), protoUuid.getLeastSigBits());
}
HLogKey key = new HLogKey(walKey.getEncodedRegionName().toByteArray(),
walKey.getTableName().toByteArray(), walKey.getLogSequenceNumber(),
walKey.getWriteTime(), clusterId);
WALEntry.WALEdit walEdit = entry.getEdit();
WALEdit edit = new WALEdit();
for (ByteString keyValue: walEdit.getKeyValueList()) {
edit.add(new KeyValue(keyValue.toByteArray()));
}
if (walEdit.getFamilyScopeCount() > 0) {
TreeMap<byte[], Integer> scopes = new TreeMap<byte[], Integer>();
for (FamilyScope scope: walEdit.getFamilyScopeList()) {
scopes.put(scope.getFamily().toByteArray(),
Integer.valueOf(scope.getScopeType().ordinal()));
}
edit.setScopes(scopes);
}
entries.add(new HLog.Entry(key, edit));
}
return entries.toArray(new HLog.Entry[entries.size()]);
}
/**
* Convert a protocol buffer Parameter to a Java object
*
* @param parameter the protocol buffer Parameter to convert
* @return the converted Java object
* @throws IOException if failed to deserialize the parameter
*/
public static Object toObject(
final NameBytesPair parameter) throws IOException {
if (parameter == null || !parameter.hasValue()) return null;
byte[] bytes = parameter.getValue().toByteArray();
ByteArrayInputStream bais = null;
try {
bais = new ByteArrayInputStream(bytes);
DataInput in = new DataInputStream(bais);
return HbaseObjectWritable.readObject(in, null);
} finally {
if (bais != null) {
bais.close();
}
}
}
/**
* Convert a stringified protocol buffer exception Parameter to a Java Exception
*
* @param parameter the protocol buffer Parameter to convert
* @return the converted Exception
* @throws IOException if failed to deserialize the parameter
*/
@SuppressWarnings("unchecked")
public static Throwable toException(
final NameBytesPair parameter) throws IOException {
if (parameter == null || !parameter.hasValue()) return null;
String desc = parameter.getValue().toStringUtf8();
String type = parameter.getName();
try {
Class<? extends Throwable> c =
(Class<? extends Throwable>)Class.forName(type);
Constructor<? extends Throwable> cn =
c.getDeclaredConstructor(String.class);
return cn.newInstance(desc);
} catch (Exception e) {
throw new IOException(e);
}
}
/**
* Serialize a Java Object into a Parameter. The Java Object should be a
* Writable or protocol buffer Message
*
* @param value the Writable/Message object to be serialized
* @return the converted protocol buffer Parameter
* @throws IOException if failed to serialize the object
*/
public static NameBytesPair toParameter(
final Object value) throws IOException {
Class<?> declaredClass = Object.class;
if (value != null) {
declaredClass = value.getClass();
}
return toParameter(declaredClass, value);
}
/**
* Serialize a Java Object into a Parameter. The Java Object should be a
* Writable or protocol buffer Message
*
* @param declaredClass the declared class of the parameter
* @param value the Writable/Message object to be serialized
* @return the converted protocol buffer Parameter
* @throws IOException if failed to serialize the object
*/
public static NameBytesPair toParameter(
final Class<?> declaredClass, final Object value) throws IOException {
NameBytesPair.Builder builder = NameBytesPair.newBuilder();
builder.setName(declaredClass.getName());
if (value != null) {
ByteArrayOutputStream baos = null;
try {
baos = new ByteArrayOutputStream();
DataOutput out = new DataOutputStream(baos);
Class<?> clz = declaredClass;
if (HbaseObjectWritable.getClassCode(declaredClass) == null) {
clz = value.getClass();
}
HbaseObjectWritable.writeObject(out, value, clz, null);
builder.setValue(
ByteString.copyFrom(baos.toByteArray()));
} finally {
if (baos != null) {
baos.close();
}
}
}
return builder.build();
}
}

View File

@ -0,0 +1,782 @@
/**
* 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.protobuf;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.NavigableMap;
import java.util.NavigableSet;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Action;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Row;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.coprocessor.Exec;
import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest.FamilyPath;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition.CompareType;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.LockRowRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.UnlockRowRequest;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import com.google.protobuf.ByteString;
import com.google.protobuf.Message;
/**
* Helper utility to build protocol buffer requests,
* or build components for protocol buffer requests.
*/
@InterfaceAudience.Private
public final class RequestConverter {
private RequestConverter() {
}
// Start utilities for Client
/**
* Create a new protocol buffer GetRequest to get a row, all columns in a family.
* If there is no such row, return the closest row before it.
*
* @param regionName the name of the region to get
* @param row the row to get
* @param family the column family to get
* @param closestRowBefore if the requested row doesn't exist,
* should return the immediate row before
* @return a protocol buffer GetReuqest
*/
public static GetRequest buildGetRequest(final byte[] regionName,
final byte[] row, final byte[] family, boolean closestRowBefore) {
GetRequest.Builder builder = GetRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setClosestRowBefore(closestRowBefore);
builder.setRegion(region);
Column.Builder columnBuilder = Column.newBuilder();
columnBuilder.setFamily(ByteString.copyFrom(family));
ClientProtos.Get.Builder getBuilder =
ClientProtos.Get.newBuilder();
getBuilder.setRow(ByteString.copyFrom(row));
getBuilder.addColumn(columnBuilder.build());
builder.setGet(getBuilder.build());
return builder.build();
}
/**
* Create a protocol buffer GetRequest for a client Get
*
* @param regionName the name of the region to get
* @param get the client Get
* @return a protocol buffer GetReuqest
*/
public static GetRequest buildGetRequest(final byte[] regionName,
final Get get) throws IOException {
return buildGetRequest(regionName, get, false);
}
/**
* Create a protocol buffer GetRequest for a client Get
*
* @param regionName the name of the region to get
* @param get the client Get
* @param existenceOnly indicate if check row existence only
* @return a protocol buffer GetReuqest
*/
public static GetRequest buildGetRequest(final byte[] regionName,
final Get get, final boolean existenceOnly) throws IOException {
GetRequest.Builder builder = GetRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setExistenceOnly(existenceOnly);
builder.setRegion(region);
builder.setGet(buildGet(get));
return builder.build();
}
/**
* Create a protocol buffer MutateRequest for a client increment
*
* @param regionName
* @param row
* @param family
* @param qualifier
* @param amount
* @param writeToWAL
* @return a mutate request
*/
public static MutateRequest buildMutateRequest(
final byte[] regionName, final byte[] row, final byte[] family,
final byte [] qualifier, final long amount, final boolean writeToWAL) {
MutateRequest.Builder builder = MutateRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
Mutate.Builder mutateBuilder = Mutate.newBuilder();
mutateBuilder.setRow(ByteString.copyFrom(row));
mutateBuilder.setMutateType(MutateType.INCREMENT);
mutateBuilder.setWriteToWAL(writeToWAL);
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
columnBuilder.setFamily(ByteString.copyFrom(family));
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
valueBuilder.setValue(ByteString.copyFrom(Bytes.toBytes(amount)));
valueBuilder.setQualifier(ByteString.copyFrom(qualifier));
columnBuilder.addQualifierValue(valueBuilder.build());
mutateBuilder.addColumnValue(columnBuilder.build());
builder.setMutate(mutateBuilder.build());
return builder.build();
}
/**
* Create a protocol buffer MutateRequest for a conditioned put
*
* @param regionName
* @param row
* @param family
* @param qualifier
* @param comparator
* @param compareType
* @param put
* @return a mutate request
* @throws IOException
*/
public static MutateRequest buildMutateRequest(
final byte[] regionName, final byte[] row, final byte[] family,
final byte [] qualifier, final WritableByteArrayComparable comparator,
final CompareType compareType, final Put put) throws IOException {
MutateRequest.Builder builder = MutateRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
Condition condition = buildCondition(
row, family, qualifier, comparator, compareType);
builder.setMutate(buildMutate(MutateType.PUT, put));
builder.setCondition(condition);
return builder.build();
}
/**
* Create a protocol buffer MutateRequest for a conditioned delete
*
* @param regionName
* @param row
* @param family
* @param qualifier
* @param comparator
* @param compareType
* @param delete
* @return a mutate request
* @throws IOException
*/
public static MutateRequest buildMutateRequest(
final byte[] regionName, final byte[] row, final byte[] family,
final byte [] qualifier, final WritableByteArrayComparable comparator,
final CompareType compareType, final Delete delete) throws IOException {
MutateRequest.Builder builder = MutateRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
Condition condition = buildCondition(
row, family, qualifier, comparator, compareType);
builder.setMutate(buildMutate(MutateType.DELETE, delete));
builder.setCondition(condition);
return builder.build();
}
/**
* Create a protocol buffer MutateRequest for a put
*
* @param regionName
* @param put
* @return a mutate request
* @throws IOException
*/
public static MutateRequest buildMutateRequest(
final byte[] regionName, final Put put) throws IOException {
MutateRequest.Builder builder = MutateRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
builder.setMutate(buildMutate(MutateType.PUT, put));
return builder.build();
}
/**
* Create a protocol buffer MutateRequest for an append
*
* @param regionName
* @param append
* @return a mutate request
* @throws IOException
*/
public static MutateRequest buildMutateRequest(
final byte[] regionName, final Append append) throws IOException {
MutateRequest.Builder builder = MutateRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
builder.setMutate(buildMutate(MutateType.APPEND, append));
return builder.build();
}
/**
* Create a protocol buffer MutateRequest for a client increment
*
* @param regionName
* @param increment
* @return a mutate request
*/
public static MutateRequest buildMutateRequest(
final byte[] regionName, final Increment increment) {
MutateRequest.Builder builder = MutateRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
builder.setMutate(buildMutate(increment));
return builder.build();
}
/**
* Create a protocol buffer MutateRequest for a delete
*
* @param regionName
* @param delete
* @return a mutate request
* @throws IOException
*/
public static MutateRequest buildMutateRequest(
final byte[] regionName, final Delete delete) throws IOException {
MutateRequest.Builder builder = MutateRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
builder.setMutate(buildMutate(MutateType.DELETE, delete));
return builder.build();
}
/**
* Create a protocol buffer MultiRequest for a row mutations
*
* @param regionName
* @param rowMutations
* @return a multi request
* @throws IOException
*/
public static MultiRequest buildMultiRequest(final byte[] regionName,
final RowMutations rowMutations) throws IOException {
MultiRequest.Builder builder = MultiRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
builder.setAtomic(true);
for (Mutation mutation: rowMutations.getMutations()) {
MutateType mutateType = null;
if (mutation instanceof Put) {
mutateType = MutateType.PUT;
} else if (mutation instanceof Delete) {
mutateType = MutateType.DELETE;
} else {
throw new DoNotRetryIOException(
"RowMutations supports only put and delete, not "
+ mutation.getClass().getName());
}
Mutate mutate = buildMutate(mutateType, mutation);
builder.addAction(ProtobufUtil.toParameter(mutate));
}
return builder.build();
}
/**
* Create a protocol buffer ScanRequest for a client Scan
*
* @param regionName
* @param scan
* @param numberOfRows
* @param closeScanner
* @return a scan request
* @throws IOException
*/
public static ScanRequest buildScanRequest(final byte[] regionName,
final Scan scan, final int numberOfRows,
final boolean closeScanner) throws IOException {
ScanRequest.Builder builder = ScanRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setNumberOfRows(numberOfRows);
builder.setCloseScanner(closeScanner);
builder.setRegion(region);
ClientProtos.Scan.Builder scanBuilder =
ClientProtos.Scan.newBuilder();
scanBuilder.setCacheBlocks(scan.getCacheBlocks());
scanBuilder.setBatchSize(scan.getBatch());
scanBuilder.setMaxVersions(scan.getMaxVersions());
TimeRange timeRange = scan.getTimeRange();
if (!timeRange.isAllTime()) {
HBaseProtos.TimeRange.Builder timeRangeBuilder =
HBaseProtos.TimeRange.newBuilder();
timeRangeBuilder.setFrom(timeRange.getMin());
timeRangeBuilder.setTo(timeRange.getMax());
scanBuilder.setTimeRange(timeRangeBuilder.build());
}
Map<String, byte[]> attributes = scan.getAttributesMap();
if (!attributes.isEmpty()) {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey());
attributeBuilder.setValue(ByteString.copyFrom(attribute.getValue()));
scanBuilder.addAttribute(attributeBuilder.build());
}
}
byte[] startRow = scan.getStartRow();
if (startRow != null && startRow.length > 0) {
scanBuilder.setStartRow(ByteString.copyFrom(startRow));
}
byte[] stopRow = scan.getStopRow();
if (stopRow != null && stopRow.length > 0) {
scanBuilder.setStopRow(ByteString.copyFrom(stopRow));
}
if (scan.hasFilter()) {
scanBuilder.setFilter(ProtobufUtil.toParameter(scan.getFilter()));
}
Column.Builder columnBuilder = Column.newBuilder();
for (Map.Entry<byte[],NavigableSet<byte []>>
family: scan.getFamilyMap().entrySet()) {
columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
NavigableSet<byte []> columns = family.getValue();
columnBuilder.clearQualifier();
if (columns != null && columns.size() > 0) {
for (byte [] qualifier: family.getValue()) {
if (qualifier != null) {
columnBuilder.addQualifier(ByteString.copyFrom(qualifier));
}
}
}
scanBuilder.addColumn(columnBuilder.build());
}
builder.setScan(scanBuilder.build());
return builder.build();
}
/**
* Create a protocol buffer ScanRequest for a scanner id
*
* @param scannerId
* @param numberOfRows
* @param closeScanner
* @return a scan request
*/
public static ScanRequest buildScanRequest(final long scannerId,
final int numberOfRows, final boolean closeScanner) {
ScanRequest.Builder builder = ScanRequest.newBuilder();
builder.setNumberOfRows(numberOfRows);
builder.setCloseScanner(closeScanner);
builder.setScannerId(scannerId);
return builder.build();
}
/**
* Create a protocol buffer LockRowRequest
*
* @param regionName
* @param row
* @return a lock row request
*/
public static LockRowRequest buildLockRowRequest(
final byte[] regionName, final byte[] row) {
LockRowRequest.Builder builder = LockRowRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
builder.addRow(ByteString.copyFrom(row));
return builder.build();
}
/**
* Create a protocol buffer UnlockRowRequest
*
* @param regionName
* @param lockId
* @return a unlock row request
*/
public static UnlockRowRequest buildUnlockRowRequest(
final byte[] regionName, final long lockId) {
UnlockRowRequest.Builder builder = UnlockRowRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
builder.setLockId(lockId);
return builder.build();
}
/**
* Create a protocol buffer bulk load request
*
* @param familyPaths
* @param regionName
* @return a bulk load request
*/
public static BulkLoadHFileRequest buildBulkLoadHFileRequest(
final List<Pair<byte[], String>> familyPaths, final byte[] regionName) {
BulkLoadHFileRequest.Builder builder = BulkLoadHFileRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
FamilyPath.Builder familyPathBuilder = FamilyPath.newBuilder();
for (Pair<byte[], String> familyPath: familyPaths) {
familyPathBuilder.setFamily(ByteString.copyFrom(familyPath.getFirst()));
familyPathBuilder.setPath(familyPath.getSecond());
builder.addFamilyPath(familyPathBuilder.build());
}
return builder.build();
}
/**
* Create a protocol buffer coprocessor exec request
*
* @param regionName
* @param exec
* @return a coprocessor exec request
* @throws IOException
*/
public static ExecCoprocessorRequest buildExecCoprocessorRequest(
final byte[] regionName, final Exec exec) throws IOException {
ExecCoprocessorRequest.Builder builder = ExecCoprocessorRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
builder.setCall(buildExec(exec));
return builder.build();
}
/**
* Create a protocol buffer multi request for a list of actions.
* RowMutations in the list (if any) will be ignored.
*
* @param regionName
* @param actions
* @return a multi request
* @throws IOException
*/
public static <R> MultiRequest buildMultiRequest(final byte[] regionName,
final List<Action<R>> actions) throws IOException {
MultiRequest.Builder builder = MultiRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
for (Action<R> action: actions) {
Message protoAction = null;
Row row = action.getAction();
if (row instanceof Get) {
protoAction = buildGet((Get)row);
} else if (row instanceof Put) {
protoAction = buildMutate(MutateType.PUT, (Put)row);
} else if (row instanceof Delete) {
protoAction = buildMutate(MutateType.DELETE, (Delete)row);
} else if (row instanceof Exec) {
protoAction = buildExec((Exec)row);
} else if (row instanceof Append) {
protoAction = buildMutate(MutateType.APPEND, (Append)row);
} else if (row instanceof Increment) {
protoAction = buildMutate((Increment)row);
} else if (row instanceof RowMutations) {
continue; // ignore RowMutations
} else {
throw new DoNotRetryIOException(
"multi doesn't support " + row.getClass().getName());
}
builder.addAction(ProtobufUtil.toParameter(protoAction));
}
return builder.build();
}
// End utilities for Client
/**
* Create a protocol buffer Condition
*
* @param row
* @param family
* @param qualifier
* @param comparator
* @param compareType
* @return a Condition
* @throws IOException
*/
private static Condition buildCondition(final byte[] row,
final byte[] family, final byte [] qualifier,
final WritableByteArrayComparable comparator,
final CompareType compareType) throws IOException {
Condition.Builder builder = Condition.newBuilder();
builder.setRow(ByteString.copyFrom(row));
builder.setFamily(ByteString.copyFrom(family));
builder.setQualifier(ByteString.copyFrom(qualifier));
builder.setComparator(ProtobufUtil.toParameter(comparator));
builder.setCompareType(compareType);
return builder.build();
}
/**
* Create a new protocol buffer Exec based on a client Exec
*
* @param exec
* @return
* @throws IOException
*/
private static ClientProtos.Exec buildExec(
final Exec exec) throws IOException {
ClientProtos.Exec.Builder
builder = ClientProtos.Exec.newBuilder();
Configuration conf = exec.getConf();
if (conf != null) {
NameStringPair.Builder propertyBuilder = NameStringPair.newBuilder();
Iterator<Entry<String, String>> iterator = conf.iterator();
while (iterator.hasNext()) {
Entry<String, String> entry = iterator.next();
propertyBuilder.setName(entry.getKey());
propertyBuilder.setValue(entry.getValue());
builder.addProperty(propertyBuilder.build());
}
}
builder.setProtocolName(exec.getProtocolName());
builder.setMethodName(exec.getMethodName());
builder.setRow(ByteString.copyFrom(exec.getRow()));
Object[] parameters = exec.getParameters();
if (parameters != null && parameters.length > 0) {
Class<?>[] declaredClasses = exec.getParameterClasses();
for (int i = 0, n = parameters.length; i < n; i++) {
builder.addParameter(
ProtobufUtil.toParameter(declaredClasses[i], parameters[i]));
}
}
return builder.build();
}
/**
* Create a protocol buffer Get based on a client Get.
*
* @param get the client Get
* @return a protocol buffer Get
* @throws IOException
*/
private static ClientProtos.Get buildGet(
final Get get) throws IOException {
ClientProtos.Get.Builder builder =
ClientProtos.Get.newBuilder();
builder.setRow(ByteString.copyFrom(get.getRow()));
builder.setCacheBlocks(get.getCacheBlocks());
builder.setMaxVersions(get.getMaxVersions());
if (get.getLockId() >= 0) {
builder.setLockId(get.getLockId());
}
if (get.getFilter() != null) {
builder.setFilter(ProtobufUtil.toParameter(get.getFilter()));
}
TimeRange timeRange = get.getTimeRange();
if (!timeRange.isAllTime()) {
HBaseProtos.TimeRange.Builder timeRangeBuilder =
HBaseProtos.TimeRange.newBuilder();
timeRangeBuilder.setFrom(timeRange.getMin());
timeRangeBuilder.setTo(timeRange.getMax());
builder.setTimeRange(timeRangeBuilder.build());
}
Map<String, byte[]> attributes = get.getAttributesMap();
if (!attributes.isEmpty()) {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey());
attributeBuilder.setValue(ByteString.copyFrom(attribute.getValue()));
builder.addAttribute(attributeBuilder.build());
}
}
if (get.hasFamilies()) {
Column.Builder columnBuilder = Column.newBuilder();
Map<byte[], NavigableSet<byte[]>> families = get.getFamilyMap();
for (Map.Entry<byte[], NavigableSet<byte[]>> family: families.entrySet()) {
NavigableSet<byte[]> qualifiers = family.getValue();
columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
columnBuilder.clearQualifier();
if (qualifiers != null && qualifiers.size() > 0) {
for (byte[] qualifier: qualifiers) {
if (qualifier != null) {
columnBuilder.addQualifier(ByteString.copyFrom(qualifier));
}
}
}
builder.addColumn(columnBuilder.build());
}
}
return builder.build();
}
private static Mutate buildMutate(final Increment increment) {
Mutate.Builder builder = Mutate.newBuilder();
builder.setRow(ByteString.copyFrom(increment.getRow()));
builder.setMutateType(MutateType.INCREMENT);
builder.setWriteToWAL(increment.getWriteToWAL());
if (increment.getLockId() >= 0) {
builder.setLockId(increment.getLockId());
}
TimeRange timeRange = increment.getTimeRange();
if (!timeRange.isAllTime()) {
HBaseProtos.TimeRange.Builder timeRangeBuilder =
HBaseProtos.TimeRange.newBuilder();
timeRangeBuilder.setFrom(timeRange.getMin());
timeRangeBuilder.setTo(timeRange.getMax());
builder.setTimeRange(timeRangeBuilder.build());
}
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
for (Map.Entry<byte[],NavigableMap<byte[], Long>>
family: increment.getFamilyMap().entrySet()) {
columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
columnBuilder.clearQualifierValue();
NavigableMap<byte[], Long> values = family.getValue();
if (values != null && values.size() > 0) {
for (Map.Entry<byte[], Long> value: values.entrySet()) {
valueBuilder.setQualifier(ByteString.copyFrom(value.getKey()));
valueBuilder.setValue(ByteString.copyFrom(
Bytes.toBytes(value.getValue().longValue())));
columnBuilder.addQualifierValue(valueBuilder.build());
}
}
builder.addColumnValue(columnBuilder.build());
}
return builder.build();
}
/**
* Create a protocol buffer Mutate based on a client Mutation
*
* @param mutateType
* @param mutation
* @return a mutate
* @throws IOException
*/
private static Mutate buildMutate(final MutateType mutateType,
final Mutation mutation) throws IOException {
Mutate.Builder mutateBuilder = Mutate.newBuilder();
mutateBuilder.setRow(ByteString.copyFrom(mutation.getRow()));
mutateBuilder.setMutateType(mutateType);
mutateBuilder.setWriteToWAL(mutation.getWriteToWAL());
if (mutation.getLockId() >= 0) {
mutateBuilder.setLockId(mutation.getLockId());
}
mutateBuilder.setTimestamp(mutation.getTimeStamp());
Map<String, byte[]> attributes = mutation.getAttributesMap();
if (!attributes.isEmpty()) {
NameBytesPair.Builder attributeBuilder = NameBytesPair.newBuilder();
for (Map.Entry<String, byte[]> attribute: attributes.entrySet()) {
attributeBuilder.setName(attribute.getKey());
attributeBuilder.setValue(ByteString.copyFrom(attribute.getValue()));
mutateBuilder.addAttribute(attributeBuilder.build());
}
}
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
for (Map.Entry<byte[],List<KeyValue>>
family: mutation.getFamilyMap().entrySet()) {
columnBuilder.setFamily(ByteString.copyFrom(family.getKey()));
columnBuilder.clearQualifierValue();
for (KeyValue value: family.getValue()) {
valueBuilder.setQualifier(ByteString.copyFrom(value.getQualifier()));
valueBuilder.setValue(ByteString.copyFrom(value.getValue()));
valueBuilder.setTimestamp(value.getTimestamp());
if (mutateType == MutateType.DELETE) {
KeyValue.Type keyValueType = KeyValue.Type.codeToType(value.getType());
valueBuilder.setDeleteType(toDeleteType(keyValueType));
}
columnBuilder.addQualifierValue(valueBuilder.build());
}
mutateBuilder.addColumnValue(columnBuilder.build());
}
return mutateBuilder.build();
}
/**
* Convert a byte array to a protocol buffer RegionSpecifier
*
* @param type the region specifier type
* @param value the region specifier byte array value
* @return a protocol buffer RegionSpecifier
*/
private static RegionSpecifier buildRegionSpecifier(
final RegionSpecifierType type, final byte[] value) {
RegionSpecifier.Builder regionBuilder = RegionSpecifier.newBuilder();
regionBuilder.setValue(ByteString.copyFrom(value));
regionBuilder.setType(type);
return regionBuilder.build();
}
/**
* Convert a delete KeyValue type to protocol buffer DeleteType.
*
* @param type
* @return
* @throws IOException
*/
private static DeleteType toDeleteType(
KeyValue.Type type) throws IOException {
switch (type) {
case Delete:
return DeleteType.DELETE_ONE_VERSION;
case DeleteColumn:
return DeleteType.DELETE_MULTIPLE_VERSIONS;
case DeleteFamily:
return DeleteType.DELETE_FAMILY;
default:
throw new IOException("Unknown delete type: " + type);
}
}
}

View File

@ -0,0 +1,187 @@
/**
* 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.protobuf;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.util.StringUtils;
import com.google.protobuf.ByteString;
/**
* Helper utility to build protocol buffer responses,
* or retrieve data from protocol buffer responses.
*/
@InterfaceAudience.Private
public final class ResponseConverter {
private ResponseConverter() {
}
// Start utilities for Client
/**
* Get the client Results from a protocol buffer ScanResponse
*
* @param response the protocol buffer ScanResponse
* @return the client Results in the response
*/
public static Result[] getResults(final ScanResponse response) {
if (response == null) return null;
int count = response.getResultCount();
Result[] results = new Result[count];
for (int i = 0; i < count; i++) {
results[i] = ProtobufUtil.toResult(response.getResult(i));
}
return results;
}
/**
* Get the results from a protocol buffer MultiResponse
*
* @param proto the protocol buffer MultiResponse to convert
* @return the results in the MultiResponse
* @throws IOException
*/
public static List<Object> getResults(
final ClientProtos.MultiResponse proto) throws IOException {
List<Object> results = new ArrayList<Object>();
List<ActionResult> resultList = proto.getResultList();
for (int i = 0, n = resultList.size(); i < n; i++) {
ActionResult result = resultList.get(i);
if (result.hasException()) {
results.add(ProtobufUtil.toException(result.getException()));
} else if (result.hasValue()) {
Object value = ProtobufUtil.toObject(result.getValue());
if (value instanceof ClientProtos.Result) {
results.add(ProtobufUtil.toResult((ClientProtos.Result)value));
} else {
results.add(value);
}
} else {
results.add(new Result());
}
}
return results;
}
/**
* Wrap a throwable to an action result.
*
* @param t
* @return an action result
*/
public static ActionResult buildActionResult(final Throwable t) {
ActionResult.Builder builder = ActionResult.newBuilder();
NameBytesPair.Builder parameterBuilder = NameBytesPair.newBuilder();
parameterBuilder.setName(t.getClass().getName());
parameterBuilder.setValue(
ByteString.copyFromUtf8(StringUtils.stringifyException(t)));
builder.setException(parameterBuilder.build());
return builder.build();
}
// End utilities for Client
// Start utilities for Admin
/**
* Get the list of regions to flush from a RollLogWriterResponse
*
* @param proto the RollLogWriterResponse
* @return the the list of regions to flush
*/
public static byte[][] getRegions(final RollWALWriterResponse proto) {
if (proto == null || proto.getRegionToFlushCount() == 0) return null;
List<byte[]> regions = new ArrayList<byte[]>();
for (ByteString region: proto.getRegionToFlushList()) {
regions.add(region.toByteArray());
}
return (byte[][])regions.toArray();
}
/**
* Get the list of region info from a GetOnlineRegionResponse
*
* @param proto the GetOnlineRegionResponse
* @return the list of region info
*/
public static List<HRegionInfo> getRegionInfos
(final GetOnlineRegionResponse proto) {
if (proto == null || proto.getRegionInfoCount() == 0) return null;
List<HRegionInfo> regionInfos = new ArrayList<HRegionInfo>();
for (RegionInfo regionInfo: proto.getRegionInfoList()) {
regionInfos.add(ProtobufUtil.toRegionInfo(regionInfo));
}
return regionInfos;
}
/**
* Get the region info from a GetRegionInfoResponse
*
* @param proto the GetRegionInfoResponse
* @return the region info
*/
public static HRegionInfo getRegionInfo
(final GetRegionInfoResponse proto) {
if (proto == null || proto.getRegionInfo() == null) return null;
return ProtobufUtil.toRegionInfo(proto.getRegionInfo());
}
/**
* Get the region opening state from a OpenRegionResponse
*
* @param proto the OpenRegionResponse
* @return the region opening state
*/
public static RegionOpeningState getRegionOpeningState
(final OpenRegionResponse proto) {
if (proto == null || proto.getOpeningStateCount() != 1) return null;
return RegionOpeningState.valueOf(
proto.getOpeningState(0).name());
}
/**
* Check if the region is closed from a CloseRegionResponse
*
* @param proto the CloseRegionResponse
* @return the region close state
*/
public static boolean isClosed
(final CloseRegionResponse proto) {
if (proto == null || !proto.hasClosed()) return false;
return proto.getClosed();
}
// End utilities for Admin
}

View File

@ -40,11 +40,9 @@ import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Random;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
@ -119,7 +117,7 @@ import org.apache.hadoop.hbase.ipc.Invocation;
import org.apache.hadoop.hbase.ipc.ProtocolSignature;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
import org.apache.hadoop.hbase.protobuf.ClientProtocol;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
@ -139,7 +137,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CompressionTest;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.InfoServer;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Sleeper;
@ -170,36 +167,22 @@ import com.google.common.collect.Lists;
* the HMaster. There are many HRegionServers in a single HBase deployment.
*/
@InterfaceAudience.Private
public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
Runnable, RegionServerServices {
public class HRegionServer extends RegionServer
implements HRegionInterface, HBaseRPCErrorHandler {
public static final Log LOG = LogFactory.getLog(HRegionServer.class);
// Set when a report to the master comes back with a message asking us to
// shutdown. Also set by call to stop when debugging or running unit tests
// of HRegionServer in isolation.
protected volatile boolean stopped = false;
// A state before we go into stopped state. At this stage we're closing user
// space regions.
private boolean stopping = false;
// Go down hard. Used if file system becomes unavailable and also in
// debugging and unit tests.
protected volatile boolean abortRequested;
private volatile boolean killed = false;
// If false, the file system has become unavailable
protected volatile boolean fsOk;
protected final Configuration conf;
protected final AtomicBoolean haveRootRegion = new AtomicBoolean(false);
private HFileSystem fs;
private boolean useHBaseChecksum; // verify hbase checksums?
private Path rootDir;
private final Random rand = new Random();
//RegionName vs current action in progress
//true - if open region action in progress
@ -207,13 +190,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
private final ConcurrentSkipListMap<byte[], Boolean> regionsInTransitionInRS =
new ConcurrentSkipListMap<byte[], Boolean>(Bytes.BYTES_COMPARATOR);
/**
* Map of regions currently being served by this region server. Key is the
* encoded region name. All access should be synchronized.
*/
protected final Map<String, HRegion> onlineRegions =
new ConcurrentHashMap<String, HRegion>();
protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
final int numRetries;
@ -222,8 +198,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
protected final int numRegionsToReport;
private final long maxScannerResultSize;
// Remote HMaster
private HMasterRegionInterface hbaseMaster;
@ -233,13 +207,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
private final InetSocketAddress isa;
// Leases
private Leases leases;
// Request counter.
// Do we need this? Can't we just sum region counters? St.Ack 20110412
private AtomicInteger requestCount = new AtomicInteger();
// Info server. Default access so can be used by unit tests. REGIONSERVER
// is name of the webapp and the attribute name used stuffing this instance
// into web context.
@ -263,9 +230,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
// Compactions
public CompactSplitThread compactSplitThread;
// Cache flushing
MemStoreFlusher cacheFlusher;
/*
* Check for compactions requests.
*/
@ -279,9 +243,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
// flag set after we're done setting up server threads (used for testing)
protected volatile boolean isOnline;
final Map<String, RegionScanner> scanners =
new ConcurrentHashMap<String, RegionScanner>();
// zookeeper connection and watcher
private ZooKeeperWatcher zooKeeper;
@ -405,8 +366,10 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
if (initialIsa.getAddress() == null) {
throw new IllegalArgumentException("Failed resolve of " + initialIsa);
}
this.rpcServer = HBaseRPC.getServer(this,
new Class<?>[]{HRegionInterface.class, HBaseRPCErrorHandler.class,
new Class<?>[]{HRegionInterface.class, ClientProtocol.class,
HBaseRPCErrorHandler.class,
OnlineRegions.class},
initialIsa.getHostName(), // BindAddress is IP we got for this server.
initialIsa.getPort(),
@ -445,12 +408,8 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
}
}
private static final int NORMAL_QOS = 0;
private static final int QOS_THRESHOLD = 10; // the line between low and high qos
private static final int HIGH_QOS = 100;
@Retention(RetentionPolicy.RUNTIME)
private @interface QosPriority {
protected @interface QosPriority {
int priority() default 0;
}
@ -648,7 +607,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
// Create the thread for the ThriftServer.
if (conf.getBoolean("hbase.regionserver.export.thrift", false)) {
thriftServer = new HRegionThriftServer(this, conf);
thriftServer = new HRegionThriftServer((RegionServer)this, conf);
thriftServer.start();
LOG.info("Started Thrift API from Region Server.");
}
@ -1080,110 +1039,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
return r != null ? createRegionLoad(r) : null;
}
/*
* Cleanup after Throwable caught invoking method. Converts <code>t</code> to
* IOE if it isn't already.
*
* @param t Throwable
*
* @return Throwable converted to an IOE; methods can only let out IOEs.
*/
private Throwable cleanup(final Throwable t) {
return cleanup(t, null);
}
/*
* Cleanup after Throwable caught invoking method. Converts <code>t</code> to
* IOE if it isn't already.
*
* @param t Throwable
*
* @param msg Message to log in error. Can be null.
*
* @return Throwable converted to an IOE; methods can only let out IOEs.
*/
private Throwable cleanup(final Throwable t, final String msg) {
// Don't log as error if NSRE; NSRE is 'normal' operation.
if (t instanceof NotServingRegionException) {
LOG.debug("NotServingRegionException; " + t.getMessage());
return t;
}
if (msg == null) {
LOG.error("", RemoteExceptionHandler.checkThrowable(t));
} else {
LOG.error(msg, RemoteExceptionHandler.checkThrowable(t));
}
if (!checkOOME(t)) {
checkFileSystem();
}
return t;
}
/*
* @param t
*
* @return Make <code>t</code> an IOE if it isn't already.
*/
private IOException convertThrowableToIOE(final Throwable t) {
return convertThrowableToIOE(t, null);
}
/*
* @param t
*
* @param msg Message to put in new IOE if passed <code>t</code> is not an IOE
*
* @return Make <code>t</code> an IOE if it isn't already.
*/
private IOException convertThrowableToIOE(final Throwable t, final String msg) {
return (t instanceof IOException ? (IOException) t : msg == null
|| msg.length() == 0 ? new IOException(t) : new IOException(msg, t));
}
/*
* Check if an OOME and, if so, abort immediately to avoid creating more objects.
*
* @param e
*
* @return True if we OOME'd and are aborting.
*/
public boolean checkOOME(final Throwable e) {
boolean stop = false;
try {
if (e instanceof OutOfMemoryError
|| (e.getCause() != null && e.getCause() instanceof OutOfMemoryError)
|| (e.getMessage() != null && e.getMessage().contains(
"java.lang.OutOfMemoryError"))) {
stop = true;
LOG.fatal(
"Run out of memory; HRegionServer will abort itself immediately", e);
}
} finally {
if (stop) {
Runtime.getRuntime().halt(1);
}
}
return stop;
}
/**
* Checks to see if the file system is still accessible. If not, sets
* abortRequested and stopRequested
*
* @return false if file system is not available
*/
public boolean checkFileSystem() {
if (this.fsOk && this.fs != null) {
try {
FSUtils.checkFileSystemAvailable(this.fs);
} catch (IOException e) {
abort("File System not available", e);
this.fsOk = false;
}
}
return this.fsOk;
}
/*
* Inner class that runs on a long period checking if regions need compaction.
*/
@ -2334,15 +2189,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
}
}
protected long addScanner(RegionScanner s) throws LeaseStillHeldException {
long scannerId = -1L;
scannerId = rand.nextLong();
String scannerName = String.valueOf(scannerId);
scanners.put(scannerName, s);
this.leases.createLease(scannerName, new ScannerListener(scannerName));
return scannerId;
}
public Result next(final long scannerId) throws IOException {
Result[] res = next(scannerId, 1);
if (res == null || res.length == 0) {
@ -2468,42 +2314,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
}
}
/**
* Instantiated as a scanner lease. If the lease times out, the scanner is
* closed
*/
private class ScannerListener implements LeaseListener {
private final String scannerName;
ScannerListener(final String n) {
this.scannerName = n;
}
public void leaseExpired() {
RegionScanner s = scanners.remove(this.scannerName);
if (s != null) {
LOG.info("Scanner " + this.scannerName + " lease expired on region "
+ s.getRegionInfo().getRegionNameAsString());
try {
HRegion region = getRegion(s.getRegionInfo().getRegionName());
if (region != null && region.getCoprocessorHost() != null) {
region.getCoprocessorHost().preScannerClose(s);
}
s.close();
if (region != null && region.getCoprocessorHost() != null) {
region.getCoprocessorHost().postScannerClose(s);
}
} catch (IOException e) {
LOG.error("Closing scanner for "
+ s.getRegionInfo().getRegionNameAsString(), e);
}
} else {
LOG.info("Scanner " + this.scannerName + " lease expired");
}
}
}
//
// Methods that do the actual work for the remote API
//
@ -2585,39 +2395,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
}
}
protected long addRowLock(Integer r, HRegion region)
throws LeaseStillHeldException {
long lockId = -1L;
lockId = rand.nextLong();
String lockName = String.valueOf(lockId);
rowlocks.put(lockName, r);
this.leases.createLease(lockName, new RowLockListener(lockName, region));
return lockId;
}
/**
* Method to get the Integer lock identifier used internally from the long
* lock identifier used by the client.
*
* @param lockId
* long row lock identifier from client
* @return intId Integer row lock used internally in HRegion
* @throws IOException
* Thrown if this is not a valid client lock id.
*/
Integer getLockFromId(long lockId) throws IOException {
if (lockId == -1L) {
return null;
}
String lockName = String.valueOf(lockId);
Integer rl = rowlocks.get(lockName);
if (rl == null) {
throw new UnknownRowLockException("Invalid row lock");
}
this.leases.renewLease(lockName);
return rl;
}
@Override
@QosPriority(priority=HIGH_QOS)
public void unlockRow(byte[] regionName, long lockId) throws IOException {
@ -2663,30 +2440,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
return region.bulkLoadHFiles(familyPaths);
}
Map<String, Integer> rowlocks = new ConcurrentHashMap<String, Integer>();
/**
* Instantiated as a row lock lease. If the lease times out, the row lock is
* released
*/
private class RowLockListener implements LeaseListener {
private final String lockName;
private final HRegion region;
RowLockListener(final String lockName, final HRegion region) {
this.lockName = lockName;
this.region = region;
}
public void leaseExpired() {
LOG.info("Row Lock " + this.lockName + " lease expired");
Integer r = rowlocks.remove(this.lockName);
if (r != null) {
region.releaseRowLock(r);
}
}
}
// Region open/close direct RPCs
@Override
@ -3057,15 +2810,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
return r;
}
/**
* @param regionName
* @return HRegion for the passed binary <code>regionName</code> or null if
* named region is not member of the online regions.
*/
public HRegion getOnlineRegion(final byte[] regionName) {
return getFromOnlineRegions(HRegionInfo.encodeRegionName(regionName));
}
/** @return the request count */
public AtomicInteger getRequestCount() {
return this.requestCount;
@ -3083,25 +2827,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
return this.cacheFlusher;
}
/**
* Protected utility method for safely obtaining an HRegion handle.
*
* @param regionName
* Name of online {@link HRegion} to return
* @return {@link HRegion} for <code>regionName</code>
* @throws NotServingRegionException
*/
protected HRegion getRegion(final byte[] regionName)
throws NotServingRegionException {
HRegion region = null;
region = getOnlineRegion(regionName);
if (region == null) {
throw new NotServingRegionException("Region is not online: " +
Bytes.toStringBinary(regionName));
}
return region;
}
/**
* Get the top N most loaded regions this server is serving so we can tell the
* master which regions it can reallocate if we're overloaded. TODO: actually
@ -3123,21 +2848,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
return regions.toArray(new HRegionInfo[regions.size()]);
}
/**
* Called to verify that this server is up and running.
*
* @throws IOException
*/
protected void checkOpen() throws IOException {
if (this.stopped || this.abortRequested) {
throw new RegionServerStoppedException("Server " + getServerName() +
" not running" + (this.abortRequested ? ", aborting" : ""));
}
if (!fsOk) {
throw new RegionServerStoppedException("File system not available");
}
}
@Override
@QosPriority(priority=HIGH_QOS)
public ProtocolSignature getProtocolSignature(
@ -3145,6 +2855,8 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
throws IOException {
if (protocol.equals(HRegionInterface.class.getName())) {
return new ProtocolSignature(HRegionInterface.VERSION, null);
} else if (protocol.equals(ClientProtocol.class.getName())) {
return new ProtocolSignature(ClientProtocol.VERSION, null);
}
throw new IOException("Unknown protocol: " + protocol);
}
@ -3155,6 +2867,8 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
throws IOException {
if (protocol.equals(HRegionInterface.class.getName())) {
return HRegionInterface.VERSION;
} else if (protocol.equals(ClientProtocol.class.getName())) {
return ClientProtocol.VERSION;
}
throw new IOException("Unknown protocol: " + protocol);
}
@ -3687,4 +3401,4 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
mxBeanInfo);
LOG.info("Registered RegionServer MXBean");
}
}
}

View File

@ -36,11 +36,17 @@ import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
import org.apache.hadoop.hbase.thrift.ThriftServerRunner;
import org.apache.hadoop.hbase.thrift.ThriftUtilities;
import org.apache.hadoop.hbase.thrift.generated.IOError;
import org.apache.hadoop.hbase.thrift.generated.TRowResult;
import com.google.protobuf.ServiceException;
/**
* HRegionThriftServer - this class starts up a Thrift server in the same
* JVM where the RegionServer is running. It inherits most of the
@ -56,14 +62,14 @@ public class HRegionThriftServer extends Thread {
public static final Log LOG = LogFactory.getLog(HRegionThriftServer.class);
private final HRegionServer rs;
private final RegionServer rs;
private final ThriftServerRunner serverRunner;
/**
* Create an instance of the glue object that connects the
* RegionServer with the standard ThriftServer implementation
*/
HRegionThriftServer(HRegionServer regionServer, Configuration conf)
HRegionThriftServer(RegionServer regionServer, Configuration conf)
throws IOException {
super("Region Thrift Server");
this.rs = regionServer;
@ -130,7 +136,10 @@ public class HRegionThriftServer extends Thread {
if (columns == null) {
Get get = new Get(row);
get.setTimeRange(Long.MIN_VALUE, timestamp);
Result result = rs.get(regionName, get);
GetRequest request =
RequestConverter.buildGetRequest(regionName, get);
GetResponse response = rs.get(null, request);
Result result = ProtobufUtil.toResult(response.getResult());
return ThriftUtilities.rowResultFromHBase(result);
}
Get get = new Get(row);
@ -143,7 +152,10 @@ public class HRegionThriftServer extends Thread {
}
}
get.setTimeRange(Long.MIN_VALUE, timestamp);
Result result = rs.get(regionName, get);
GetRequest request =
RequestConverter.buildGetRequest(regionName, get);
GetResponse response = rs.get(null, request);
Result result = ProtobufUtil.toResult(response.getResult());
return ThriftUtilities.rowResultFromHBase(result);
} catch (NotServingRegionException e) {
if (!redirect) {
@ -153,6 +165,10 @@ public class HRegionThriftServer extends Thread {
LOG.debug("ThriftServer redirecting getRowWithColumnsTs");
return super.getRowWithColumnsTs(tableName, rowb, columns, timestamp,
attributes);
} catch (ServiceException se) {
IOException e = ProtobufUtil.getRemoteException(se);
LOG.warn(e.getMessage(), e);
throw new IOError(e.getMessage());
} catch (IOException e) {
LOG.warn(e.getMessage(), e);
throw new IOError(e.getMessage());

View File

@ -55,7 +55,7 @@ import java.io.IOException;
@InterfaceAudience.Private
public class Leases extends HasThread {
private static final Log LOG = LogFactory.getLog(Leases.class.getName());
private final int leasePeriod;
protected final int leasePeriod;
private final int leaseCheckFrequency;
private volatile DelayQueue<Lease> leaseQueue = new DelayQueue<Lease>();
protected final Map<String, Lease> leases = new HashMap<String, Lease>();

File diff suppressed because it is too large Load Diff

View File

@ -16,10 +16,10 @@
* limitations under the License.
*/
// This file contains protocol buffers that are used for RegionAdmin service.
// This file contains protocol buffers that are used for Admin service.
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
option java_outer_classname = "RegionAdminProtos";
option java_outer_classname = "AdminProtos";
option java_generic_services = true;
option java_generate_equals_and_hash = true;
option optimize_for = SPEED;
@ -146,7 +146,7 @@ message WALEntry {
}
message WALEdit {
repeated KeyValue keyValue = 1;
repeated bytes keyValue = 1;
repeated FamilyScope familyScope = 2;
enum ScopeType {
@ -197,7 +197,7 @@ message GetServerInfoResponse {
required ServerName serverName = 1;
}
service RegionAdminService {
service AdminService {
rpc getRegionInfo(GetRegionInfoRequest)
returns(GetRegionInfoResponse);

View File

@ -16,10 +16,10 @@
* limitations under the License.
*/
// This file contains protocol buffers that are used for RegionClient service.
// This file contains protocol buffers that are used for Client service.
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
option java_outer_classname = "RegionClientProtos";
option java_outer_classname = "ClientProtos";
option java_generic_services = true;
option java_generate_equals_and_hash = true;
option optimize_for = SPEED;
@ -34,27 +34,26 @@ message Column {
repeated bytes qualifier = 2;
}
message Attribute {
required string name = 1;
optional bytes value = 2;
}
/**
* The protocol buffer version of Get
*/
message Get {
required bytes row = 1;
repeated Column column = 2;
repeated Attribute attribute = 3;
repeated NameBytesPair attribute = 3;
optional uint64 lockId = 4;
optional Parameter filter = 5;
optional NameBytesPair filter = 5;
optional TimeRange timeRange = 6;
optional uint32 maxVersions = 7 [default = 1];
optional bool cacheBlocks = 8 [default = true];
}
/**
* For performance reason, we don't use KeyValue
* here. We use the actual KeyValue bytes.
*/
message Result {
repeated KeyValue value = 1;
repeated bytes keyValueBytes = 1;
}
/**
@ -89,8 +88,6 @@ message GetResponse {
/**
* Condition to check if the value of a given cell (row,
* family, qualifier) matches a value via a given comparator.
* The value is optional since some comparator may not require
* a value to compare, for example, checking null.
*
* Condition is used in check and mutate operations.
*/
@ -99,8 +96,7 @@ message Condition {
required bytes family = 2;
required bytes qualifier = 3;
required CompareType compareType = 4;
required Comparator comparator = 5;
optional bytes value = 6;
required NameBytesPair comparator = 5;
enum CompareType {
LESS = 0;
@ -111,17 +107,6 @@ message Condition {
GREATER = 5;
NO_OP = 6;
}
enum Comparator {
BINARY_COMPARATOR = 0;
BINARY_PREFIX_COMPARATOR = 1;
BIT_AND_COMPARATOR = 2;
BIT_OR_COMPARATOR = 3;
BIT_XOR_COMPARATOR = 4;
NULL_COMPARATOR = 5;
REGEX_STRING_COMPARATOR = 6;
SUBSTRING_COMPARATOR = 7;
}
}
/**
@ -133,7 +118,7 @@ message Mutate {
required bytes row = 1;
required MutateType mutateType = 2;
repeated ColumnValue columnValue = 3;
repeated Attribute attribute = 4;
repeated NameBytesPair attribute = 4;
optional uint64 timestamp = 5;
optional uint64 lockId = 6;
optional bool writeToWAL = 7 [default = true];
@ -147,22 +132,23 @@ message Mutate {
INCREMENT = 1;
PUT = 2;
DELETE = 3;
DELETE_COLUMN = 4;
DELETE_FAMILY = 5;
}
enum DeleteType {
DELETE_ONE_VERSION = 0;
DELETE_MULTIPLE_VERSIONS = 1;
DELETE_FAMILY = 2;
}
message ColumnValue {
required bytes family = 1;
repeated QualifierValue qualifierValue = 2;
// Default timestamp for qalifier values,
// or timestamp of the column family to be deleted
optional uint64 timestamp = 3;
message QualifierValue {
required bytes qualifier = 1;
optional bytes qualifier = 1;
optional bytes value = 2;
optional uint64 timestamp = 3;
optional DeleteType deleteType = 4;
}
}
}
@ -200,15 +186,14 @@ message MutateResponse {
*/
message Scan {
repeated Column column = 1;
repeated Attribute attribute = 2;
repeated NameBytesPair attribute = 2;
optional bytes startRow = 3;
optional bytes stopRow = 4;
optional Parameter filter = 5;
optional NameBytesPair filter = 5;
optional TimeRange timeRange = 6;
optional uint32 maxVersions = 7 [default = 1];
optional bool cacheBlocks = 8 [default = true];
optional uint32 rowsToCache = 9;
optional uint32 batchSize = 10;
optional uint32 batchSize = 9;
}
/**
@ -223,10 +208,11 @@ message Scan {
* a trip if you are not interested in remaining results.
*/
message ScanRequest {
optional uint64 scannerId = 1;
optional RegionSpecifier region = 1;
optional Scan scan = 2;
optional uint32 numberOfRows = 3;
optional bool closeScanner = 4;
optional uint64 scannerId = 3;
optional uint32 numberOfRows = 4;
optional bool closeScanner = 5;
}
/**
@ -276,16 +262,6 @@ message BulkLoadHFileResponse {
required bool loaded = 1;
}
message Parameter {
required string type = 1;
optional bytes binaryValue = 2;
}
message Property {
required string name = 1;
required string value = 2;
}
/**
* An individual coprocessor call. You must specify the protocol,
* the method, and the row to which the call will be executed.
@ -302,8 +278,8 @@ message Exec {
required bytes row = 1;
required string protocolName = 2;
required string methodName = 3;
repeated Property property = 4;
repeated Parameter parameter = 5;
repeated NameStringPair property = 4;
repeated NameBytesPair parameter = 5;
}
/**
@ -320,32 +296,40 @@ message ExecCoprocessorRequest {
}
message ExecCoprocessorResponse {
required bytes regionName = 1;
required Parameter value = 2;
required NameBytesPair value = 1;
}
/**
* You can execute a list of actions on regions assigned
* to the same region server, if you can't find an individual
* call which meets your requirement.
* An individual action result. The result will in the
* same order as the action in the request. If an action
* returns a value, it is set in value field. If it doesn't
* return anything, the result will be empty. If an action
* fails to execute due to any exception, the exception
* is returned as a stringified parameter.
*/
message ActionResult {
optional NameBytesPair value = 1;
optional NameBytesPair exception = 2;
}
/**
* You can execute a list of actions on a given region in order.
*
* The multi request can have a list of requests. Each request
* should be a protocol buffer encoded request such as GetRequest,
* MutateRequest, ExecCoprocessorRequest.
*
* If the list contains multiple mutate requests only, atomic can
* be set to make sure they can be processed atomically.
* If it is a list of mutate actions, atomic can be set
* to make sure they can be processed atomically, just like
* RowMutations.
*/
message MultiRequest {
repeated Parameter request = 1;
optional bool atomic = 2;
required RegionSpecifier region = 1;
repeated NameBytesPair action = 2;
optional bool atomic = 3;
}
message MultiResponse {
repeated Parameter response = 1;
repeated ActionResult result = 1;
}
service RegionClientService {
service ClientService {
rpc get(GetRequest)
returns(GetResponse);

View File

@ -101,3 +101,16 @@ message ServerName {
optional uint32 port = 2;
optional uint64 startCode = 3;
}
// Comment data structures
message NameStringPair {
required string name = 1;
required string value = 2;
}
message NameBytesPair {
required string name = 1;
optional bytes value = 2;
}

View File

@ -41,6 +41,9 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.client.ServerCallable;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.protobuf.ClientProtocol;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.Writables;
@ -58,6 +61,9 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
/**
* Test {@link CatalogTracker}
*/
@ -131,13 +137,16 @@ public class TestCatalogTracker {
/**
* Test interruptable while blocking wait on root and meta.
* @throws IOException
* @throws ServiceException
* @throws InterruptedException
*/
@Test public void testInterruptWaitOnMetaAndRoot()
throws IOException, InterruptedException {
HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
HConnection connection = mockConnection(implementation);
throws IOException, InterruptedException, ServiceException {
final ClientProtocol client = Mockito.mock(ClientProtocol.class);
HConnection connection = mockConnection(null, client);
try {
Mockito.when(client.get((RpcController)Mockito.any(), (GetRequest)Mockito.any())).
thenReturn(GetResponse.newBuilder().build());
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
ServerName hsa = ct.getRootLocation();
Assert.assertNull(hsa);
@ -176,10 +185,11 @@ public class TestCatalogTracker {
*/
@Test
public void testServerNotRunningIOException()
throws IOException, InterruptedException, KeeperException {
throws IOException, InterruptedException, KeeperException, ServiceException {
// Mock an HRegionInterface.
final HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
HConnection connection = mockConnection(implementation);
final ClientProtocol client = Mockito.mock(ClientProtocol.class);
HConnection connection = mockConnection(implementation, client);
try {
// If a 'getRegionInfo' is called on mocked HRegionInterface, throw IOE
// the first time. 'Succeed' the second time we are called.
@ -198,6 +208,8 @@ public class TestCatalogTracker {
Mockito.when(connection.getRegionServerWithRetries((ServerCallable<Result>)Mockito.any())).
thenReturn(getMetaTableRowResult());
Mockito.when(client.get((RpcController)Mockito.any(), (GetRequest)Mockito.any())).
thenReturn(GetResponse.newBuilder().build());
// Now start up the catalogtracker with our doctored Connection.
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
try {
@ -245,17 +257,18 @@ public class TestCatalogTracker {
* @throws IOException
* @throws InterruptedException
* @throws KeeperException
* @throws ServiceException
*/
@Test
public void testGetMetaServerConnectionFails()
throws IOException, InterruptedException, KeeperException {
// Mock an HRegionInterface.
final HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
HConnection connection = mockConnection(implementation);
throws IOException, InterruptedException, KeeperException, ServiceException {
// Mock an ClientProtocol.
final ClientProtocol implementation = Mockito.mock(ClientProtocol.class);
HConnection connection = mockConnection(null, implementation);
try {
// If a 'get' is called on mocked interface, throw connection refused.
Mockito.when(implementation.get((byte[]) Mockito.any(), (Get) Mockito.any())).
thenThrow(new ConnectException("Connection refused"));
Mockito.when(implementation.get((RpcController) Mockito.any(), (GetRequest) Mockito.any())).
thenThrow(new ServiceException(new ConnectException("Connection refused")));
// Now start up the catalogtracker with our doctored Connection.
final CatalogTracker ct = constructAndStartCatalogTracker(connection);
try {
@ -371,7 +384,7 @@ public class TestCatalogTracker {
// to make our test work.
// Mock an HRegionInterface.
final HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
HConnection connection = mockConnection(implementation);
HConnection connection = mockConnection(implementation, null);
try {
// Now the ct is up... set into the mocks some answers that make it look
// like things have been getting assigned. Make it so we'll return a
@ -419,6 +432,7 @@ public class TestCatalogTracker {
/**
* @param implementation An {@link HRegionInterface} instance; you'll likely
* want to pass a mocked HRS; can be null.
* @param client A mocked ClientProtocol instance, can be null
* @return Mock up a connection that returns a {@link Configuration} when
* {@link HConnection#getConfiguration()} is called, a 'location' when
* {@link HConnection#getRegionLocation(byte[], byte[], boolean)} is called,
@ -429,7 +443,8 @@ public class TestCatalogTracker {
* when done with this mocked Connection.
* @throws IOException
*/
private HConnection mockConnection(final HRegionInterface implementation)
private HConnection mockConnection(
final HRegionInterface implementation, final ClientProtocol client)
throws IOException {
HConnection connection =
HConnectionTestingUtility.getMockedConnection(UTIL.getConfiguration());
@ -449,6 +464,11 @@ public class TestCatalogTracker {
Mockito.when(connection.getHRegionConnection(Mockito.anyString(), Mockito.anyInt())).
thenReturn(implementation);
}
if (client != null) {
// If a call to getClient, return this implementation.
Mockito.when(connection.getClient(Mockito.anyString(), Mockito.anyInt())).
thenReturn(client);
}
return connection;
}

View File

@ -31,8 +31,10 @@ import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ClientProtocol;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -42,6 +44,9 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
/**
* Test MetaReader/Editor but without spinning up a cluster.
* We mock regionserver back and forth (we do spin up a zk cluster).
@ -82,7 +87,8 @@ public class TestMetaReaderEditorNoCluster {
* @throws InterruptedException
*/
@Test
public void testRideOverServerNotRunning() throws IOException, InterruptedException {
public void testRideOverServerNotRunning()
throws IOException, InterruptedException, ServiceException {
// Need a zk watcher.
ZooKeeperWatcher zkw = new ZooKeeperWatcher(UTIL.getConfiguration(),
this.getClass().getSimpleName(), ABORTABLE, true);
@ -92,27 +98,16 @@ public class TestMetaReaderEditorNoCluster {
HConnection connection = null;
CatalogTracker ct = null;
try {
// Mock an HRegionInterface. Our mock implementation will fail a few
// Mock an ClientProtocol. Our mock implementation will fail a few
// times when we go to open a scanner.
final HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
// When openScanner called throw IOE 'Server not running' a few times
final ClientProtocol implementation = Mockito.mock(ClientProtocol.class);
// When scan called throw IOE 'Server not running' a few times
// before we return a scanner id. Whats WEIRD is that these
// exceptions do not show in the log because they are caught and only
// printed if we FAIL. We eventually succeed after retry so these don't
// show. We will know if they happened or not because we will ask
// mockito at the end of this test to verify that openscanner was indeed
// mockito at the end of this test to verify that scan was indeed
// called the wanted number of times.
final long scannerid = 123L;
Mockito.when(implementation.openScanner((byte [])Mockito.any(),
(Scan)Mockito.any())).
thenThrow(new IOException("Server not running (1 of 3)")).
thenThrow(new IOException("Server not running (2 of 3)")).
thenThrow(new IOException("Server not running (3 of 3)")).
thenReturn(scannerid);
// Make it so a verifiable answer comes back when next is called. Return
// the verifiable answer and then a null so we stop scanning. Our
// verifiable answer is something that looks like a row in META with
// a server and startcode that is that of the above defined servername.
List<KeyValue> kvs = new ArrayList<KeyValue>();
final byte [] rowToVerify = Bytes.toBytes("rowToVerify");
kvs.add(new KeyValue(rowToVerify,
@ -124,10 +119,19 @@ public class TestMetaReaderEditorNoCluster {
kvs.add(new KeyValue(rowToVerify,
HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
Bytes.toBytes(sn.getStartcode())));
final Result [] result = new Result [] {new Result(kvs)};
Mockito.when(implementation.next(Mockito.anyLong(), Mockito.anyInt())).
thenReturn(result).
thenReturn(null);
final Result [] results = new Result [] {new Result(kvs)};
ScanResponse.Builder builder = ScanResponse.newBuilder();
for (Result result: results) {
builder.addResult(ProtobufUtil.toResult(result));
}
Mockito.when(implementation.scan(
(RpcController)Mockito.any(), (ScanRequest)Mockito.any())).
thenThrow(new ServiceException("Server not running (1 of 3)")).
thenThrow(new ServiceException("Server not running (2 of 3)")).
thenThrow(new ServiceException("Server not running (3 of 3)")).
thenReturn(ScanResponse.newBuilder().setScannerId(1234567890L).build())
.thenReturn(builder.build()).thenReturn(
ScanResponse.newBuilder().setMoreResults(false).build());
// Associate a spied-upon HConnection with UTIL.getConfiguration. Need
// to shove this in here first so it gets picked up all over; e.g. by
@ -150,7 +154,7 @@ public class TestMetaReaderEditorNoCluster {
// Now shove our HRI implementation into the spied-upon connection.
Mockito.doReturn(implementation).
when(connection).getHRegionConnection(Mockito.anyString(), Mockito.anyInt());
when(connection).getClient(Mockito.anyString(), Mockito.anyInt());
// Now start up the catalogtracker with our doctored Connection.
ct = new CatalogTracker(zkw, null, connection, ABORTABLE, 0);
@ -160,10 +164,10 @@ public class TestMetaReaderEditorNoCluster {
assertTrue(hris.size() == 1);
assertTrue(hris.firstEntry().getKey().equals(HRegionInfo.FIRST_META_REGIONINFO));
assertTrue(Bytes.equals(rowToVerify, hris.firstEntry().getValue().getRow()));
// Finally verify that openscanner was called four times -- three times
// with exception and then on 4th attempt we succeed.
Mockito.verify(implementation, Mockito.times(4)).
openScanner((byte [])Mockito.any(), (Scan)Mockito.any());
// Finally verify that scan was called four times -- three times
// with exception and then on 4th, 5th and 6th attempt we succeed
Mockito.verify(implementation, Mockito.times(6)).
scan((RpcController)Mockito.any(), (ScanRequest)Mockito.any());
} finally {
if (ct != null) ct.stop();
HConnectionManager.deleteConnection(UTIL.getConfiguration(), true);

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionImplementation;
import org.apache.hadoop.hbase.client.HConnectionManager.HConnectionKey;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.protobuf.ClientProtocol;
import org.mockito.Mockito;
/**
@ -92,7 +93,8 @@ public class HConnectionTestingUtility {
* @throws IOException
*/
public static HConnection getMockedConnectionAndDecorate(final Configuration conf,
final HRegionInterface implementation, final ServerName sn, final HRegionInfo hri)
final HRegionInterface implementation, final ClientProtocol client,
final ServerName sn, final HRegionInfo hri)
throws IOException {
HConnection c = HConnectionTestingUtility.getMockedConnection(conf);
Mockito.doNothing().when(c).close();
@ -108,6 +110,11 @@ public class HConnectionTestingUtility {
Mockito.when(c.getHRegionConnection(Mockito.anyString(), Mockito.anyInt())).
thenReturn(implementation);
}
if (client != null) {
// If a call to getClient, return this client.
Mockito.when(c.getClient(Mockito.anyString(), Mockito.anyInt())).
thenReturn(client);
}
return c;
}

View File

@ -99,6 +99,7 @@ import org.junit.experimental.categories.Category;
import com.google.common.collect.Lists;
import com.google.protobuf.Message;
import com.google.protobuf.RpcController;
@Category(SmallTests.class)
public class TestHbaseObjectWritable extends TestCase {
@ -523,6 +524,7 @@ public class TestHbaseObjectWritable extends TestCase {
assertEquals(80,HbaseObjectWritable.getClassCode(Message.class).intValue());
assertEquals(81,HbaseObjectWritable.getClassCode(Array.class).intValue());
assertEquals(82,HbaseObjectWritable.getClassCode(RpcController.class).intValue());
}
/**
@ -531,7 +533,7 @@ public class TestHbaseObjectWritable extends TestCase {
* note on the test above.
*/
public void testGetNextObjectCode(){
assertEquals(82,HbaseObjectWritable.getNextClassCode());
assertEquals(83,HbaseObjectWritable.getNextClassCode());
}
@org.junit.Rule

View File

@ -50,6 +50,23 @@ import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.ipc.ProtocolSignature;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ClientProtocol;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.LockRowRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.LockRowResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
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.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.UnlockRowRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.UnlockRowResponse;
import org.apache.hadoop.hbase.regionserver.CompactionRequestor;
import org.apache.hadoop.hbase.regionserver.FlushRequester;
import org.apache.hadoop.hbase.regionserver.HRegion;
@ -64,6 +81,9 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
/**
* A mock RegionServer implementation.
* Use this when you can't bend Mockito to your liking (e.g. return null result
@ -72,7 +92,7 @@ import org.apache.zookeeper.KeeperException;
* {@link #setGetResult(byte[], byte[], Result)} for how to fill the backing data
* store that the get pulls from.
*/
class MockRegionServer implements HRegionInterface, RegionServerServices {
class MockRegionServer implements HRegionInterface, ClientProtocol, RegionServerServices {
private final ServerName sn;
private final ZooKeeperWatcher zkw;
private final Configuration conf;
@ -245,9 +265,8 @@ class MockRegionServer implements HRegionInterface, RegionServerServices {
@Override
public Result get(byte[] regionName, Get get) throws IOException {
Map<byte [], Result> m = this.gets.get(regionName);
if (m == null) return null;
return m.get(get.getRow());
// TODO Auto-generated method stub
return null;
}
@Override
@ -597,4 +616,87 @@ class MockRegionServer implements HRegionInterface, RegionServerServices {
public void mutateRow(byte[] regionName, RowMutations rm) throws IOException {
// TODO Auto-generated method stub
}
@Override
public GetResponse get(RpcController controller, GetRequest request)
throws ServiceException {
byte[] regionName = request.getRegion().getValue().toByteArray();
Map<byte [], Result> m = this.gets.get(regionName);
GetResponse.Builder builder = GetResponse.newBuilder();
if (m != null) {
byte[] row = request.getGet().getRow().toByteArray();
builder.setResult(ProtobufUtil.toResult(m.get(row)));
}
return builder.build();
}
@Override
public MutateResponse mutate(RpcController controller, MutateRequest request)
throws ServiceException {
// TODO Auto-generated method stub
return null;
}
@Override
public ScanResponse scan(RpcController controller, ScanRequest request)
throws ServiceException {
ScanResponse.Builder builder = ScanResponse.newBuilder();
try {
if (request.hasScan()) {
byte[] regionName = request.getRegion().getValue().toByteArray();
builder.setScannerId(openScanner(regionName, null));
builder.setMoreResults(true);
}
else {
long scannerId = request.getScannerId();
Result result = next(scannerId);
if (result != null) {
builder.addResult(ProtobufUtil.toResult(result));
builder.setMoreResults(true);
}
else {
builder.setMoreResults(false);
close(scannerId);
}
}
} catch (IOException ie) {
throw new ServiceException(ie);
}
return builder.build();
}
@Override
public LockRowResponse lockRow(RpcController controller,
LockRowRequest request) throws ServiceException {
// TODO Auto-generated method stub
return null;
}
@Override
public UnlockRowResponse unlockRow(RpcController controller,
UnlockRowRequest request) throws ServiceException {
// TODO Auto-generated method stub
return null;
}
@Override
public BulkLoadHFileResponse bulkLoadHFile(RpcController controller,
BulkLoadHFileRequest request) throws ServiceException {
// TODO Auto-generated method stub
return null;
}
@Override
public ExecCoprocessorResponse execCoprocessor(RpcController controller,
ExecCoprocessorRequest request) throws ServiceException {
// TODO Auto-generated method stub
return null;
}
@Override
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse multi(
RpcController controller, MultiRequest request) throws ServiceException {
// TODO Auto-generated method stub
return null;
}
}

View File

@ -31,23 +31,25 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerLoad;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
import org.apache.hadoop.hbase.executor.RegionTransitionData;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ClientProtocol;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
@ -65,6 +67,9 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
/**
* Test {@link AssignmentManager}
@ -151,7 +156,7 @@ public class TestAssignmentManager {
*/
@Test(timeout = 5000)
public void testBalanceOnMasterFailoverScenarioWithOpenedNode()
throws IOException, KeeperException, InterruptedException {
throws IOException, KeeperException, InterruptedException, ServiceException {
AssignmentManagerWithExtrasForTesting am =
setUpMockedAssignmentManager(this.server, this.serverManager);
try {
@ -194,7 +199,7 @@ public class TestAssignmentManager {
@Test(timeout = 5000)
public void testBalanceOnMasterFailoverScenarioWithClosedNode()
throws IOException, KeeperException, InterruptedException {
throws IOException, KeeperException, InterruptedException, ServiceException {
AssignmentManagerWithExtrasForTesting am =
setUpMockedAssignmentManager(this.server, this.serverManager);
try {
@ -238,7 +243,7 @@ public class TestAssignmentManager {
@Test(timeout = 5000)
public void testBalanceOnMasterFailoverScenarioWithOfflineNode()
throws IOException, KeeperException, InterruptedException {
throws IOException, KeeperException, InterruptedException, ServiceException {
AssignmentManagerWithExtrasForTesting am =
setUpMockedAssignmentManager(this.server, this.serverManager);
try {
@ -363,7 +368,8 @@ public class TestAssignmentManager {
* @throws IOException
*/
@Test
public void testShutdownHandler() throws KeeperException, IOException {
public void testShutdownHandler()
throws KeeperException, IOException, ServiceException {
// Create and startup an executor. This is used by AssignmentManager
// handling zk callbacks.
ExecutorService executor = startupMasterExecutor("testShutdownHandler");
@ -380,19 +386,20 @@ public class TestAssignmentManager {
// Need to set up a fake scan of meta for the servershutdown handler
// Make an RS Interface implementation. Make it so a scanner can go against it.
HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
ClientProtocol implementation = Mockito.mock(ClientProtocol.class);
// Get a meta row result that has region up on SERVERNAME_A
Result r = Mocking.getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
Mockito.when(implementation.openScanner((byte [])Mockito.any(), (Scan)Mockito.any())).
thenReturn(System.currentTimeMillis());
// Return a good result first and then return null to indicate end of scan
Mockito.when(implementation.next(Mockito.anyLong(), Mockito.anyInt())).
thenReturn(new Result [] {r}, (Result [])null);
ScanResponse.Builder builder = ScanResponse.newBuilder();
builder.setMoreResults(false);
builder.addResult(ProtobufUtil.toResult(r));
Mockito.when(implementation.scan(
(RpcController)Mockito.any(), (ScanRequest)Mockito.any())).
thenReturn(builder.build());
// Get a connection w/ mocked up common methods.
HConnection connection =
HConnectionTestingUtility.getMockedConnectionAndDecorate(HTU.getConfiguration(),
implementation, SERVERNAME_B, REGIONINFO);
null, implementation, SERVERNAME_B, REGIONINFO);
// Make it so we can get a catalogtracker from servermanager.. .needed
// down in guts of server shutdown handler.
@ -531,7 +538,7 @@ public class TestAssignmentManager {
*/
private AssignmentManagerWithExtrasForTesting setUpMockedAssignmentManager(final Server server,
final ServerManager manager)
throws IOException, KeeperException {
throws IOException, KeeperException, ServiceException {
// We need a mocked catalog tracker. Its used by our AM instance.
CatalogTracker ct = Mockito.mock(CatalogTracker.class);
// Make an RS Interface implementation. Make it so a scanner can go against
@ -539,21 +546,24 @@ public class TestAssignmentManager {
// messing with. Needed when "new master" joins cluster. AM will try and
// rebuild its list of user regions and it will also get the HRI that goes
// with an encoded name by doing a Get on .META.
HRegionInterface ri = Mockito.mock(HRegionInterface.class);
ClientProtocol ri = Mockito.mock(ClientProtocol.class);
// Get a meta row result that has region up on SERVERNAME_A for REGIONINFO
Result r = Mocking.getMetaTableRowResult(REGIONINFO, SERVERNAME_A);
Mockito.when(ri .openScanner((byte[]) Mockito.any(), (Scan) Mockito.any())).
thenReturn(System.currentTimeMillis());
// Return good result 'r' first and then return null to indicate end of scan
Mockito.when(ri.next(Mockito.anyLong(), Mockito.anyInt())).
thenReturn(new Result[] { r }, (Result[]) null);
ScanResponse.Builder builder = ScanResponse.newBuilder();
builder.setMoreResults(false);
builder.addResult(ProtobufUtil.toResult(r));
Mockito.when(ri.scan(
(RpcController)Mockito.any(), (ScanRequest)Mockito.any())).
thenReturn(builder.build());
// If a get, return the above result too for REGIONINFO
Mockito.when(ri.get((byte[]) Mockito.any(), (Get) Mockito.any())).
thenReturn(r);
GetResponse.Builder getBuilder = GetResponse.newBuilder();
getBuilder.setResult(ProtobufUtil.toResult(r));
Mockito.when(ri.get((RpcController)Mockito.any(), (GetRequest) Mockito.any())).
thenReturn(getBuilder.build());
// Get a connection w/ mocked up common methods.
HConnection connection = HConnectionTestingUtility.
getMockedConnectionAndDecorate(HTU.getConfiguration(), ri, SERVERNAME_B,
REGIONINFO);
getMockedConnectionAndDecorate(HTU.getConfiguration(), null,
ri, SERVERNAME_B, REGIONINFO);
// Make it so we can get the connection from our mocked catalogtracker
Mockito.when(ct.getConnection()).thenReturn(connection);
// Create and startup an executor. Used by AM handling zk callbacks.

View File

@ -54,6 +54,10 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ClientProtocol;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables;
@ -62,6 +66,9 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
@Category(SmallTests.class)
public class TestCatalogJanitor {
/**
@ -76,12 +83,22 @@ public class TestCatalogJanitor {
MockServer(final HBaseTestingUtility htu)
throws NotAllMetaRegionsOnlineException, IOException, InterruptedException {
this.c = htu.getConfiguration();
ClientProtocol ri = Mockito.mock(ClientProtocol.class);
MutateResponse.Builder builder = MutateResponse.newBuilder();
builder.setProcessed(true);
try {
Mockito.when(ri.mutate(
(RpcController)Mockito.any(), (MutateRequest)Mockito.any())).
thenReturn(builder.build());
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
// Mock an HConnection and a HRegionInterface implementation. Have the
// HConnection return the HRI. Have the HRI return a few mocked up responses
// to make our test work.
this.connection =
HConnectionTestingUtility.getMockedConnectionAndDecorate(this.c,
Mockito.mock(HRegionInterface.class),
Mockito.mock(HRegionInterface.class), ri,
new ServerName("example.org,12345,6789"),
HRegionInfo.FIRST_META_REGIONINFO);
// Set hbase.rootdir into test dir.

View File

@ -194,7 +194,7 @@ public class TestMasterNoCluster {
// associate so the below mocking of a connection will fail.
HConnection connection =
HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(),
rs0, rs0.getServerName(), HRegionInfo.ROOT_REGIONINFO);
rs0, rs0, rs0.getServerName(), HRegionInfo.ROOT_REGIONINFO);
return new CatalogTracker(zk, conf, connection, abortable, defaultTimeout);
}
};
@ -271,7 +271,7 @@ public class TestMasterNoCluster {
// of a connection will fail.
HConnection connection =
HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(),
rs0, rs0.getServerName(), HRegionInfo.ROOT_REGIONINFO);
rs0, rs0, rs0.getServerName(), HRegionInfo.ROOT_REGIONINFO);
return new CatalogTracker(zk, conf, connection, abortable, defaultTimeout);
}
};

View File

@ -25,6 +25,11 @@ import java.util.List;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
import com.google.protobuf.ServiceException;
/**
* A region server that will OOME.
@ -42,10 +47,16 @@ public class OOMERegionServer extends HRegionServer {
public void put(byte [] regionName, Put put)
throws IOException {
super.put(regionName, put);
for (int i = 0; i < 30; i++) {
// Add the batch update 30 times to bring on the OOME faster.
this.retainer.add(put);
try {
MutateRequest request =
RequestConverter.buildMutateRequest(regionName, put);
super.mutate(null, request);
for (int i = 0; i < 30; i++) {
// Add the batch update 30 times to bring on the OOME faster.
this.retainer.add(put);
}
} catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se);
}
}

View File

@ -39,6 +39,9 @@ import org.apache.hadoop.hbase.client.ServerCallable;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.ipc.HRegionInterface;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.junit.Test;
@ -145,7 +148,9 @@ public class TestHRegionServerBulkLoad {
LOG.debug("Going to connect to server " + location + " for row "
+ Bytes.toStringBinary(row));
byte[] regionName = location.getRegionInfo().getRegionName();
server.bulkLoadHFiles(famPaths, regionName);
BulkLoadHFileRequest request =
RequestConverter.buildBulkLoadHFileRequest(famPaths, regionName);
server.bulkLoadHFile(null, request);
return null;
}
}.withRetries();
@ -159,6 +164,8 @@ public class TestHRegionServerBulkLoad {
public Void call() throws Exception {
LOG.debug("compacting " + location + " for row "
+ Bytes.toStringBinary(row));
HRegionInterface server = connection.getHRegionConnection(
location.getHostname(), location.getPort());
server.compactRegion(location.getRegionInfo(), true);
numCompactions.incrementAndGet();
return null;