HBASE-7905 Add passing of optional cell blocks over rpc

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1459013 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2013-03-20 19:36:46 +00:00
parent 75105027a1
commit 5fde90f68f
59 changed files with 7315 additions and 4547 deletions

View File

@ -90,7 +90,6 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.SoftValueSortedMap; import org.apache.hadoop.hbase.util.SoftValueSortedMap;
import org.apache.hadoop.hbase.util.Triple; import org.apache.hadoop.hbase.util.Triple;
import org.apache.hadoop.hbase.zookeeper.*; import org.apache.hadoop.hbase.zookeeper.*;
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
@ -523,7 +522,6 @@ public class HConnectionManager {
// package protected for the tests // package protected for the tests
ClusterStatusListener clusterStatusListener; ClusterStatusListener clusterStatusListener;
private final Object metaRegionLock = new Object();
private final Object userRegionLock = new Object(); private final Object userRegionLock = new Object();
// We have a single lock for master & zk to prevent deadlocks. Having // We have a single lock for master & zk to prevent deadlocks. Having
@ -645,7 +643,7 @@ public class HConnectionManager {
* @return * @return
*/ */
public String toString(){ public String toString(){
return "hconnection 0x" + Integer.toHexString( hashCode() ); return "hconnection-0x" + Integer.toHexString(hashCode());
} }
private String clusterId = null; private String clusterId = null;
@ -1011,13 +1009,16 @@ public class HConnectionManager {
if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) { if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
ZooKeeperKeepAliveConnection zkw = getKeepAliveZooKeeperWatcher(); ZooKeeperKeepAliveConnection zkw = getKeepAliveZooKeeperWatcher();
try { try {
LOG.debug("Looking up meta region location in ZK," + if (LOG.isTraceEnabled()) {
" connection=" + this); LOG.trace("Looking up meta region location in ZK," + " connection=" + this);
}
ServerName servername = ServerName servername =
MetaRegionTracker.blockUntilAvailable(zkw, this.rpcTimeout); MetaRegionTracker.blockUntilAvailable(zkw, this.rpcTimeout);
if (LOG.isTraceEnabled()) {
LOG.debug("Looked up meta region location, connection=" + this + LOG.debug("Looked up meta region location, connection=" + this +
"; serverName=" + ((servername == null) ? "null" : servername)); "; serverName=" + ((servername == null) ? "null" : servername));
}
if (servername == null) return null; if (servername == null) return null;
return new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, servername, 0); return new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, servername, 0);
} catch (InterruptedException e) { } catch (InterruptedException e) {
@ -1821,26 +1822,17 @@ public class HConnectionManager {
} }
@Deprecated @Deprecated
private <R> Callable<MultiResponse> createCallable( private <R> Callable<MultiResponse> createCallable(final HRegionLocation loc,
final HRegionLocation loc, final MultiAction<R> multi, final MultiAction<R> multi, final byte[] tableName) {
final byte [] tableName) {
// TODO: This does not belong in here!!! St.Ack HConnections should // TODO: This does not belong in here!!! St.Ack HConnections should
// not be dealing in Callables; Callables have HConnections, not other // not be dealing in Callables; Callables have HConnections, not other
// way around. // way around.
final HConnection connection = this; final HConnection connection = this;
return new Callable<MultiResponse>() { return new Callable<MultiResponse>() {
public MultiResponse call() throws IOException {
ServerCallable<MultiResponse> callable =
new ServerCallable<MultiResponse>(connection, tableName, null) {
public MultiResponse call() throws IOException {
return ProtobufUtil.multi(server, multi);
}
@Override @Override
public void connect(boolean reload) throws IOException { public MultiResponse call() throws Exception {
server = connection.getClient(loc.getServerName()); ServerCallable<MultiResponse> callable =
} new MultiServerCallable<R>(connection, tableName, loc, multi);
};
return callable.withoutRetries(); return callable.withoutRetries();
} }
}; };
@ -2162,8 +2154,7 @@ public class HConnectionManager {
} else // success } else // success
if (callback != null) { if (callback != null) {
this.callback.update(resultsForRS.getKey(), this.callback.update(resultsForRS.getKey(),
this.rows.get(regionResult.getFirst()).getRow(), this.rows.get(regionResult.getFirst()).getRow(), (R) result);
(R) result);
} }
} }
} }
@ -2222,8 +2213,6 @@ public class HConnectionManager {
} }
} }
/** /**
* Put the action that has to be retried in the Replay list. * Put the action that has to be retried in the Replay list.
* @return true if we're out of numRetries and it's the last retry. * @return true if we're out of numRetries and it's the last retry.

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.HConnectionManager.HConnectable;
import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
import org.apache.hadoop.hbase.ipc.RegionCoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.RegionCoprocessorRpcChannel;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.RequestConverter;
@ -728,9 +729,11 @@ public class HTable implements HTableInterface {
try { try {
MutateRequest request = RequestConverter.buildMutateRequest( MutateRequest request = RequestConverter.buildMutateRequest(
location.getRegionInfo().getRegionName(), append); location.getRegionInfo().getRegionName(), append);
MutateResponse response = server.mutate(null, request); PayloadCarryingRpcController rpcController =
new PayloadCarryingRpcController();
MutateResponse response = server.mutate(rpcController, request);
if (!response.hasResult()) return null; if (!response.hasResult()) return null;
return ProtobufUtil.toResult(response.getResult()); return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
} catch (ServiceException se) { } catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se); throw ProtobufUtil.getRemoteException(se);
} }
@ -752,8 +755,9 @@ public class HTable implements HTableInterface {
try { try {
MutateRequest request = RequestConverter.buildMutateRequest( MutateRequest request = RequestConverter.buildMutateRequest(
location.getRegionInfo().getRegionName(), increment); location.getRegionInfo().getRegionName(), increment);
MutateResponse response = server.mutate(null, request); PayloadCarryingRpcController rpcContoller = new PayloadCarryingRpcController();
return ProtobufUtil.toResult(response.getResult()); MutateResponse response = server.mutate(rpcContoller, request);
return ProtobufUtil.toResult(response.getResult(), rpcContoller.cellScanner());
} catch (ServiceException se) { } catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se); throw ProtobufUtil.getRemoteException(se);
} }
@ -796,8 +800,10 @@ public class HTable implements HTableInterface {
MutateRequest request = RequestConverter.buildMutateRequest( MutateRequest request = RequestConverter.buildMutateRequest(
location.getRegionInfo().getRegionName(), row, family, location.getRegionInfo().getRegionName(), row, family,
qualifier, amount, writeToWAL); qualifier, amount, writeToWAL);
MutateResponse response = server.mutate(null, request); PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
Result result = ProtobufUtil.toResult(response.getResult()); MutateResponse response = server.mutate(rpcController, request);
Result result =
ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier))); return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier)));
} catch (ServiceException se) { } catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se); throw ProtobufUtil.getRemoteException(se);

View File

@ -0,0 +1,120 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
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;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
import com.google.protobuf.ServiceException;
/**
* Callable that handles the <code>multi</code> method call going against a single
* regionserver; i.e. A {@link ServerCallable} for the multi call (It is not a
* {@link Callable} that goes against multiple regions.
* @param <R>
*/
class MultiServerCallable<R> extends ServerCallable<MultiResponse> {
private final MultiAction<R> multi;
private final HRegionLocation loc;
MultiServerCallable(final HConnection connection, final byte [] tableName,
final HRegionLocation loc, final MultiAction<R> multi) {
super(connection, tableName, null);
this.multi = multi;
this.loc = loc;
}
@Override
public MultiResponse call() throws IOException {
MultiResponse response = new MultiResponse();
// The multi object is a list of Actions by region.
for (Map.Entry<byte[], List<Action<R>>> e: this.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();
// Row Mutations are a set of Puts and/or Deletes all to be applied atomically
// on the one row. We do these a row at a time.
if (row instanceof RowMutations) {
try {
RowMutations rms = (RowMutations)row;
// Stick all Cells for all RowMutations in here into 'cells'. Populated when we call
// buildNoDataMultiRequest in the below.
List<CellScannable> cells = new ArrayList<CellScannable>(rms.getMutations().size());
// Build a multi request absent its Cell payload (this is the 'nodata' in the below).
MultiRequest multiRequest =
RequestConverter.buildNoDataMultiRequest(regionName, rms, cells);
// Carry the cells over the proxy/pb Service interface using the payload carrying
// rpc controller.
server.multi(new PayloadCarryingRpcController(cells), multiRequest);
// This multi call does not return results.
response.add(regionName, action.getOriginalIndex(), Result.EMPTY_RESULT);
} catch (ServiceException se) {
response.add(regionName, action.getOriginalIndex(),
ProtobufUtil.getRemoteException(se));
}
rowMutations++;
}
}
// Are there any non-RowMutation actions to send for this region?
if (actions.size() > rowMutations) {
Exception ex = null;
List<Object> results = null;
// Stick all Cells for the multiRequest in here into 'cells'. Gets filled in when we
// call buildNoDataMultiRequest
List<CellScannable> cells = new ArrayList<CellScannable>(actions.size() - rowMutations);
try {
// The call to buildNoDataMultiRequest will skip RowMutations. They have
// already been handled above.
MultiRequest multiRequest =
RequestConverter.buildNoDataMultiRequest(regionName, actions, cells);
// Controller optionally carries cell data over the proxy/service boundary and also
// optionally ferries cell response data back out again.
PayloadCarryingRpcController controller = new PayloadCarryingRpcController(cells);
ClientProtos.MultiResponse responseProto = server.multi(controller, multiRequest);
results = ResponseConverter.getResults(responseProto, controller.cellScanner());
} catch (ServiceException se) {
ex = ProtobufUtil.getRemoteException(se);
}
for (int i = 0, n = actions.size(); i < n; i++) {
int originalIndex = actions.get(i).getOriginalIndex();
response.add(regionName, originalIndex, results == null ? ex : results.get(i));
}
}
}
return response;
}
@Override
public void connect(boolean reload) throws IOException {
server = connection.getClient(loc.getServerName());
}
}

View File

@ -19,13 +19,11 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
/** /**
@ -52,10 +50,8 @@ public class RowMutations implements Row {
* @param row row key * @param row row key
*/ */
public RowMutations(byte [] row) { public RowMutations(byte [] row) {
if(row == null || row.length > HConstants.MAX_ROW_LENGTH) { Mutation.checkRow(row);
throw new IllegalArgumentException("Row key is invalid"); this.row = Bytes.copy(row);
}
this.row = Arrays.copyOf(row, row.length);
} }
/** /**
@ -79,9 +75,9 @@ public class RowMutations implements Row {
private void internalAdd(Mutation m) throws IOException { private void internalAdd(Mutation m) throws IOException {
int res = Bytes.compareTo(this.row, m.getRow()); int res = Bytes.compareTo(this.row, m.getRow());
if (res != 0) { if (res != 0) {
throw new IOException("The row in the recently added Put/Delete " + throw new WrongRowIOException("The row in the recently added Put/Delete <" +
Bytes.toStringBinary(m.getRow()) + " doesn't match the original one " + Bytes.toStringBinary(m.getRow()) + "> doesn't match the original one <" +
Bytes.toStringBinary(this.row)); Bytes.toStringBinary(this.row) + ">");
} }
mutations.add(m); mutations.add(m);
} }

View File

@ -0,0 +1,42 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
/**
* Thrown when server finds fatal issue w/ connection setup: e.g. bad rpc version
* or unsupported auth method.
* Closes connection after throwing this exception with message on why the failure.
*/
@SuppressWarnings("serial")
@InterfaceAudience.Private
public class FatalConnectionException extends DoNotRetryIOException {
public FatalConnectionException() {
super();
}
public FatalConnectionException(String msg) {
super(msg);
}
public FatalConnectionException(String msg, Throwable t) {
super(msg, t);
}
}

View File

@ -19,24 +19,50 @@
package org.apache.hadoop.hbase.ipc; package org.apache.hadoop.hbase.ipc;
import com.google.protobuf.CodedOutputStream; import java.io.BufferedInputStream;
import com.google.protobuf.Message; import java.io.BufferedOutputStream;
import com.google.protobuf.Message.Builder; import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.FilterInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.lang.reflect.Method;
import java.net.ConnectException;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.SocketException;
import java.net.SocketTimeoutException;
import java.net.UnknownHostException;
import java.nio.ByteBuffer;
import java.security.PrivilegedExceptionAction;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Random;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import javax.net.SocketFactory;
import javax.security.sasl.SaslException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.IpcProtocol; import org.apache.hadoop.hbase.IpcProtocol;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.codec.Codec;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos; import org.apache.hadoop.hbase.codec.KeyValueCodec;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcException; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseBody;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcResponseHeader.Status;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
import org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo; import org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo;
import org.apache.hadoop.hbase.security.AuthMethod; import org.apache.hadoop.hbase.security.AuthMethod;
@ -52,6 +78,7 @@ import org.apache.hadoop.hbase.util.PoolMap;
import org.apache.hadoop.hbase.util.PoolMap.PoolType; import org.apache.hadoop.hbase.util.PoolMap.PoolType;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.SecurityUtil;
@ -62,54 +89,23 @@ import org.apache.hadoop.security.token.TokenSelector;
import org.cloudera.htrace.Span; import org.cloudera.htrace.Span;
import org.cloudera.htrace.Trace; import org.cloudera.htrace.Trace;
import javax.net.SocketFactory; import com.google.protobuf.Message;
import javax.security.sasl.SaslException; import com.google.protobuf.Message.Builder;
import java.io.BufferedInputStream; import com.google.protobuf.TextFormat;
import java.io.BufferedOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.EOFException;
import java.io.FilterInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.lang.reflect.Method;
import java.net.ConnectException;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.SocketException;
import java.net.SocketTimeoutException;
import java.net.UnknownHostException;
import java.security.PrivilegedExceptionAction;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Random;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
/** A client for an IPC service. IPC calls take a single Protobuf message as a /**
* parameter, and return a single Protobuf message as their value. A service runs on * A client for an IPC service. IPC calls take a single Protobuf message as a
* request and returns a single Protobuf message as result. A service runs on
* a port and is defined by a parameter class and a value class. * a port and is defined by a parameter class and a value class.
* *
* <p>This is the org.apache.hadoop.ipc.Client renamed as HBaseClient and * <p>See HBaseServer
* moved into this package so can access package-private methods.
*
* See HBaseServer
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class HBaseClient { public class HBaseClient {
public static final Log LOG = LogFactory.getLog("org.apache.hadoop.ipc.HBaseClient");
public static final Log LOG =
LogFactory.getLog("org.apache.hadoop.ipc.HBaseClient");
protected final PoolMap<ConnectionId, Connection> connections; protected final PoolMap<ConnectionId, Connection> connections;
private static final Map<String, Method> methodInstances = private ReflectionCache reflectionCache = new ReflectionCache();
new ConcurrentHashMap<String, Method>();
protected int counter; // counter for call ids protected int counter; // counter for call ids
protected final AtomicBoolean running = new AtomicBoolean(true); // if client runs protected final AtomicBoolean running = new AtomicBoolean(true); // if client runs
@ -123,6 +119,9 @@ public class HBaseClient {
protected int pingInterval; // how often sends ping to the server in msecs protected int pingInterval; // how often sends ping to the server in msecs
protected int socketTimeout; // socket timeout protected int socketTimeout; // socket timeout
protected FailedServers failedServers; protected FailedServers failedServers;
private final Codec codec;
private final CompressionCodec compressor;
private final IPCUtil ipcUtil;
protected final SocketFactory socketFactory; // how to create sockets protected final SocketFactory socketFactory; // how to create sockets
protected String clusterId; protected String clusterId;
@ -187,9 +186,8 @@ public class HBaseClient {
} }
} }
@SuppressWarnings("serial")
public static class FailedServerException extends IOException { public static class FailedServerException extends IOException {
private static final long serialVersionUID = -4744376109431464127L;
public FailedServerException(String s) { public FailedServerException(String s) {
super(s); super(s);
} }
@ -201,6 +199,8 @@ public class HBaseClient {
* @param conf Configuration * @param conf Configuration
* @param pingInterval the ping interval * @param pingInterval the ping interval
*/ */
// Any reason we couldn't just do tcp keepalive instead of this pingery?
// St.Ack 20130121
public static void setPingInterval(Configuration conf, int pingInterval) { public static void setPingInterval(Configuration conf, int pingInterval) {
conf.setInt(PING_INTERVAL_NAME, pingInterval); conf.setInt(PING_INTERVAL_NAME, pingInterval);
} }
@ -235,20 +235,34 @@ public class HBaseClient {
/** A call waiting for a value. */ /** A call waiting for a value. */
protected class Call { protected class Call {
final int id; // call id final int id; // call id
final RpcRequestBody param; // rpc request object final Message param; // rpc request method param object
Message value; // value, null if error /**
* Optionally has cells when making call. Optionally has cells set on response. Used
* passing cells to the rpc and receiving the response.
*/
CellScanner cells;
Message response; // value, null if error
IOException error; // exception, null if value IOException error; // exception, null if value
boolean done; // true when call is done boolean done; // true when call is done
long startTime; long startTime;
final Method method;
protected Call(RpcRequestBody param) { protected Call(final Method method, Message param, final CellScanner cells) {
this.param = param; this.param = param;
this.method = method;
this.cells = cells;
this.startTime = System.currentTimeMillis(); this.startTime = System.currentTimeMillis();
synchronized (HBaseClient.this) { synchronized (HBaseClient.this) {
this.id = counter++; this.id = counter++;
} }
} }
@Override
public String toString() {
return "callId: " + this.id + " methodName: " + this.method.getName() + " param {" +
(this.param != null? TextFormat.shortDebugString(this.param): "") + "}";
}
/** Indicate when the call is complete and the /** Indicate when the call is complete and the
* value or error are available. Notifies by default. */ * value or error are available. Notifies by default. */
protected synchronized void callComplete() { protected synchronized void callComplete() {
@ -269,10 +283,12 @@ public class HBaseClient {
/** Set the return value when there is no error. /** Set the return value when there is no error.
* Notify the caller the call is done. * Notify the caller the call is done.
* *
* @param value return value of the call. * @param response return value of the call.
* @param cells Can be null
*/ */
public synchronized void setValue(Message value) { public synchronized void setResponse(Message response, final CellScanner cells) {
this.value = value; this.response = response;
this.cells = cells;
callComplete(); callComplete();
} }
@ -281,7 +297,7 @@ public class HBaseClient {
} }
} }
protected static Map<String,TokenSelector<? extends TokenIdentifier>> tokenHandlers = protected final static Map<String,TokenSelector<? extends TokenIdentifier>> tokenHandlers =
new HashMap<String,TokenSelector<? extends TokenIdentifier>>(); new HashMap<String,TokenSelector<? extends TokenIdentifier>>();
static { static {
tokenHandlers.put(AuthenticationTokenIdentifier.AUTH_TOKEN_TYPE.toString(), tokenHandlers.put(AuthenticationTokenIdentifier.AUTH_TOKEN_TYPE.toString(),
@ -292,8 +308,10 @@ public class HBaseClient {
* Creates a connection. Can be overridden by a subclass for testing. * Creates a connection. Can be overridden by a subclass for testing.
* @param remoteId - the ConnectionId to use for the connection creation. * @param remoteId - the ConnectionId to use for the connection creation.
*/ */
protected Connection createConnection(ConnectionId remoteId) throws IOException { protected Connection createConnection(ConnectionId remoteId, final Codec codec,
return new Connection(remoteId); final CompressionCodec compressor)
throws IOException {
return new Connection(remoteId, codec, compressor);
} }
/** Thread that reads responses and notifies callers. Each connection owns a /** Thread that reads responses and notifies callers. Each connection owns a
@ -312,6 +330,8 @@ public class HBaseClient {
private Token<? extends TokenIdentifier> token; private Token<? extends TokenIdentifier> token;
private HBaseSaslRpcClient saslRpcClient; private HBaseSaslRpcClient saslRpcClient;
private int reloginMaxBackoff; // max pause before relogin on sasl failure private int reloginMaxBackoff; // max pause before relogin on sasl failure
private final Codec codec;
private final CompressionCodec compressor;
// currently active calls // currently active calls
protected final ConcurrentSkipListMap<Integer, Call> calls = protected final ConcurrentSkipListMap<Integer, Call> calls =
@ -322,12 +342,14 @@ public class HBaseClient {
new AtomicBoolean(); // indicate if the connection is closed new AtomicBoolean(); // indicate if the connection is closed
protected IOException closeException; // close reason protected IOException closeException; // close reason
Connection(ConnectionId remoteId) throws IOException { Connection(ConnectionId remoteId, final Codec codec, final CompressionCodec compressor)
throws IOException {
if (remoteId.getAddress().isUnresolved()) { if (remoteId.getAddress().isUnresolved()) {
throw new UnknownHostException("unknown host: " + throw new UnknownHostException("unknown host: " + remoteId.getAddress().getHostName());
remoteId.getAddress().getHostName());
} }
this.server = remoteId.getAddress(); this.server = remoteId.getAddress();
this.codec = codec;
this.compressor = compressor;
UserGroupInformation ticket = remoteId.getTicket().getUGI(); UserGroupInformation ticket = remoteId.getTicket().getUGI();
Class<?> protocol = remoteId.getProtocol(); Class<?> protocol = remoteId.getProtocol();
@ -368,19 +390,23 @@ public class HBaseClient {
authMethod = AuthMethod.KERBEROS; authMethod = AuthMethod.KERBEROS;
} }
if (LOG.isDebugEnabled()) if (LOG.isDebugEnabled()) {
LOG.debug("Use " + authMethod + " authentication for protocol " LOG.debug("Use " + authMethod + " authentication for protocol "
+ protocol.getSimpleName()); + protocol.getSimpleName());
}
reloginMaxBackoff = conf.getInt("hbase.security.relogin.maxbackoff", 5000); reloginMaxBackoff = conf.getInt("hbase.security.relogin.maxbackoff", 5000);
this.remoteId = remoteId; this.remoteId = remoteId;
ConnectionHeader.Builder builder = ConnectionHeader.newBuilder(); ConnectionHeader.Builder builder = ConnectionHeader.newBuilder();
builder.setProtocol(protocol == null ? "" : protocol.getName()); builder.setProtocol(protocol == null ? "" : protocol.getName());
UserInformation userInfoPB; UserInformation userInfoPB;
if ((userInfoPB = getUserInfoPB(ticket)) != null) { if ((userInfoPB = getUserInfo(ticket)) != null) {
builder.setUserInfo(userInfoPB); builder.setUserInfo(userInfoPB);
} }
builder.setCellBlockCodecClass(this.codec.getClass().getCanonicalName());
if (this.compressor != null) {
builder.setCellBlockCompressorClass(this.compressor.getClass().getCanonicalName());
}
this.header = builder.build(); this.header = builder.build();
this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " + this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " +
@ -390,7 +416,7 @@ public class HBaseClient {
this.setDaemon(true); this.setDaemon(true);
} }
private UserInformation getUserInfoPB(UserGroupInformation ugi) { private UserInformation getUserInfo(UserGroupInformation ugi) {
if (ugi == null || authMethod == AuthMethod.DIGEST) { if (ugi == null || authMethod == AuthMethod.DIGEST) {
// Don't send user for token auth // Don't send user for token auth
return null; return null;
@ -582,8 +608,7 @@ public class HBaseClient {
*/ */
protected synchronized boolean waitForWork() { protected synchronized boolean waitForWork() {
if (calls.isEmpty() && !shouldCloseConnection.get() && running.get()) { if (calls.isEmpty() && !shouldCloseConnection.get() && running.get()) {
long timeout = maxIdleTime- long timeout = maxIdleTime - (System.currentTimeMillis()-lastActivity.get());
(System.currentTimeMillis()-lastActivity.get());
if (timeout>0) { if (timeout>0) {
try { try {
wait(timeout); wait(timeout);
@ -613,6 +638,7 @@ public class HBaseClient {
* since last I/O activity is equal to or greater than the ping interval * since last I/O activity is equal to or greater than the ping interval
*/ */
protected synchronized void sendPing() throws IOException { protected synchronized void sendPing() throws IOException {
// Can we do tcp keepalive instead of this pinging?
long curTime = System.currentTimeMillis(); long curTime = System.currentTimeMillis();
if ( curTime - lastActivity.get() >= pingInterval) { if ( curTime - lastActivity.get() >= pingInterval) {
lastActivity.set(curTime); lastActivity.set(curTime);
@ -626,24 +652,23 @@ public class HBaseClient {
@Override @Override
public void run() { public void run() {
if (LOG.isDebugEnabled()) if (LOG.isDebugEnabled()) {
LOG.debug(getName() + ": starting, having connections " LOG.debug(getName() + ": starting, connections " + connections.size());
+ connections.size()); }
try { try {
while (waitForWork()) {//wait here for work - read or close connection while (waitForWork()) {//wait here for work - read or close connection
receiveResponse(); readResponse();
} }
} catch (Throwable t) { } catch (Throwable t) {
LOG.warn("Unexpected exception receiving call responses", t); LOG.warn(getName() + ": unexpected exception receiving call responses", t);
markClosed(new IOException("Unexpected exception receiving call responses", t)); markClosed(new IOException("Unexpected exception receiving call responses", t));
} }
close(); close();
if (LOG.isDebugEnabled()) if (LOG.isDebugEnabled())
LOG.debug(getName() + ": stopped, remaining connections " LOG.debug(getName() + ": stopped, connections " + connections.size());
+ connections.size());
} }
private synchronized void disposeSasl() { private synchronized void disposeSasl() {
@ -775,7 +800,8 @@ public class HBaseClient {
setupConnection(); setupConnection();
InputStream inStream = NetUtils.getInputStream(socket); InputStream inStream = NetUtils.getInputStream(socket);
OutputStream outStream = NetUtils.getOutputStream(socket); OutputStream outStream = NetUtils.getOutputStream(socket);
writeRpcHeader(outStream); // Write out the preamble -- MAGIC, version, and auth to use.
writeConnectionHeaderPreamble(outStream);
if (useSasl) { if (useSasl) {
final InputStream in2 = inStream; final InputStream in2 = inStream;
final OutputStream out2 = outStream; final OutputStream out2 = outStream;
@ -787,6 +813,9 @@ public class HBaseClient {
} }
boolean continueSasl = false; boolean continueSasl = false;
try { try {
if (ticket == null) {
throw new NullPointerException("ticket is null");
} else {
continueSasl = continueSasl =
ticket.doAs(new PrivilegedExceptionAction<Boolean>() { ticket.doAs(new PrivilegedExceptionAction<Boolean>() {
@Override @Override
@ -794,12 +823,12 @@ public class HBaseClient {
return setupSaslConnection(in2, out2); return setupSaslConnection(in2, out2);
} }
}); });
}
} catch (Exception ex) { } catch (Exception ex) {
if (rand == null) { if (rand == null) {
rand = new Random(); rand = new Random();
} }
handleSaslConnectionFailure(numRetries++, MAX_RETRIES, ex, rand, handleSaslConnectionFailure(numRetries++, MAX_RETRIES, ex, rand, ticket);
ticket);
continue; continue;
} }
if (continueSasl) { if (continueSasl) {
@ -812,11 +841,10 @@ public class HBaseClient {
useSasl = false; useSasl = false;
} }
} }
this.in = new DataInputStream(new BufferedInputStream this.in = new DataInputStream(new BufferedInputStream(new PingInputStream(inStream)));
(new PingInputStream(inStream))); this.out = new DataOutputStream(new BufferedOutputStream(outStream));
this.out = new DataOutputStream // Now write out the connection header
(new BufferedOutputStream(outStream)); writeConnectionHeader();
writeHeader();
// update last activity time // update last activity time
touch(); touch();
@ -840,30 +868,38 @@ public class HBaseClient {
} }
} }
/* Write the RPC header */ /**
private void writeRpcHeader(OutputStream outStream) throws IOException { * Write the RPC header: <MAGIC WORD -- 'HBas'> <ONEBYTE_VERSION> <ONEBYTE_AUTH_TYPE>
DataOutputStream out = new DataOutputStream(new BufferedOutputStream(outStream)); */
// Write out the header, version and authentication method private void writeConnectionHeaderPreamble(OutputStream outStream) throws IOException {
out.write(HConstants.RPC_HEADER.array()); // Assemble the preamble up in a buffer first and then send it. Writing individual elements,
out.write(HConstants.CURRENT_VERSION); // they are getting sent across piecemeal according to wireshark and then server is messing
authMethod.write(out); // up the reading on occasion (the passed in stream is not buffered yet).
out.flush();
// Preamble is six bytes -- 'HBas' + VERSION + AUTH_CODE
int rpcHeaderLen = HConstants.RPC_HEADER.array().length;
byte [] preamble = new byte [rpcHeaderLen + 2];
System.arraycopy(HConstants.RPC_HEADER.array(), 0, preamble, 0, rpcHeaderLen);
preamble[rpcHeaderLen] = HConstants.RPC_CURRENT_VERSION;
preamble[rpcHeaderLen + 1] = authMethod.code;
outStream.write(preamble);
outStream.flush();
} }
/** /**
* Write the protocol header for each connection * Write the connection header.
* Out is not synchronized because only the first thread does this. * Out is not synchronized because only the first thread does this.
*/ */
private void writeHeader() throws IOException { private void writeConnectionHeader() throws IOException {
// Write out the ConnectionHeader this.out.writeInt(this.header.getSerializedSize());
out.writeInt(header.getSerializedSize()); this.header.writeTo(this.out);
header.writeTo(out); this.out.flush();
} }
/** Close the connection. */ /** Close the connection. */
protected synchronized void close() { protected synchronized void close() {
if (!shouldCloseConnection.get()) { if (!shouldCloseConnection.get()) {
LOG.error("The connection is not in the closed state"); LOG.error(getName() + ": the connection is not in the closed state");
return; return;
} }
@ -883,8 +919,7 @@ public class HBaseClient {
// clean up all calls // clean up all calls
if (closeException == null) { if (closeException == null) {
if (!calls.isEmpty()) { if (!calls.isEmpty()) {
LOG.warn( LOG.warn(getName() + ": connection is closed for no cause and calls are not empty. " +
"A connection is closed for no cause and calls are not empty. " +
"#Calls: " + calls.size()); "#Calls: " + calls.size());
// clean up calls anyway // clean up calls anyway
@ -894,7 +929,7 @@ public class HBaseClient {
} else { } else {
// log the info // log the info
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("closing ipc connection to " + server + ": " + LOG.debug(getName() + ": closing ipc connection to " + server + ": " +
closeException.getMessage(),closeException); closeException.getMessage(),closeException);
} }
@ -905,126 +940,100 @@ public class HBaseClient {
LOG.debug(getName() + ": closed"); LOG.debug(getName() + ": closed");
} }
/* Initiates a call by sending the parameter to the remote server. /**
* Initiates a call by sending the parameter to the remote server.
* Note: this is not called from the Connection thread, but by other * Note: this is not called from the Connection thread, but by other
* threads. * threads.
* @param call
* @see #readResponse()
*/ */
protected void sendParam(Call call) { protected void writeRequest(Call call) {
if (shouldCloseConnection.get()) { if (shouldCloseConnection.get()) return;
return;
}
try { try {
if (LOG.isDebugEnabled()) RequestHeader.Builder builder = RequestHeader.newBuilder();
LOG.debug(getName() + " sending #" + call.id); builder.setCallId(call.id);
RpcRequestHeader.Builder headerBuilder = RPCProtos.RpcRequestHeader.newBuilder();
headerBuilder.setCallId(call.id);
if (Trace.isTracing()) { if (Trace.isTracing()) {
Span s = Trace.currentTrace(); Span s = Trace.currentTrace();
headerBuilder.setTinfo(RPCTInfo.newBuilder() builder.setTraceInfo(RPCTInfo.newBuilder().
.setParentId(s.getSpanId()) setParentId(s.getSpanId()).setTraceId(s.getTraceId()));
.setTraceId(s.getTraceId())); }
builder.setMethodName(call.method.getName());
builder.setRequestParam(call.param != null);
ByteBuffer cellBlock = ipcUtil.buildCellBlock(this.codec, this.compressor, call.cells);
if (cellBlock != null) {
CellBlockMeta.Builder cellBlockBuilder = CellBlockMeta.newBuilder();
cellBlockBuilder.setLength(cellBlock.limit());
builder.setCellBlockMeta(cellBlockBuilder.build());
} }
//noinspection SynchronizeOnNonFinalField //noinspection SynchronizeOnNonFinalField
RequestHeader header = builder.build();
synchronized (this.out) { // FindBugs IS2_INCONSISTENT_SYNC synchronized (this.out) { // FindBugs IS2_INCONSISTENT_SYNC
RpcRequestHeader header = headerBuilder.build(); IPCUtil.write(this.out, header, call.param, cellBlock);
int serializedHeaderSize = header.getSerializedSize(); }
int requestSerializedSize = call.param.getSerializedSize(); if (LOG.isDebugEnabled()) {
this.out.writeInt(serializedHeaderSize + LOG.debug(getName() + ": wrote request header " + TextFormat.shortDebugString(header));
CodedOutputStream.computeRawVarint32Size(serializedHeaderSize) +
requestSerializedSize +
CodedOutputStream.computeRawVarint32Size(requestSerializedSize));
header.writeDelimitedTo(this.out);
call.param.writeDelimitedTo(this.out);
this.out.flush();
} }
} catch(IOException e) { } catch(IOException e) {
markClosed(e); markClosed(e);
} }
} }
private Method getMethod(Class<? extends IpcProtocol> protocol,
String methodName) {
Method method = methodInstances.get(methodName);
if (method != null) {
return method;
}
Method[] methods = protocol.getMethods();
for (Method m : methods) {
if (m.getName().equals(methodName)) {
m.setAccessible(true);
methodInstances.put(methodName, m);
return m;
}
}
return null;
}
/* Receive a response. /* Receive a response.
* Because only one receiver, so no synchronization on in. * Because only one receiver, so no synchronization on in.
*/ */
protected void receiveResponse() { protected void readResponse() {
if (shouldCloseConnection.get()) { if (shouldCloseConnection.get()) return;
return;
}
touch(); touch();
try { try {
// See HBaseServer.Call.setResponse for where we write out the response. // See HBaseServer.Call.setResponse for where we write out the response.
// It writes the call.id (int), a boolean signifying any error (and if
// so the exception name/trace), and the response bytes
// Read the call id. // Total size of the response. Unused. But have to read it in anyways.
RpcResponseHeader response = RpcResponseHeader.parseDelimitedFrom(in); /*int totalSize =*/ in.readInt();
if (response == null) {
// When the stream is closed, protobuf doesn't raise an EOFException, // Read the header
// instead, it returns a null message object. ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in);
throw new EOFException(); int id = responseHeader.getCallId();
if (LOG.isDebugEnabled()) {
LOG.debug(getName() + ": got response header " +
TextFormat.shortDebugString(responseHeader));
} }
int id = response.getCallId();
if (LOG.isDebugEnabled())
LOG.debug(getName() + " got value #" + id);
Call call = calls.get(id); Call call = calls.get(id);
if (responseHeader.hasException()) {
Status status = response.getStatus(); ExceptionResponse exceptionResponse = responseHeader.getException();
if (status == Status.SUCCESS) { RemoteException re = createRemoteException(exceptionResponse);
if (isFatalConnectionException(exceptionResponse)) {
markClosed(re);
} else {
if (call != null) call.setException(re);
}
} else {
Message rpcResponseType; Message rpcResponseType;
try { try {
rpcResponseType = ProtobufRpcClientEngine.Invoker.getReturnProtoType( // TODO: Why pb engine pollution in here in this class? FIX.
getMethod(remoteId.getProtocol(), rpcResponseType =
call.param.getMethodName())); ProtobufRpcClientEngine.Invoker.getReturnProtoType(
reflectionCache.getMethod(remoteId.getProtocol(), call.method.getName()));
} catch (Exception e) { } catch (Exception e) {
throw new RuntimeException(e); //local exception throw new RuntimeException(e); //local exception
} }
Message value = null;
if (rpcResponseType != null) {
Builder builder = rpcResponseType.newBuilderForType(); Builder builder = rpcResponseType.newBuilderForType();
builder.mergeDelimitedFrom(in); builder.mergeDelimitedFrom(in);
Message value = builder.build(); value = builder.build();
}
CellScanner cellBlockScanner = null;
if (responseHeader.hasCellBlockMeta()) {
int size = responseHeader.getCellBlockMeta().getLength();
byte [] cellBlock = new byte[size];
IOUtils.readFully(this.in, cellBlock, 0, cellBlock.length);
cellBlockScanner = ipcUtil.createCellScanner(this.codec, this.compressor, cellBlock);
}
// it's possible that this call may have been cleaned up due to a RPC // it's possible that this call may have been cleaned up due to a RPC
// timeout, so check if it still exists before setting the value. // timeout, so check if it still exists before setting the value.
if (call != null) { if (call != null) call.setResponse(value, cellBlockScanner);
call.setValue(value);
}
calls.remove(id);
} else if (status == Status.ERROR) {
RpcException exceptionResponse = RpcException.parseDelimitedFrom(in);
if (call != null) {
//noinspection ThrowableInstanceNeverThrown
call.setException(new RemoteException(
exceptionResponse.getExceptionName(),
exceptionResponse.getStackTrace()));
calls.remove(id);
}
} else if (status == Status.FATAL) {
RpcException exceptionResponse = RpcException.parseDelimitedFrom(in);
// Close the connection
markClosed(new RemoteException(
exceptionResponse.getExceptionName(),
exceptionResponse.getStackTrace()));
} }
if (call != null) calls.remove(id);
} catch (IOException e) { } catch (IOException e) {
if (e instanceof SocketTimeoutException && remoteId.rpcTimeout > 0) { if (e instanceof SocketTimeoutException && remoteId.rpcTimeout > 0) {
// Clean up open calls but don't treat this as a fatal condition, // Clean up open calls but don't treat this as a fatal condition,
@ -1043,6 +1052,30 @@ public class HBaseClient {
} }
} }
/**
* @param e
* @return True if the exception is a fatal connection exception.
*/
private boolean isFatalConnectionException(final ExceptionResponse e) {
return e.getExceptionClassName().
equals(FatalConnectionException.class.getName());
}
/**
* @param e
* @return RemoteException made from passed <code>e</code>
*/
private RemoteException createRemoteException(final ExceptionResponse e) {
String innerExceptionClassName = e.getExceptionClassName();
boolean doNotRetry = e.getDoNotRetry();
return e.hasHostname()?
// If a hostname then add it to the RemoteWithExtrasException
new RemoteWithExtrasException(innerExceptionClassName,
e.getStackTrace(), e.getHostname(), e.getPort(), doNotRetry):
new RemoteWithExtrasException(innerExceptionClassName,
e.getStackTrace(), doNotRetry);
}
protected synchronized void markClosed(IOException e) { protected synchronized void markClosed(IOException e) {
if (shouldCloseConnection.compareAndSet(false, true)) { if (shouldCloseConnection.compareAndSet(false, true)) {
closeException = e; closeException = e;
@ -1103,53 +1136,13 @@ public class HBaseClient {
/** /**
* Client-side call timeout * Client-side call timeout
*/ */
@SuppressWarnings("serial")
public static class CallTimeoutException extends IOException { public static class CallTimeoutException extends IOException {
public CallTimeoutException(final String msg) { public CallTimeoutException(final String msg) {
super(msg); super(msg);
} }
} }
/** Call implementation used for parallel calls. */
protected class ParallelCall extends Call {
private final ParallelResults results;
protected final int index;
public ParallelCall(RpcRequestBody param, ParallelResults results, int index) {
super(param);
this.results = results;
this.index = index;
}
/** Deliver result to result collector. */
@Override
protected void callComplete() {
results.callComplete(this);
}
}
/** Result collector for parallel calls. */
protected static class ParallelResults {
protected final Message[] values;
protected int size;
protected int count;
public ParallelResults(int size) {
this.values = new RpcResponseBody[size];
this.size = size;
}
/*
* Collect a result.
*/
synchronized void callComplete(ParallelCall call) {
// FindBugs IS2_INCONSISTENT_SYNC
values[call.index] = call.value; // store the value
count++; // count it
if (count == size) // if all values are in
notify(); // then notify waiting caller
}
}
/** /**
* Construct an IPC client whose values are of the {@link Message} * Construct an IPC client whose values are of the {@link Message}
* class. * class.
@ -1165,9 +1158,12 @@ public class HBaseClient {
this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true); this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true);
this.pingInterval = getPingInterval(conf); this.pingInterval = getPingInterval(conf);
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("The ping interval is" + this.pingInterval + "ms."); LOG.debug("Ping interval: " + this.pingInterval + "ms.");
} }
this.ipcUtil = new IPCUtil(conf);
this.conf = conf; this.conf = conf;
this.codec = getCodec(conf);
this.compressor = getCompressor(conf);
this.socketFactory = factory; this.socketFactory = factory;
this.clusterId = clusterId != null ? clusterId : HConstants.CLUSTER_ID_DEFAULT; this.clusterId = clusterId != null ? clusterId : HConstants.CLUSTER_ID_DEFAULT;
this.connections = new PoolMap<ConnectionId, Connection>( this.connections = new PoolMap<ConnectionId, Connection>(
@ -1175,6 +1171,35 @@ public class HBaseClient {
this.failedServers = new FailedServers(conf); this.failedServers = new FailedServers(conf);
} }
/**
* Encapsulate the ugly casting and RuntimeException conversion in private method.
* @param conf
* @return Codec to use on this client.
*/
private static Codec getCodec(final Configuration conf) {
String className = conf.get("hbase.client.rpc.codec", KeyValueCodec.class.getCanonicalName());
try {
return (Codec)Class.forName(className).newInstance();
} catch (Exception e) {
throw new RuntimeException("Failed getting codec " + className, e);
}
}
/**
* Encapsulate the ugly casting and RuntimeException conversion in private method.
* @param conf
* @return The compressor to use on this client.
*/
private static CompressionCodec getCompressor(final Configuration conf) {
String className = conf.get("hbase.client.rpc.compressor", null);
if (className == null || className.isEmpty()) return null;
try {
return (CompressionCodec)Class.forName(className).newInstance();
} catch (Exception e) {
throw new RuntimeException("Failed getting compressor " + className, e);
}
}
/** /**
* Construct an IPC client with the default SocketFactory * Construct an IPC client with the default SocketFactory
* @param conf configuration * @param conf configuration
@ -1249,37 +1274,31 @@ public class HBaseClient {
} }
} }
/** Make a call, passing <code>param</code>, to the IPC server running at
* <code>address</code>, returning the value. Throws exceptions if there are
* network problems or if the remote code threw an exception.
* @param param RpcRequestBody parameter
* @param address network address
* @return Message
* @throws IOException e
*/
public Message call(RpcRequestBody param, InetSocketAddress address)
throws IOException, InterruptedException {
return call(param, address, null, 0);
}
public Message call(RpcRequestBody param, InetSocketAddress addr,
User ticket, int rpcTimeout)
throws IOException, InterruptedException {
return call(param, addr, null, ticket, rpcTimeout);
}
/** Make a call, passing <code>param</code>, to the IPC server running at /** Make a call, passing <code>param</code>, to the IPC server running at
* <code>address</code> which is servicing the <code>protocol</code> protocol, * <code>address</code> which is servicing the <code>protocol</code> protocol,
* with the <code>ticket</code> credentials, returning the value. * with the <code>ticket</code> credentials, returning the value.
* Throws exceptions if there are network problems or if the remote code * Throws exceptions if there are network problems or if the remote code
* threw an exception. */ * threw an exception.
public Message call(RpcRequestBody param, InetSocketAddress addr, * @param method
Class<? extends IpcProtocol> protocol, * @param param
User ticket, int rpcTimeout) * @param cells
* @param addr
* @param protocol
* @param ticket Be careful which ticket you pass. A new user will mean a new Connection.
* {@link User#getCurrent()} makes a new instance of User each time so will be a new Connection
* each time.
* @param rpcTimeout
* @return A pair with the Message response and the Cell data (if any).
* @throws InterruptedException
* @throws IOException
*/
public Pair<Message, CellScanner> call(Method method, Message param, CellScanner cells,
InetSocketAddress addr, Class<? extends IpcProtocol> protocol, User ticket, int rpcTimeout)
throws InterruptedException, IOException { throws InterruptedException, IOException {
Call call = new Call(param); Call call = new Call(method, param, cells);
Connection connection = getConnection(addr, protocol, ticket, rpcTimeout, call); Connection connection =
connection.sendParam(call); // send the parameter getConnection(addr, protocol, ticket, rpcTimeout, call, this.codec, this.compressor);
connection.writeRequest(call); // send the parameter
boolean interrupted = false; boolean interrupted = false;
//noinspection SynchronizationOnLocalVariableOrMethodParameter //noinspection SynchronizationOnLocalVariableOrMethodParameter
synchronized (call) { synchronized (call) {
@ -1305,7 +1324,7 @@ public class HBaseClient {
// local exception // local exception
throw wrapException(addr, call.error); throw wrapException(addr, call.error);
} }
return call.value; return new Pair<Message, CellScanner>(call.response, call.cells);
} }
} }
@ -1329,14 +1348,11 @@ public class HBaseClient {
"Call to " + addr + " failed on connection exception: " + exception) "Call to " + addr + " failed on connection exception: " + exception)
.initCause(exception); .initCause(exception);
} else if (exception instanceof SocketTimeoutException) { } else if (exception instanceof SocketTimeoutException) {
return (SocketTimeoutException)new SocketTimeoutException( return (SocketTimeoutException)new SocketTimeoutException("Call to " + addr +
"Call to " + addr + " failed on socket timeout exception: " " failed on socket timeout exception: " + exception).initCause(exception);
+ exception).initCause(exception);
} else { } else {
return (IOException)new IOException( return (IOException)new IOException("Call to " + addr + " failed on local exception: " +
"Call to " + addr + " failed on local exception: " + exception) exception).initCause(exception);
.initCause(exception);
} }
} }
@ -1364,50 +1380,10 @@ public class HBaseClient {
} }
} }
/** Makes a set of calls in parallel. Each parameter is sent to the
* corresponding address. When all values are available, or have timed out
* or errored, the collected results are returned in an array. The array
* contains nulls for calls that timed out or errored. */
public Message[] call(RpcRequestBody[] params, InetSocketAddress[] addresses,
Class<? extends IpcProtocol> protocol,
User ticket)
throws IOException, InterruptedException {
if (addresses.length == 0) return new RpcResponseBody[0];
ParallelResults results = new ParallelResults(params.length);
// TODO this synchronization block doesnt make any sense, we should possibly fix it
//noinspection SynchronizationOnLocalVariableOrMethodParameter
synchronized (results) {
for (int i = 0; i < params.length; i++) {
ParallelCall call = new ParallelCall(params[i], results, i);
try {
Connection connection =
getConnection(addresses[i], protocol, ticket, 0, call);
connection.sendParam(call); // send each parameter
} catch (IOException e) {
// log errors
LOG.info("Calling "+addresses[i]+" caught: " +
e.getMessage(),e);
results.size--; // wait for one fewer result
}
}
while (results.count != results.size) {
try {
results.wait(); // wait for all results
} catch (InterruptedException ignored) {}
}
return results.values;
}
}
/* Get a connection from the pool, or create a new one and add it to the /* Get a connection from the pool, or create a new one and add it to the
* pool. Connections to a given host/port are reused. */ * pool. Connections to a given host/port are reused. */
protected Connection getConnection(InetSocketAddress addr, protected Connection getConnection(InetSocketAddress addr, Class<? extends IpcProtocol> protocol,
Class<? extends IpcProtocol> protocol, User ticket, int rpcTimeout, Call call, final Codec codec, final CompressionCodec compressor)
User ticket,
int rpcTimeout,
Call call)
throws IOException, InterruptedException { throws IOException, InterruptedException {
if (!running.get()) { if (!running.get()) {
// the client is stopped // the client is stopped
@ -1422,7 +1398,7 @@ public class HBaseClient {
synchronized (connections) { synchronized (connections) {
connection = connections.get(remoteId); connection = connections.get(remoteId);
if (connection == null) { if (connection == null) {
connection = createConnection(remoteId); connection = createConnection(remoteId, this.codec, this.compressor);
connections.put(remoteId, connection); connections.put(remoteId, connection);
} }
} }
@ -1471,6 +1447,12 @@ public class HBaseClient {
return ticket; return ticket;
} }
@Override
public String toString() {
return this.address.toString() + "/" + this.protocol + "/" + this.ticket + "/" +
this.rpcTimeout;
}
@Override @Override
public boolean equals(Object obj) { public boolean equals(Object obj) {
if (obj instanceof ConnectionId) { if (obj instanceof ConnectionId) {
@ -1484,9 +1466,9 @@ public class HBaseClient {
@Override // simply use the default Object#hashcode() ? @Override // simply use the default Object#hashcode() ?
public int hashCode() { public int hashCode() {
return (address.hashCode() + PRIME * ( int hashcode = (address.hashCode() + PRIME * (PRIME * System.identityHashCode(protocol) ^
PRIME * System.identityHashCode(protocol) ^
(ticket == null ? 0 : ticket.hashCode()) )) ^ rpcTimeout; (ticket == null ? 0 : ticket.hashCode()) )) ^ rpcTimeout;
return hashcode;
} }
} }
} }

View File

@ -0,0 +1,246 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.ipc;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import org.apache.commons.io.IOUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.codec.Codec;
import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.compress.CodecPool;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.CompressionInputStream;
import org.apache.hadoop.io.compress.Compressor;
import org.apache.hadoop.io.compress.Decompressor;
import com.google.common.base.Preconditions;
import com.google.protobuf.CodedInputStream;
import com.google.protobuf.CodedOutputStream;
import com.google.protobuf.Message;
/**
* Utility to help ipc'ing.
*/
class IPCUtil {
public static final Log LOG = LogFactory.getLog(IPCUtil.class);
private final int cellBlockBuildingInitialBufferSize;
/**
* How much we think the decompressor will expand the original compressed content.
*/
private final int cellBlockDecompressionMultiplier;
private final Configuration conf;
IPCUtil(final Configuration conf) {
super();
this.conf = conf;
this.cellBlockBuildingInitialBufferSize =
conf.getInt("hbase.ipc.cellblock.building.initial.buffersize", 16 * 1024);
this.cellBlockDecompressionMultiplier =
conf.getInt("hbase.ipc.cellblock.decompression.buffersize.multiplier", 3);
}
/**
* Build a cell block using passed in <code>codec</code>
* @param codec
* @param compressor
* @Param cells
* @return Null or byte buffer filled with passed-in Cells encoded using passed in
* <code>codec</code>; the returned buffer has been flipped and is ready for
* reading. Use limit to find total size.
* @throws IOException
*/
@SuppressWarnings("resource")
ByteBuffer buildCellBlock(final Codec codec, final CompressionCodec compressor,
final CellScanner cells)
throws IOException {
if (cells == null) return null;
// TOOD: Reuse buffers?
// Presizing doesn't work because can't tell what size will be when serialized.
// BBOS will resize itself.
ByteBufferOutputStream baos =
new ByteBufferOutputStream(this.cellBlockBuildingInitialBufferSize);
OutputStream os = baos;
Compressor poolCompressor = null;
try {
if (compressor != null) {
if (compressor instanceof Configurable) ((Configurable)compressor).setConf(this.conf);
poolCompressor = CodecPool.getCompressor(compressor);
os = compressor.createOutputStream(os, poolCompressor);
}
Codec.Encoder encoder = codec.getEncoder(os);
while (cells.advance()) {
encoder.write(cells.current());
}
encoder.flush();
} finally {
os.close();
if (poolCompressor != null) CodecPool.returnCompressor(poolCompressor);
}
if (this.cellBlockBuildingInitialBufferSize < baos.size()) {
LOG.warn("Buffer grew from " + this.cellBlockBuildingInitialBufferSize +
" to " + baos.size());
}
return baos.getByteBuffer();
}
/**
* @param codec
* @param cellBlock
* @return CellScanner to work against the content of <code>cellBlock</code>
* @throws IOException
*/
CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor,
final byte [] cellBlock)
throws IOException {
return createCellScanner(codec, compressor, cellBlock, 0, cellBlock.length);
}
/**
* @param codec
* @param cellBlock
* @param offset
* @param length
* @return CellScanner to work against the content of <code>cellBlock</code>
* @throws IOException
*/
CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor,
final byte [] cellBlock, final int offset, final int length)
throws IOException {
// If compressed, decompress it first before passing it on else we will leak compression
// resources if the stream is not closed properly after we let it out.
InputStream is = null;
if (compressor != null) {
// GZIPCodec fails w/ NPE if no configuration.
if (compressor instanceof Configurable) ((Configurable)compressor).setConf(this.conf);
Decompressor poolDecompressor = CodecPool.getDecompressor(compressor);
CompressionInputStream cis =
compressor.createInputStream(new ByteArrayInputStream(cellBlock, offset, length),
poolDecompressor);
try {
// TODO: This is ugly. The buffer will be resized on us if we guess wrong.
// TODO: Reuse buffers.
ByteBufferOutputStream bbos = new ByteBufferOutputStream((length - offset) *
this.cellBlockDecompressionMultiplier);
IOUtils.copy(cis, bbos);
bbos.close();
ByteBuffer bb = bbos.getByteBuffer();
is = new ByteArrayInputStream(bb.array(), 0, bb.limit());
} finally {
if (is != null) is.close();
CodecPool.returnDecompressor(poolDecompressor);
}
} else {
is = new ByteArrayInputStream(cellBlock, offset, length);
}
return codec.getDecoder(is);
}
/**
* Write out header, param, and cell block if there to a {@link ByteBufferOutputStream} sized
* to hold these elements.
* @param header
* @param param
* @param cellBlock
* @return A {@link ByteBufferOutputStream} filled with the content of the passed in
* <code>header</code>, <code>param</code>, and <code>cellBlock</code>.
* @throws IOException
*/
static ByteBufferOutputStream write(final Message header, final Message param,
final ByteBuffer cellBlock)
throws IOException {
int totalSize = getTotalSizeWhenWrittenDelimited(header, param);
if (cellBlock != null) totalSize += cellBlock.limit();
ByteBufferOutputStream bbos = new ByteBufferOutputStream(totalSize);
write(bbos, header, param, cellBlock, totalSize);
bbos.close();
return bbos;
}
/**
* Write out header, param, and cell block if there is one.
* @param dos
* @param header
* @param param
* @param cellBlock
* @return Total number of bytes written.
* @throws IOException
*/
static int write(final OutputStream dos, final Message header, final Message param,
final ByteBuffer cellBlock)
throws IOException {
// Must calculate total size and write that first so other side can read it all in in one
// swoop. This is dictated by how the server is currently written. Server needs to change
// if we are to be able to write without the length prefixing.
int totalSize = IPCUtil.getTotalSizeWhenWrittenDelimited(header, param);
if (cellBlock != null) totalSize += cellBlock.remaining();
return write(dos, header, param, cellBlock, totalSize);
}
private static int write(final OutputStream dos, final Message header, final Message param,
final ByteBuffer cellBlock, final int totalSize)
throws IOException {
// I confirmed toBytes does same as say DataOutputStream#writeInt.
dos.write(Bytes.toBytes(totalSize));
header.writeDelimitedTo(dos);
if (param != null) param.writeDelimitedTo(dos);
if (cellBlock != null) dos.write(cellBlock.array(), 0, cellBlock.remaining());
dos.flush();
return totalSize;
}
/**
* @param in Stream cue'd up just before a delimited message
* @return Bytes that hold the bytes that make up the message read from <code>in</code>
* @throws IOException
*/
static byte [] getDelimitedMessageBytes(final DataInputStream in) throws IOException {
byte b = in.readByte();
int size = CodedInputStream.readRawVarint32(b, in);
// Allocate right-sized buffer rather than let pb allocate its default minimum 4k.
byte [] bytes = new byte[size];
IOUtils.readFully(in, bytes);
return bytes;
}
/**
* @param header
* @param body
* @return Size on the wire when the two messages are written with writeDelimitedTo
*/
static int getTotalSizeWhenWrittenDelimited(Message ... messages) {
int totalSize = 0;
for (Message m: messages) {
if (m == null) continue;
totalSize += m.getSerializedSize();
totalSize += CodedOutputStream.computeRawVarint32Size(m.getSerializedSize());
}
Preconditions.checkArgument(totalSize < Integer.MAX_VALUE);
return totalSize;
}
}

View File

@ -0,0 +1,106 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.ipc;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController;
/**
* Optionally carries Cells across the proxy/service interface down into ipc. On its
* way out it optionally carries a set of result Cell data. We stick the Cells here when we want
* to avoid having to protobuf them. This class is used ferrying data across the proxy/protobuf
* service chasm. Used by client and server ipc'ing.
*/
@InterfaceAudience.Private
public class PayloadCarryingRpcController implements RpcController, CellScannable {
// TODO: Fill out the rest of this class methods rather than return UnsupportedOperationException
/**
* They are optionally set on construction, cleared after we make the call, and then optionally
* set on response with the result. We use this lowest common denominator access to Cells because
* sometimes the scanner is backed by a List of Cells and other times, it is backed by an
* encoded block that implements CellScanner.
*/
private CellScanner cellScanner;
public PayloadCarryingRpcController() {
this((CellScanner)null);
}
public PayloadCarryingRpcController(final CellScanner cellScanner) {
this.cellScanner = cellScanner;
}
public PayloadCarryingRpcController(final List<CellScannable> cellIterables) {
this.cellScanner = CellUtil.createCellScanner(cellIterables);
}
/**
* @return One-shot cell scanner (you cannot back it up and restart)
*/
public CellScanner cellScanner() {
return cellScanner;
}
public void setCellScanner(final CellScanner cellScanner) {
this.cellScanner = cellScanner;
}
@Override
public String errorText() {
throw new UnsupportedOperationException();
}
@Override
public boolean failed() {
throw new UnsupportedOperationException();
}
@Override
public boolean isCanceled() {
throw new UnsupportedOperationException();
}
@Override
public void notifyOnCancel(RpcCallback<Object> arg0) {
throw new UnsupportedOperationException();
}
@Override
public void reset() {
throw new UnsupportedOperationException();
}
@Override
public void setFailed(String arg0) {
throw new UnsupportedOperationException();
}
@Override
public void startCancel() {
throw new UnsupportedOperationException();
}
}

View File

@ -24,9 +24,10 @@ import com.google.protobuf.ServiceException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.IpcProtocol; import org.apache.hadoop.hbase.IpcProtocol;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import java.io.IOException; import java.io.IOException;
@ -77,7 +78,8 @@ public class ProtobufRpcClientEngine implements RpcClientEngine {
final private int rpcTimeout; final private int rpcTimeout;
public Invoker(Class<? extends IpcProtocol> protocol, InetSocketAddress addr, User ticket, public Invoker(Class<? extends IpcProtocol> protocol, InetSocketAddress addr, User ticket,
int rpcTimeout, HBaseClient client) throws IOException { int rpcTimeout, HBaseClient client)
throws IOException {
this.protocol = protocol; this.protocol = protocol;
this.address = addr; this.address = addr;
this.ticket = ticket; this.ticket = ticket;
@ -85,30 +87,6 @@ public class ProtobufRpcClientEngine implements RpcClientEngine {
this.rpcTimeout = rpcTimeout; this.rpcTimeout = rpcTimeout;
} }
private RpcRequestBody constructRpcRequest(Method method,
Object[] params) throws ServiceException {
RpcRequestBody rpcRequest;
RpcRequestBody.Builder builder = RpcRequestBody.newBuilder();
builder.setMethodName(method.getName());
Message param;
int length = params.length;
if (length == 2) {
// RpcController + Message in the method args
// (generated code from RPC bits in .proto files have RpcController)
param = (Message)params[1];
} else if (length == 1) { // Message
param = (Message)params[0];
} else {
throw new ServiceException("Too many parameters for request. Method: ["
+ method.getName() + "]" + ", Expected: 2, Actual: "
+ params.length);
}
builder.setRequestClassName(param.getClass().getName());
builder.setRequest(param.toByteString());
rpcRequest = builder.build();
return rpcRequest;
}
/** /**
* This is the client side invoker of RPC method. It only throws * This is the client side invoker of RPC method. It only throws
* ServiceException, since the invocation proxy expects only * ServiceException, since the invocation proxy expects only
@ -122,7 +100,7 @@ public class ProtobufRpcClientEngine implements RpcClientEngine {
* set as cause in ServiceException</li> * set as cause in ServiceException</li>
* </ol> * </ol>
* *
* Note that the client calling protobuf RPC methods, must handle * <p>Note that the client calling protobuf RPC methods, must handle
* ServiceException by getting the cause from the ServiceException. If the * ServiceException by getting the cause from the ServiceException. If the
* cause is RemoteException, then unwrap it to get the exception thrown by * cause is RemoteException, then unwrap it to get the exception thrown by
* the server. * the server.
@ -131,24 +109,42 @@ public class ProtobufRpcClientEngine implements RpcClientEngine {
public Object invoke(Object proxy, Method method, Object[] args) public Object invoke(Object proxy, Method method, Object[] args)
throws ServiceException { throws ServiceException {
long startTime = 0; long startTime = 0;
if (LOG.isDebugEnabled()) { if (LOG.isTraceEnabled()) {
startTime = System.currentTimeMillis(); startTime = System.currentTimeMillis();
} }
if (args.length != 2) {
RpcRequestBody rpcRequest = constructRpcRequest(method, args); throw new ServiceException(method.getName() + " didn't get two args: " + args.length);
Message val = null; }
// Get the controller. Often null. Presume payload carrying controller. Payload is optional.
// It is cells/data that we do not want to protobuf.
PayloadCarryingRpcController controller = (PayloadCarryingRpcController)args[0];
CellScanner cells = null;
if (controller != null) {
cells = controller.cellScanner();
// Clear it here so we don't by mistake try and these cells processing results.
controller.setCellScanner(null);
}
// The request parameter
Message param = (Message)args[1];
Pair<Message, CellScanner> val = null;
try { try {
val = client.call(rpcRequest, address, protocol, ticket, rpcTimeout); val = client.call(method, param, cells, address, protocol, ticket, rpcTimeout);
if (controller != null) {
// Shove the results into controller so can be carried across the proxy/pb service void.
if (val.getSecond() != null) controller.setCellScanner(val.getSecond());
} else if (val.getSecond() != null) {
throw new ServiceException("Client dropping data on the floor!");
}
if (LOG.isDebugEnabled()) { if (LOG.isTraceEnabled()) {
long callTime = System.currentTimeMillis() - startTime; long callTime = System.currentTimeMillis() - startTime;
if (LOG.isTraceEnabled()) LOG.trace("Call: " + method.getName() + " " + callTime); if (LOG.isTraceEnabled()) LOG.trace("Call: " + method.getName() + " " + callTime);
} }
return val; return val.getFirst();
} catch (Throwable e) { } catch (Throwable e) {
if (e instanceof RemoteException) { if (e instanceof RemoteException) {
Throwable cause = ((RemoteException)e).unwrapRemoteException(); Throwable cause = ((RemoteException)e).unwrapRemoteException();
throw new ServiceException(cause); throw new ServiceException("methodName=" + method.getName(), cause);
} }
throw new ServiceException(e); throw new ServiceException(e);
} }
@ -158,8 +154,8 @@ public class ProtobufRpcClientEngine implements RpcClientEngine {
if (returnTypes.containsKey(method.getName())) { if (returnTypes.containsKey(method.getName())) {
return returnTypes.get(method.getName()); return returnTypes.get(method.getName());
} }
Class<?> returnType = method.getReturnType(); Class<?> returnType = method.getReturnType();
if (returnType.getName().equals("void")) return null;
Method newInstMethod = returnType.getMethod("getDefaultInstance"); Method newInstMethod = returnType.getMethod("getDefaultInstance");
newInstMethod.setAccessible(true); newInstMethod.setAccessible(true);
Message protoType = (Message) newInstMethod.invoke(null, (Object[]) null); Message protoType = (Message) newInstMethod.invoke(null, (Object[]) null);

View File

@ -0,0 +1,76 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.ipc;
import java.lang.reflect.Method;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.hadoop.hbase.IpcProtocol;
import com.google.protobuf.Message;
/**
* Save on relection by keeping around method, method argument, and constructor instances
*/
class ReflectionCache {
private final Map<String, Message> methodArgCache = new ConcurrentHashMap<String, Message>();
private final Map<String, Method> methodInstanceCache = new ConcurrentHashMap<String, Method>();
public ReflectionCache() {
super();
}
Method getMethod(Class<? extends IpcProtocol> protocol, String methodName) {
Method method = this.methodInstanceCache.get(methodName);
if (method != null) return method;
Method [] methods = protocol.getMethods();
for (Method m : methods) {
if (m.getName().equals(methodName)) {
m.setAccessible(true);
this.methodInstanceCache.put(methodName, m);
return m;
}
}
return null;
}
Message getMethodArgType(Method method) throws Exception {
Message protoType = this.methodArgCache.get(method.getName());
if (protoType != null) return protoType;
Class<?>[] args = method.getParameterTypes();
Class<?> arg;
if (args.length == 2) {
// RpcController + Message in the method args
// (generated code from RPC bits in .proto files have RpcController)
arg = args[1];
} else if (args.length == 1) {
arg = args[0];
} else {
//unexpected
return null;
}
//in the protobuf methods, args[1] is the only significant argument
Method newInstMethod = arg.getMethod("getDefaultInstance");
newInstMethod.setAccessible(true);
protoType = (Message) newInstMethod.invoke(null, (Object[]) null);
this.methodArgCache.put(method.getName(), protoType);
return protoType;
}
}

View File

@ -0,0 +1,67 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
import org.apache.hadoop.ipc.RemoteException;
/**
* An {@link RemoteException} with some extra information. If source exception
* was a {@link DoNotRetryIOException}, {@link #isDoNotRetry()} will return true.
*/
@SuppressWarnings("serial")
@InterfaceAudience.Private
public class RemoteWithExtrasException extends RemoteException {
private final String hostname;
private final int port;
private final boolean doNotRetry;
public RemoteWithExtrasException(String className, String msg, final boolean doNotRetry) {
this(className, msg, null, -1, doNotRetry);
}
public RemoteWithExtrasException(String className, String msg, final String hostname,
final int port, final boolean doNotRetry) {
super(className, msg);
this.hostname = hostname;
this.port = port;
this.doNotRetry = doNotRetry;
}
/**
* @return null if not set
*/
public String getHostname() {
return this.hostname;
}
/**
* @return -1 if not set
*/
public int getPort() {
return this.port;
}
/**
* @return True if origin exception was a do not retry type.
*/
public boolean isDoNotRetry() {
return this.doNotRetry;
}
}

View File

@ -17,6 +17,7 @@
*/ */
package org.apache.hadoop.hbase.protobuf; package org.apache.hadoop.hbase.protobuf;
import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME; import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
@ -34,7 +35,21 @@ import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.NavigableSet; import java.util.NavigableSet;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ListMultimap;
import com.google.common.collect.Lists;
import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.Message;
import com.google.protobuf.RpcChannel;
import com.google.protobuf.Service;
import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
@ -42,20 +57,15 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.MasterAdminProtocol; import org.apache.hadoop.hbase.MasterAdminProtocol;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Action;
import org.apache.hadoop.hbase.client.AdminProtocol; import org.apache.hadoop.hbase.client.AdminProtocol;
import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.ClientProtocol; import org.apache.hadoop.hbase.client.ClientProtocol;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.MultiAction;
import org.apache.hadoop.hbase.client.MultiResponse;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
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.Scan;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
@ -88,12 +98,11 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServic
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest; 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.GetResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType;
import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos; import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; 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.NameBytesPair;
@ -114,16 +123,6 @@ import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ListMultimap;
import com.google.common.collect.Lists;
import com.google.protobuf.ByteString;
import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.Message;
import com.google.protobuf.RpcChannel;
import com.google.protobuf.Service;
import com.google.protobuf.ServiceException;
/** /**
* Protobufs utility. * Protobufs utility.
*/ */
@ -342,27 +341,54 @@ public final class ProtobufUtil {
} }
/** /**
* Convert a protocol buffer Mutate to a Put * Convert a protocol buffer Mutate to a Put.
* *
* @param proto the protocol buffer Mutate to convert * @param proto The protocol buffer MutationProto to convert
* @return the converted client Put * @return A client Put.
* @throws DoNotRetryIOException * @throws IOException
*/ */
public static Put toPut( public static Put toPut(final MutationProto proto)
final Mutate proto) throws DoNotRetryIOException { throws IOException {
MutateType type = proto.getMutateType(); return toPut(proto, null);
assert type == MutateType.PUT : type.name();
byte[] row = proto.getRow().toByteArray();
long timestamp = HConstants.LATEST_TIMESTAMP;
if (proto.hasTimestamp()) {
timestamp = proto.getTimestamp();
} }
Put put = new Put(row, timestamp);
put.setWriteToWAL(proto.getWriteToWAL()); /**
for (NameBytesPair attribute: proto.getAttributeList()) { * Convert a protocol buffer Mutate to a Put.
put.setAttribute(attribute.getName(), *
attribute.getValue().toByteArray()); * @param proto The protocol buffer MutationProto to convert
* @param cellScanner If non-null, the Cell data that goes with this proto.
* @return A client Put.
* @throws IOException
*/
public static Put toPut(final MutationProto proto, final CellScanner cellScanner)
throws IOException {
// TODO: Server-side at least why do we convert back to the Client types? Why not just pb it?
MutationType type = proto.getMutateType();
assert type == MutationType.PUT: type.name();
byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
long timestamp = proto.hasTimestamp()? proto.getTimestamp(): HConstants.LATEST_TIMESTAMP;
Put put = null;
int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
if (cellCount > 0) {
// The proto has metadata only and the data is separate to be found in the cellScanner.
if (cellScanner == null) {
throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " +
TextFormat.shortDebugString(proto));
} }
for (int i = 0; i < cellCount; i++) {
if (!cellScanner.advance()) {
throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i +
" no cell returned: " + TextFormat.shortDebugString(proto));
}
Cell cell = cellScanner.current();
if (put == null) {
put = new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), timestamp);
}
put.add(KeyValueUtil.ensureKeyValue(cell));
}
} else {
put = new Put(row, timestamp);
// The proto has the metadata and the data itself
for (ColumnValue column: proto.getColumnValueList()) { for (ColumnValue column: proto.getColumnValueList()) {
byte[] family = column.getFamily().toByteArray(); byte[] family = column.getFamily().toByteArray();
for (QualifierValue qv: column.getQualifierValueList()) { for (QualifierValue qv: column.getQualifierValueList()) {
@ -379,6 +405,11 @@ public final class ProtobufUtil {
put.add(family, qualifier, ts, value); put.add(family, qualifier, ts, value);
} }
} }
}
put.setWriteToWAL(proto.getWriteToWAL());
for (NameBytesPair attribute: proto.getAttributeList()) {
put.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
}
return put; return put;
} }
@ -387,21 +418,52 @@ public final class ProtobufUtil {
* *
* @param proto the protocol buffer Mutate to convert * @param proto the protocol buffer Mutate to convert
* @return the converted client Delete * @return the converted client Delete
* @throws IOException
*/ */
public static Delete toDelete(final Mutate proto) { public static Delete toDelete(final MutationProto proto)
MutateType type = proto.getMutateType(); throws IOException {
assert type == MutateType.DELETE : type.name(); return toDelete(proto, null);
byte[] row = proto.getRow().toByteArray(); }
/**
* Convert a protocol buffer Mutate to a Delete
*
* @param proto the protocol buffer Mutate to convert
* @param cellScanner if non-null, the data that goes with this delete.
* @return the converted client Delete
* @throws IOException
*/
public static Delete toDelete(final MutationProto proto, final CellScanner cellScanner)
throws IOException {
MutationType type = proto.getMutateType();
assert type == MutationType.DELETE : type.name();
byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
long timestamp = HConstants.LATEST_TIMESTAMP; long timestamp = HConstants.LATEST_TIMESTAMP;
if (proto.hasTimestamp()) { if (proto.hasTimestamp()) {
timestamp = proto.getTimestamp(); timestamp = proto.getTimestamp();
} }
Delete delete = new Delete(row, timestamp); Delete delete = null;
delete.setWriteToWAL(proto.getWriteToWAL()); int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
for (NameBytesPair attribute: proto.getAttributeList()) { if (cellCount > 0) {
delete.setAttribute(attribute.getName(), // The proto has metadata only and the data is separate to be found in the cellScanner.
attribute.getValue().toByteArray()); if (cellScanner == null) {
throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " +
TextFormat.shortDebugString(proto));
} }
for (int i = 0; i < cellCount; i++) {
if (!cellScanner.advance()) {
throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i +
" no cell returned: " + TextFormat.shortDebugString(proto));
}
Cell cell = cellScanner.current();
if (delete == null) {
delete =
new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), timestamp);
}
delete.addDeleteMarker(KeyValueUtil.ensureKeyValue(cell));
}
} else {
delete = new Delete(row, timestamp);
for (ColumnValue column: proto.getColumnValueList()) { for (ColumnValue column: proto.getColumnValueList()) {
byte[] family = column.getFamily().toByteArray(); byte[] family = column.getFamily().toByteArray();
for (QualifierValue qv: column.getQualifierValueList()) { for (QualifierValue qv: column.getQualifierValueList()) {
@ -423,27 +485,47 @@ public final class ProtobufUtil {
} }
} }
} }
}
delete.setWriteToWAL(proto.getWriteToWAL());
for (NameBytesPair attribute: proto.getAttributeList()) {
delete.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
}
return delete; return delete;
} }
/** /**
* Convert a protocol buffer Mutate to an Append * Convert a protocol buffer Mutate to an Append
* * @param cellScanner
* @param proto the protocol buffer Mutate to convert * @param proto the protocol buffer Mutate to convert
* @return the converted client Append * @return the converted client Append
* @throws DoNotRetryIOException * @throws DoNotRetryIOException
*/ */
public static Append toAppend( public static Append toAppend(final MutationProto proto, final CellScanner cellScanner)
final Mutate proto) throws DoNotRetryIOException { throws DoNotRetryIOException {
MutateType type = proto.getMutateType(); MutationType type = proto.getMutateType();
assert type == MutateType.APPEND : type.name(); assert type == MutationType.APPEND : type.name();
byte[] row = proto.getRow().toByteArray(); byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
Append append = new Append(row); Append append = null;
append.setWriteToWAL(proto.getWriteToWAL()); int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
for (NameBytesPair attribute: proto.getAttributeList()) { if (cellCount > 0) {
append.setAttribute(attribute.getName(), // The proto has metadata only and the data is separate to be found in the cellScanner.
attribute.getValue().toByteArray()); if (cellScanner == null) {
throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " +
TextFormat.shortDebugString(proto));
} }
for (int i = 0; i < cellCount; i++) {
if (!cellScanner.advance()) {
throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i +
" no cell returned: " + TextFormat.shortDebugString(proto));
}
Cell cell = cellScanner.current();
if (append == null) {
append = new Append(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
}
append.add(KeyValueUtil.ensureKeyValue(cell));
}
} else {
append = new Append(row);
for (ColumnValue column: proto.getColumnValueList()) { for (ColumnValue column: proto.getColumnValueList()) {
byte[] family = column.getFamily().toByteArray(); byte[] family = column.getFamily().toByteArray();
for (QualifierValue qv: column.getQualifierValueList()) { for (QualifierValue qv: column.getQualifierValueList()) {
@ -456,6 +538,11 @@ public final class ProtobufUtil {
append.add(family, qualifier, value); append.add(family, qualifier, value);
} }
} }
}
append.setWriteToWAL(proto.getWriteToWAL());
for (NameBytesPair attribute: proto.getAttributeList()) {
append.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
}
return append; return append;
} }
@ -466,18 +553,18 @@ public final class ProtobufUtil {
* @return the converted Mutation * @return the converted Mutation
* @throws IOException * @throws IOException
*/ */
public static Mutation toMutation(final Mutate proto) throws IOException { public static Mutation toMutation(final MutationProto proto) throws IOException {
MutateType type = proto.getMutateType(); MutationType type = proto.getMutateType();
if (type == MutateType.APPEND) { if (type == MutationType.APPEND) {
return toAppend(proto); return toAppend(proto, null);
} }
if (type == MutateType.DELETE) { if (type == MutationType.DELETE) {
return toDelete(proto); return toDelete(proto, null);
} }
if (type == MutateType.PUT) { if (type == MutationType.PUT) {
return toPut(proto); return toPut(proto, null);
} }
throw new IOException("Not an understood mutate type " + type); throw new IOException("Unknown mutation type " + type);
} }
/** /**
@ -487,13 +574,44 @@ public final class ProtobufUtil {
* @return the converted client Increment * @return the converted client Increment
* @throws IOException * @throws IOException
*/ */
public static Increment toIncrement( public static Increment toIncrement(final MutationProto proto, final CellScanner cellScanner)
final Mutate proto) throws IOException { throws IOException {
MutateType type = proto.getMutateType(); MutationType type = proto.getMutateType();
assert type == MutateType.INCREMENT : type.name(); assert type == MutationType.INCREMENT : type.name();
byte[] row = proto.getRow().toByteArray(); byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
Increment increment = new Increment(row); Increment increment = null;
increment.setWriteToWAL(proto.getWriteToWAL()); int cellCount = proto.hasAssociatedCellCount()? proto.getAssociatedCellCount(): 0;
if (cellCount > 0) {
// The proto has metadata only and the data is separate to be found in the cellScanner.
if (cellScanner == null) {
throw new DoNotRetryIOException("Cell count of " + cellCount + " but no cellScanner: " +
TextFormat.shortDebugString(proto));
}
for (int i = 0; i < cellCount; i++) {
if (!cellScanner.advance()) {
throw new DoNotRetryIOException("Cell count of " + cellCount + " but at index " + i +
" no cell returned: " + TextFormat.shortDebugString(proto));
}
Cell cell = cellScanner.current();
if (increment == null) {
increment = new Increment(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
}
increment.add(KeyValueUtil.ensureKeyValue(cell));
}
} else {
increment = new Increment(row);
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);
}
}
}
if (proto.hasTimeRange()) { if (proto.hasTimeRange()) {
HBaseProtos.TimeRange timeRange = proto.getTimeRange(); HBaseProtos.TimeRange timeRange = proto.getTimeRange();
long minStamp = 0; long minStamp = 0;
@ -506,18 +624,7 @@ public final class ProtobufUtil {
} }
increment.setTimeRange(minStamp, maxStamp); increment.setTimeRange(minStamp, maxStamp);
} }
for (ColumnValue column: proto.getColumnValueList()) { increment.setWriteToWAL(proto.getWriteToWAL());
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; return increment;
} }
@ -733,10 +840,10 @@ public final class ProtobufUtil {
* @param increment * @param increment
* @return the converted mutate * @return the converted mutate
*/ */
public static Mutate toMutate(final Increment increment) { public static MutationProto toMutation(final Increment increment) {
Mutate.Builder builder = Mutate.newBuilder(); MutationProto.Builder builder = MutationProto.newBuilder();
builder.setRow(ByteString.copyFrom(increment.getRow())); builder.setRow(ByteString.copyFrom(increment.getRow()));
builder.setMutateType(MutateType.INCREMENT); builder.setMutateType(MutationType.INCREMENT);
builder.setWriteToWAL(increment.getWriteToWAL()); builder.setWriteToWAL(increment.getWriteToWAL());
TimeRange timeRange = increment.getTimeRange(); TimeRange timeRange = increment.getTimeRange();
if (!timeRange.isAllTime()) { if (!timeRange.isAllTime()) {
@ -768,27 +875,14 @@ public final class ProtobufUtil {
/** /**
* Create a protocol buffer Mutate based on a client Mutation * Create a protocol buffer Mutate based on a client Mutation
* *
* @param mutateType * @param type
* @param mutation * @param mutation
* @return a mutate * @return a protobuf'd Mutation
* @throws IOException * @throws IOException
*/ */
public static Mutate toMutate(final MutateType mutateType, public static MutationProto toMutation(final MutationType type, final Mutation mutation)
final Mutation mutation) throws IOException { throws IOException {
Mutate.Builder mutateBuilder = Mutate.newBuilder(); MutationProto.Builder builder = getMutationBuilderAndSetCommonFields(type, mutation);
mutateBuilder.setRow(ByteString.copyFrom(mutation.getRow()));
mutateBuilder.setMutateType(mutateType);
mutateBuilder.setWriteToWAL(mutation.getWriteToWAL());
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(); ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder(); QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
for (Map.Entry<byte[],List<? extends Cell>> family: mutation.getFamilyMap().entrySet()) { for (Map.Entry<byte[],List<? extends Cell>> family: mutation.getFamilyMap().entrySet()) {
@ -799,15 +893,56 @@ public final class ProtobufUtil {
valueBuilder.setQualifier(ByteString.copyFrom(kv.getQualifier())); valueBuilder.setQualifier(ByteString.copyFrom(kv.getQualifier()));
valueBuilder.setValue(ByteString.copyFrom(kv.getValue())); valueBuilder.setValue(ByteString.copyFrom(kv.getValue()));
valueBuilder.setTimestamp(kv.getTimestamp()); valueBuilder.setTimestamp(kv.getTimestamp());
if (mutateType == MutateType.DELETE) { if (type == MutationType.DELETE) {
KeyValue.Type keyValueType = KeyValue.Type.codeToType(kv.getType()); KeyValue.Type keyValueType = KeyValue.Type.codeToType(kv.getType());
valueBuilder.setDeleteType(toDeleteType(keyValueType)); valueBuilder.setDeleteType(toDeleteType(keyValueType));
} }
columnBuilder.addQualifierValue(valueBuilder.build()); columnBuilder.addQualifierValue(valueBuilder.build());
} }
mutateBuilder.addColumnValue(columnBuilder.build()); builder.addColumnValue(columnBuilder.build());
} }
return mutateBuilder.build(); return builder.build();
}
/**
* Create a protocol buffer MutationProto based on a client Mutation. Does NOT include data.
* Understanding is that the Cell will be transported other than via protobuf.
* @param type
* @param mutation
* @return a protobuf'd Mutation
* @throws IOException
*/
public static MutationProto toMutationNoData(final MutationType type, final Mutation mutation)
throws IOException {
MutationProto.Builder builder = getMutationBuilderAndSetCommonFields(type, mutation);
builder.setAssociatedCellCount(mutation.size());
return builder.build();
}
/**
* Code shared by {@link #toMutation(MutationType, Mutation)} and
* {@link #toMutationNoData(MutationType, Mutation)}
* @param type
* @param mutation
* @return A partly-filled out protobuf'd Mutation.
*/
private static MutationProto.Builder getMutationBuilderAndSetCommonFields(final MutationType type,
final Mutation mutation) {
MutationProto.Builder builder = MutationProto.newBuilder();
builder.setRow(ByteString.copyFrom(mutation.getRow()));
builder.setMutateType(type);
builder.setWriteToWAL(mutation.getWriteToWAL());
builder.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()));
builder.addAttribute(attributeBuilder.build());
}
}
return builder;
} }
/** /**
@ -821,12 +956,25 @@ public final class ProtobufUtil {
Cell [] cells = result.raw(); Cell [] cells = result.raw();
if (cells != null) { if (cells != null) {
for (Cell c : cells) { for (Cell c : cells) {
builder.addKeyValue(toKeyValue(c)); builder.addCell(toCell(c));
} }
} }
return builder.build(); return builder.build();
} }
/**
* Convert a client Result to a protocol buffer Result.
* The pb Result does not include the Cell data. That is for transport otherwise.
*
* @param result the client Result to convert
* @return the converted protocol buffer Result
*/
public static ClientProtos.Result toResultNoData(final Result result) {
ClientProtos.Result.Builder builder = ClientProtos.Result.newBuilder();
builder.setAssociatedCellCount(result.size());
return builder.build();
}
/** /**
* Convert a protocol buffer Result to a client Result * Convert a protocol buffer Result to a client Result
* *
@ -834,12 +982,40 @@ public final class ProtobufUtil {
* @return the converted client Result * @return the converted client Result
*/ */
public static Result toResult(final ClientProtos.Result proto) { public static Result toResult(final ClientProtos.Result proto) {
List<HBaseProtos.KeyValue> values = proto.getKeyValueList(); List<HBaseProtos.Cell> values = proto.getCellList();
List<KeyValue> keyValues = new ArrayList<KeyValue>(values.size()); List<Cell> cells = new ArrayList<Cell>(values.size());
for (HBaseProtos.KeyValue kv: values) { for (HBaseProtos.Cell c: values) {
keyValues.add(toKeyValue(kv)); cells.add(toCell(c));
} }
return new Result(keyValues); return new Result(cells);
}
/**
* Convert a protocol buffer Result to a client Result
*
* @param proto the protocol buffer Result to convert
* @param scanner Optional cell scanner.
* @return the converted client Result
* @throws IOException
*/
public static Result toResult(final ClientProtos.Result proto, final CellScanner scanner)
throws IOException {
// TODO: Unit test that has some Cells in scanner and some in the proto.
List<Cell> cells = null;
if (proto.hasAssociatedCellCount()) {
int count = proto.getAssociatedCellCount();
cells = new ArrayList<Cell>(count);
for (int i = 0; i < count; i++) {
if (!scanner.advance()) throw new IOException("Failed get " + i + " of " + count);
cells.add(scanner.current());
}
}
List<HBaseProtos.Cell> values = proto.getCellList();
if (cells == null) cells = new ArrayList<Cell>(values.size());
for (HBaseProtos.Cell c: values) {
cells.add(toCell(c));
}
return new Result(cells);
} }
/** /**
@ -1011,55 +1187,6 @@ public final class ProtobufUtil {
} }
} }
/**
* A helper to invoke a multi action using client protocol.
*
* @param client
* @param multi
* @return a multi response
* @throws IOException
*/
public static <R> MultiResponse multi(final ClientProtocol client,
final MultiAction<R> multi) throws IOException {
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) {
try {
MultiRequest request =
RequestConverter.buildMultiRequest(regionName, (RowMutations)row);
client.multi(null, request);
response.add(regionName, action.getOriginalIndex(), new Result());
} catch (ServiceException se) {
response.add(regionName, action.getOriginalIndex(), getRemoteException(se));
}
rowMutations++;
}
}
if (actions.size() > rowMutations) {
Exception ex = null;
List<Object> results = null;
try {
MultiRequest request =
RequestConverter.buildMultiRequest(regionName, actions);
ClientProtos.MultiResponse proto = client.multi(null, request);
results = ResponseConverter.getResults(proto);
} catch (ServiceException se) {
ex = getRemoteException(se);
}
for (int i = 0, n = actions.size(); i < n; i++) {
int originalIndex = actions.get(i).getOriginalIndex();
response.add(regionName, originalIndex, results == null ? ex : results.get(i));
}
}
}
return response;
}
/** /**
* A helper to bulk load a list of HFiles using client protocol. * A helper to bulk load a list of HFiles using client protocol.
* *
@ -1731,33 +1858,31 @@ public final class ProtobufUtil {
throw new IOException(se); throw new IOException(se);
} }
public static HBaseProtos.KeyValue toKeyValue(final Cell kv) { public static HBaseProtos.Cell toCell(final Cell kv) {
// Doing this is going to kill us if we do it for all data passed. // Doing this is going to kill us if we do it for all data passed.
// St.Ack 20121205 // St.Ack 20121205
// TODO: Do a Cell version HBaseProtos.Cell.Builder kvbuilder = HBaseProtos.Cell.newBuilder();
HBaseProtos.KeyValue.Builder kvbuilder = HBaseProtos.KeyValue.newBuilder();
kvbuilder.setRow(ByteString.copyFrom(kv.getRowArray(), kv.getRowOffset(), kvbuilder.setRow(ByteString.copyFrom(kv.getRowArray(), kv.getRowOffset(),
kv.getRowLength())); kv.getRowLength()));
kvbuilder.setFamily(ByteString.copyFrom(kv.getFamilyArray(), kvbuilder.setFamily(ByteString.copyFrom(kv.getFamilyArray(),
kv.getFamilyOffset(), kv.getFamilyLength())); kv.getFamilyOffset(), kv.getFamilyLength()));
kvbuilder.setQualifier(ByteString.copyFrom(kv.getQualifierArray(), kvbuilder.setQualifier(ByteString.copyFrom(kv.getQualifierArray(),
kv.getQualifierOffset(), kv.getQualifierLength())); kv.getQualifierOffset(), kv.getQualifierLength()));
kvbuilder.setKeyType(HBaseProtos.KeyType.valueOf(kv.getTypeByte())); kvbuilder.setCellType(HBaseProtos.CellType.valueOf(kv.getTypeByte()));
kvbuilder.setTimestamp(kv.getTimestamp()); kvbuilder.setTimestamp(kv.getTimestamp());
kvbuilder.setValue(ByteString.copyFrom(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength())); kvbuilder.setValue(ByteString.copyFrom(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
return kvbuilder.build(); return kvbuilder.build();
} }
public static KeyValue toKeyValue(final HBaseProtos.KeyValue kv) { public static Cell toCell(final HBaseProtos.Cell cell) {
// Doing this is going to kill us if we do it for all data passed. // Doing this is going to kill us if we do it for all data passed.
// St.Ack 20121205 // St.Ack 20121205
// TODO: Do a Cell version return CellUtil.createCell(cell.getRow().toByteArray(),
return new KeyValue(kv.getRow().toByteArray(), cell.getFamily().toByteArray(),
kv.getFamily().toByteArray(), cell.getQualifier().toByteArray(),
kv.getQualifier().toByteArray(), cell.getTimestamp(),
kv.getTimestamp(), (byte)cell.getCellType().getNumber(),
KeyValue.Type.codeToType((byte)kv.getKeyType().getNumber()), cell.getValue().toByteArray());
kv.getValue().toByteArray());
} }
/** /**

View File

@ -17,8 +17,11 @@
*/ */
package org.apache.hadoop.hbase.protobuf; package org.apache.hadoop.hbase.protobuf;
import com.google.protobuf.ByteString; import java.io.IOException;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
@ -58,11 +61,11 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest; 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.MutateType;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
@ -92,8 +95,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLa
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import java.io.IOException; import com.google.protobuf.ByteString;
import java.util.List;
/** /**
* Helper utility to build protocol buffer requests, * Helper utility to build protocol buffer requests,
@ -206,9 +208,9 @@ public final class RequestConverter {
RegionSpecifierType.REGION_NAME, regionName); RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region); builder.setRegion(region);
Mutate.Builder mutateBuilder = Mutate.newBuilder(); MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
mutateBuilder.setRow(ByteString.copyFrom(row)); mutateBuilder.setRow(ByteString.copyFrom(row));
mutateBuilder.setMutateType(MutateType.INCREMENT); mutateBuilder.setMutateType(MutationType.INCREMENT);
mutateBuilder.setWriteToWAL(writeToWAL); mutateBuilder.setWriteToWAL(writeToWAL);
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder(); ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
columnBuilder.setFamily(ByteString.copyFrom(family)); columnBuilder.setFamily(ByteString.copyFrom(family));
@ -217,8 +219,7 @@ public final class RequestConverter {
valueBuilder.setQualifier(ByteString.copyFrom(qualifier)); valueBuilder.setQualifier(ByteString.copyFrom(qualifier));
columnBuilder.addQualifierValue(valueBuilder.build()); columnBuilder.addQualifierValue(valueBuilder.build());
mutateBuilder.addColumnValue(columnBuilder.build()); mutateBuilder.addColumnValue(columnBuilder.build());
builder.setMutation(mutateBuilder.build());
builder.setMutate(mutateBuilder.build());
return builder.build(); return builder.build();
} }
@ -245,7 +246,7 @@ public final class RequestConverter {
builder.setRegion(region); builder.setRegion(region);
Condition condition = buildCondition( Condition condition = buildCondition(
row, family, qualifier, comparator, compareType); row, family, qualifier, comparator, compareType);
builder.setMutate(ProtobufUtil.toMutate(MutateType.PUT, put)); builder.setMutation(ProtobufUtil.toMutation(MutationType.PUT, put));
builder.setCondition(condition); builder.setCondition(condition);
return builder.build(); return builder.build();
} }
@ -273,7 +274,7 @@ public final class RequestConverter {
builder.setRegion(region); builder.setRegion(region);
Condition condition = buildCondition( Condition condition = buildCondition(
row, family, qualifier, comparator, compareType); row, family, qualifier, comparator, compareType);
builder.setMutate(ProtobufUtil.toMutate(MutateType.DELETE, delete)); builder.setMutation(ProtobufUtil.toMutation(MutationType.DELETE, delete));
builder.setCondition(condition); builder.setCondition(condition);
return builder.build(); return builder.build();
} }
@ -292,7 +293,7 @@ public final class RequestConverter {
RegionSpecifier region = buildRegionSpecifier( RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName); RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region); builder.setRegion(region);
builder.setMutate(ProtobufUtil.toMutate(MutateType.PUT, put)); builder.setMutation(ProtobufUtil.toMutation(MutationType.PUT, put));
return builder.build(); return builder.build();
} }
@ -310,7 +311,7 @@ public final class RequestConverter {
RegionSpecifier region = buildRegionSpecifier( RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName); RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region); builder.setRegion(region);
builder.setMutate(ProtobufUtil.toMutate(MutateType.APPEND, append)); builder.setMutation(ProtobufUtil.toMutation(MutationType.APPEND, append));
return builder.build(); return builder.build();
} }
@ -327,7 +328,7 @@ public final class RequestConverter {
RegionSpecifier region = buildRegionSpecifier( RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName); RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region); builder.setRegion(region);
builder.setMutate(ProtobufUtil.toMutate(increment)); builder.setMutation(ProtobufUtil.toMutation(increment));
return builder.build(); return builder.build();
} }
@ -345,7 +346,7 @@ public final class RequestConverter {
RegionSpecifier region = buildRegionSpecifier( RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName); RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region); builder.setRegion(region);
builder.setMutate(ProtobufUtil.toMutate(MutateType.DELETE, delete)); builder.setMutation(ProtobufUtil.toMutation(MutationType.DELETE, delete));
return builder.build(); return builder.build();
} }
@ -358,29 +359,64 @@ public final class RequestConverter {
* @throws IOException * @throws IOException
*/ */
public static MultiRequest buildMultiRequest(final byte[] regionName, public static MultiRequest buildMultiRequest(final byte[] regionName,
final RowMutations rowMutations) throws IOException { final RowMutations rowMutations)
MultiRequest.Builder builder = MultiRequest.newBuilder(); throws IOException {
RegionSpecifier region = buildRegionSpecifier( MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName, true);
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
builder.setAtomic(true);
for (Mutation mutation: rowMutations.getMutations()) { for (Mutation mutation: rowMutations.getMutations()) {
MutateType mutateType = null; MutationType mutateType = null;
if (mutation instanceof Put) { if (mutation instanceof Put) {
mutateType = MutateType.PUT; mutateType = MutationType.PUT;
} else if (mutation instanceof Delete) { } else if (mutation instanceof Delete) {
mutateType = MutateType.DELETE; mutateType = MutationType.DELETE;
} else { } else {
throw new DoNotRetryIOException( throw new DoNotRetryIOException("RowMutations supports only put and delete, not " +
"RowMutations supports only put and delete, not " mutation.getClass().getName());
+ mutation.getClass().getName());
} }
Mutate mutate = ProtobufUtil.toMutate(mutateType, mutation); MutationProto mp = ProtobufUtil.toMutation(mutateType, mutation);
builder.addAction(MultiAction.newBuilder().setMutate(mutate).build()); builder.addAction(MultiAction.newBuilder().setMutation(mp).build());
} }
return builder.build(); return builder.build();
} }
/**
* Create a protocol buffer MultiRequest for row mutations that does not hold data. Data/Cells
* are carried outside of protobuf. Return references to the Cells in <code>cells</code> param
*
* @param regionName
* @param rowMutations
* @param cells Return in here a list of Cells as CellIterable.
* @return a multi request minus data
* @throws IOException
*/
public static MultiRequest buildNoDataMultiRequest(final byte[] regionName,
final RowMutations rowMutations, final List<CellScannable> cells)
throws IOException {
MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName, true);
for (Mutation mutation: rowMutations.getMutations()) {
MutationType type = null;
if (mutation instanceof Put) {
type = MutationType.PUT;
} else if (mutation instanceof Delete) {
type = MutationType.DELETE;
} else {
throw new DoNotRetryIOException("RowMutations supports only put and delete, not " +
mutation.getClass().getName());
}
MutationProto mp = ProtobufUtil.toMutationNoData(type, mutation);
cells.add(mutation);
builder.addAction(MultiAction.newBuilder().setMutation(mp).build());
}
return builder.build();
}
private static MultiRequest.Builder getMultiRequestBuilderWithRegionAndAtomicSet(final byte [] regionName,
final boolean atomic) {
MultiRequest.Builder builder = MultiRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
return builder.setAtomic(atomic);
}
/** /**
* Create a protocol buffer ScanRequest for a client Scan * Create a protocol buffer ScanRequest for a client Scan
* *
@ -475,25 +511,22 @@ public final class RequestConverter {
* @throws IOException * @throws IOException
*/ */
public static <R> MultiRequest buildMultiRequest(final byte[] regionName, public static <R> MultiRequest buildMultiRequest(final byte[] regionName,
final List<Action<R>> actions) throws IOException { final List<Action<R>> actions)
MultiRequest.Builder builder = MultiRequest.newBuilder(); throws IOException {
RegionSpecifier region = buildRegionSpecifier( MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName, false);
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
for (Action<R> action: actions) { for (Action<R> action: actions) {
MultiAction.Builder protoAction = MultiAction.newBuilder(); MultiAction.Builder protoAction = MultiAction.newBuilder();
Row row = action.getAction(); Row row = action.getAction();
if (row instanceof Get) { if (row instanceof Get) {
protoAction.setGet(ProtobufUtil.toGet((Get)row)); protoAction.setGet(ProtobufUtil.toGet((Get)row));
} else if (row instanceof Put) { } else if (row instanceof Put) {
protoAction.setMutate(ProtobufUtil.toMutate(MutateType.PUT, (Put)row)); protoAction.setMutation(ProtobufUtil.toMutation(MutationType.PUT, (Put)row));
} else if (row instanceof Delete) { } else if (row instanceof Delete) {
protoAction.setMutate(ProtobufUtil.toMutate(MutateType.DELETE, (Delete)row)); protoAction.setMutation(ProtobufUtil.toMutation(MutationType.DELETE, (Delete)row));
} else if (row instanceof Append) { } else if (row instanceof Append) {
protoAction.setMutate(ProtobufUtil.toMutate(MutateType.APPEND, (Append)row)); protoAction.setMutation(ProtobufUtil.toMutation(MutationType.APPEND, (Append)row));
} else if (row instanceof Increment) { } else if (row instanceof Increment) {
protoAction.setMutate(ProtobufUtil.toMutate((Increment)row)); protoAction.setMutation(ProtobufUtil.toMutation((Increment)row));
} else if (row instanceof RowMutations) { } else if (row instanceof RowMutations) {
continue; // ignore RowMutations continue; // ignore RowMutations
} else { } else {
@ -505,6 +538,68 @@ public final class RequestConverter {
return builder.build(); return builder.build();
} }
/**
* Create a protocol buffer multirequest with NO data for a list of actions (data is carried
* otherwise than via protobuf). This means it just notes attributes, whether to write the
* WAL, etc., and the presence in protobuf serves as place holder for the data which is
* coming along otherwise. Note that Get is different. It does not contain 'data' and is always
* carried by protobuf. We return references to the data by adding them to the passed in
* <code>data</code> param.
*
* RowMutations in the list (if any) will be ignored.
*
* @param regionName
* @param actions
* @param cells Place to stuff references to actual data.
* @return a multi request that does not carry any data.
* @throws IOException
*/
public static <R> MultiRequest buildNoDataMultiRequest(final byte[] regionName,
final List<Action<R>> actions, final List<CellScannable> cells)
throws IOException {
MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName, false);
for (Action<R> action: actions) {
MultiAction.Builder protoAction = MultiAction.newBuilder();
Row row = action.getAction();
if (row instanceof Get) {
// Gets are carried by protobufs.
protoAction.setGet(ProtobufUtil.toGet((Get)row));
} else if (row instanceof Put) {
Put p = (Put)row;
cells.add(p);
protoAction.setMutation(ProtobufUtil.toMutationNoData(MutationType.PUT, p));
} else if (row instanceof Delete) {
Delete d = (Delete)row;
int size = d.size();
// Note that a legitimate Delete may have a size of zero; i.e. a Delete that has nothing
// in it but the row to delete. In this case, the current implementation does not make
// a KeyValue to represent a delete-of-all-the-row until we serialize... For such cases
// where the size returned is zero, we will send the Delete fully pb'd rather than have
// metadata only in the pb and then send the kv along the side in cells.
if (size > 0) {
cells.add(d);
protoAction.setMutation(ProtobufUtil.toMutationNoData(MutationType.DELETE, d));
} else {
protoAction.setMutation(ProtobufUtil.toMutation(MutationType.DELETE, d));
}
} else if (row instanceof Append) {
Append a = (Append)row;
cells.add(a);
protoAction.setMutation(ProtobufUtil.toMutationNoData(MutationType.APPEND, a));
} else if (row instanceof Increment) {
Increment i = (Increment)row;
cells.add(i);
protoAction.setMutation(ProtobufUtil.toMutationNoData(MutationType.INCREMENT, i));
} else if (row instanceof RowMutations) {
continue; // ignore RowMutations
} else {
throw new DoNotRetryIOException("Multi doesn't support " + row.getClass().getName());
}
builder.addAction(protoAction.build());
}
return builder.build();
}
// End utilities for Client // End utilities for Client
//Start utilities for Admin //Start utilities for Admin

View File

@ -17,9 +17,12 @@
*/ */
package org.apache.hadoop.hbase.protobuf; package org.apache.hadoop.hbase.protobuf;
import com.google.protobuf.ByteString; import java.io.IOException;
import com.google.protobuf.RpcController; import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
@ -42,9 +45,8 @@ import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.security.access.UserPermission; import org.apache.hadoop.hbase.security.access.UserPermission;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import java.io.IOException; import com.google.protobuf.ByteString;
import java.util.ArrayList; import com.google.protobuf.RpcController;
import java.util.List;
/** /**
* Helper utility to build protocol buffer responses, * Helper utility to build protocol buffer responses,
@ -78,11 +80,13 @@ public final class ResponseConverter {
* Get the results from a protocol buffer MultiResponse * Get the results from a protocol buffer MultiResponse
* *
* @param proto the protocol buffer MultiResponse to convert * @param proto the protocol buffer MultiResponse to convert
* @return the results in the MultiResponse * @param cells Cells to go with the passed in <code>proto</code>. Can be null.
* @return the results that were in the MultiResponse (a Result or an Exception).
* @throws IOException * @throws IOException
*/ */
public static List<Object> getResults( public static List<Object> getResults(final ClientProtos.MultiResponse proto,
final ClientProtos.MultiResponse proto) throws IOException { final CellScanner cells)
throws IOException {
List<Object> results = new ArrayList<Object>(); List<Object> results = new ArrayList<Object>();
List<ActionResult> resultList = proto.getResultList(); List<ActionResult> resultList = proto.getResultList();
for (int i = 0, n = resultList.size(); i < n; i++) { for (int i = 0, n = resultList.size(); i < n; i++) {
@ -90,13 +94,8 @@ public final class ResponseConverter {
if (result.hasException()) { if (result.hasException()) {
results.add(ProtobufUtil.toException(result.getException())); results.add(ProtobufUtil.toException(result.getException()));
} else if (result.hasValue()) { } else if (result.hasValue()) {
ClientProtos.Result r = result.getValue(); ClientProtos.Result value = result.getValue();
Object value = ProtobufUtil.toResult(r); results.add(ProtobufUtil.toResult(value, cells));
if (value instanceof ClientProtos.Result) {
results.add(ProtobufUtil.toResult((ClientProtos.Result)value));
} else {
results.add(value);
}
} else { } else {
results.add(new Result()); results.add(new Result());
} }

View File

@ -0,0 +1,81 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.ipc;
import static org.junit.Assert.assertEquals;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.codec.Codec;
import org.apache.hadoop.hbase.codec.KeyValueCodec;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.DefaultCodec;
import org.apache.hadoop.io.compress.GzipCodec;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(SmallTests.class)
public class TestIPCUtil {
IPCUtil util;
@Before
public void before() {
this.util = new IPCUtil(new Configuration());
}
@Test
public void testBuildCellBlock() throws IOException {
doBuildCellBlockUndoCellBlock(new KeyValueCodec(), null);
doBuildCellBlockUndoCellBlock(new KeyValueCodec(), new DefaultCodec());
doBuildCellBlockUndoCellBlock(new KeyValueCodec(), new GzipCodec());
}
void doBuildCellBlockUndoCellBlock(final Codec codec, final CompressionCodec compressor)
throws IOException {
final int count = 10;
Cell [] cells = getCells(count);
ByteBuffer bb = this.util.buildCellBlock(codec, compressor,
CellUtil.createCellScanner(Arrays.asList(cells).iterator()));
CellScanner scanner =
this.util.createCellScanner(codec, compressor, bb.array(), 0, bb.limit());
int i = 0;
while (scanner.advance()) {
i++;
}
assertEquals(count, i);
}
static Cell [] getCells(final int howMany) {
Cell [] cells = new Cell[howMany];
for (int i = 0; i < howMany; i++) {
byte [] index = Bytes.toBytes(i);
KeyValue kv = new KeyValue(index, Bytes.toBytes("f"), index, index);
cells[i] = kv;
}
return cells;
}
}

View File

@ -0,0 +1,156 @@
package org.apache.hadoop.hbase.ipc;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(SmallTests.class)
public class TestPayloadCarryingRpcController {
@Test
public void testListOfCellScannerables() {
List<CellScannable> cells = new ArrayList<CellScannable>();
final int count = 10;
for (int i = 0; i < count; i++) {
cells.add(createCell(i));
}
PayloadCarryingRpcController controller = new PayloadCarryingRpcController(cells);
CellScanner cellScanner = controller.cellScanner();
int index = 0;
for (; cellScanner.advance(); index++) {
Cell cell = cellScanner.current();
byte [] indexBytes = Bytes.toBytes(index);
assertTrue("" + index, Bytes.equals(indexBytes, 0, indexBytes.length, cell.getValueArray(),
cell.getValueOffset(), cell.getValueLength()));
}
assertEquals(count, index);
}
/**
* @param index
* @return A faked out 'Cell' that does nothing but return index as its value
*/
static CellScannable createCell(final int index) {
return new CellScannable() {
@Override
public CellScanner cellScanner() {
return new CellScanner() {
@Override
public Cell current() {
// Fake out a Cell. All this Cell has is a value that is an int in size and equal
// to the above 'index' param serialized as an int.
return new Cell() {
private final int i = index;
@Override
public byte[] getRowArray() {
// TODO Auto-generated method stub
return null;
}
@Override
public int getRowOffset() {
// TODO Auto-generated method stub
return 0;
}
@Override
public short getRowLength() {
// TODO Auto-generated method stub
return 0;
}
@Override
public byte[] getFamilyArray() {
// TODO Auto-generated method stub
return null;
}
@Override
public int getFamilyOffset() {
// TODO Auto-generated method stub
return 0;
}
@Override
public byte getFamilyLength() {
// TODO Auto-generated method stub
return 0;
}
@Override
public byte[] getQualifierArray() {
// TODO Auto-generated method stub
return null;
}
@Override
public int getQualifierOffset() {
// TODO Auto-generated method stub
return 0;
}
@Override
public int getQualifierLength() {
// TODO Auto-generated method stub
return 0;
}
@Override
public long getTimestamp() {
// TODO Auto-generated method stub
return 0;
}
@Override
public byte getTypeByte() {
// TODO Auto-generated method stub
return 0;
}
@Override
public long getMvccVersion() {
// TODO Auto-generated method stub
return 0;
}
@Override
public byte[] getValueArray() {
return Bytes.toBytes(this.i);
}
@Override
public int getValueOffset() {
return 0;
}
@Override
public int getValueLength() {
return Bytes.SIZEOF_INT;
}
};
}
private boolean hasCell = true;
@Override
public boolean advance() {
// We have one Cell only so return true first time then false ever after.
if (!hasCell) return hasCell;
hasCell = false;
return true;
}
};
}
};
}
}

View File

@ -55,8 +55,8 @@ public final class HConstants {
/** /**
* The first four bytes of Hadoop RPC connections * The first four bytes of Hadoop RPC connections
*/ */
public static final ByteBuffer RPC_HEADER = ByteBuffer.wrap("hrpc".getBytes()); public static final ByteBuffer RPC_HEADER = ByteBuffer.wrap("HBas".getBytes());
public static final byte CURRENT_VERSION = 5; public static final byte RPC_CURRENT_VERSION = 0;
// HFileBlock constants. // HFileBlock constants.

View File

@ -11,14 +11,14 @@ public final class MultiRowMutation {
public interface MultiMutateRequestOrBuilder public interface MultiMutateRequestOrBuilder
extends com.google.protobuf.MessageOrBuilder { extends com.google.protobuf.MessageOrBuilder {
// repeated .Mutate mutationRequest = 1; // repeated .MutationProto mutationRequest = 1;
java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate> java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto>
getMutationRequestList(); getMutationRequestList();
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate getMutationRequest(int index); org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getMutationRequest(int index);
int getMutationRequestCount(); int getMutationRequestCount();
java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder> java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder>
getMutationRequestOrBuilderList(); getMutationRequestOrBuilderList();
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder getMutationRequestOrBuilder( org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationRequestOrBuilder(
int index); int index);
} }
public static final class MultiMutateRequest extends public static final class MultiMutateRequest extends
@ -49,23 +49,23 @@ public final class MultiRowMutation {
return org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.internal_static_MultiMutateRequest_fieldAccessorTable; return org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.internal_static_MultiMutateRequest_fieldAccessorTable;
} }
// repeated .Mutate mutationRequest = 1; // repeated .MutationProto mutationRequest = 1;
public static final int MUTATIONREQUEST_FIELD_NUMBER = 1; public static final int MUTATIONREQUEST_FIELD_NUMBER = 1;
private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate> mutationRequest_; private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto> mutationRequest_;
public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate> getMutationRequestList() { public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto> getMutationRequestList() {
return mutationRequest_; return mutationRequest_;
} }
public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder> public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder>
getMutationRequestOrBuilderList() { getMutationRequestOrBuilderList() {
return mutationRequest_; return mutationRequest_;
} }
public int getMutationRequestCount() { public int getMutationRequestCount() {
return mutationRequest_.size(); return mutationRequest_.size();
} }
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate getMutationRequest(int index) { public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getMutationRequest(int index) {
return mutationRequest_.get(index); return mutationRequest_.get(index);
} }
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder getMutationRequestOrBuilder( public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationRequestOrBuilder(
int index) { int index) {
return mutationRequest_.get(index); return mutationRequest_.get(index);
} }
@ -393,7 +393,7 @@ public final class MultiRowMutation {
break; break;
} }
case 10: { case 10: {
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.newBuilder(); org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.newBuilder();
input.readMessage(subBuilder, extensionRegistry); input.readMessage(subBuilder, extensionRegistry);
addMutationRequest(subBuilder.buildPartial()); addMutationRequest(subBuilder.buildPartial());
break; break;
@ -404,20 +404,20 @@ public final class MultiRowMutation {
private int bitField0_; private int bitField0_;
// repeated .Mutate mutationRequest = 1; // repeated .MutationProto mutationRequest = 1;
private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate> mutationRequest_ = private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto> mutationRequest_ =
java.util.Collections.emptyList(); java.util.Collections.emptyList();
private void ensureMutationRequestIsMutable() { private void ensureMutationRequestIsMutable() {
if (!((bitField0_ & 0x00000001) == 0x00000001)) { if (!((bitField0_ & 0x00000001) == 0x00000001)) {
mutationRequest_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate>(mutationRequest_); mutationRequest_ = new java.util.ArrayList<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto>(mutationRequest_);
bitField0_ |= 0x00000001; bitField0_ |= 0x00000001;
} }
} }
private com.google.protobuf.RepeatedFieldBuilder< private com.google.protobuf.RepeatedFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder> mutationRequestBuilder_; org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder> mutationRequestBuilder_;
public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate> getMutationRequestList() { public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto> getMutationRequestList() {
if (mutationRequestBuilder_ == null) { if (mutationRequestBuilder_ == null) {
return java.util.Collections.unmodifiableList(mutationRequest_); return java.util.Collections.unmodifiableList(mutationRequest_);
} else { } else {
@ -431,7 +431,7 @@ public final class MultiRowMutation {
return mutationRequestBuilder_.getCount(); return mutationRequestBuilder_.getCount();
} }
} }
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate getMutationRequest(int index) { public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto getMutationRequest(int index) {
if (mutationRequestBuilder_ == null) { if (mutationRequestBuilder_ == null) {
return mutationRequest_.get(index); return mutationRequest_.get(index);
} else { } else {
@ -439,7 +439,7 @@ public final class MultiRowMutation {
} }
} }
public Builder setMutationRequest( public Builder setMutationRequest(
int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate value) { int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto value) {
if (mutationRequestBuilder_ == null) { if (mutationRequestBuilder_ == null) {
if (value == null) { if (value == null) {
throw new NullPointerException(); throw new NullPointerException();
@ -453,7 +453,7 @@ public final class MultiRowMutation {
return this; return this;
} }
public Builder setMutationRequest( public Builder setMutationRequest(
int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder builderForValue) { int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder builderForValue) {
if (mutationRequestBuilder_ == null) { if (mutationRequestBuilder_ == null) {
ensureMutationRequestIsMutable(); ensureMutationRequestIsMutable();
mutationRequest_.set(index, builderForValue.build()); mutationRequest_.set(index, builderForValue.build());
@ -463,7 +463,7 @@ public final class MultiRowMutation {
} }
return this; return this;
} }
public Builder addMutationRequest(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate value) { public Builder addMutationRequest(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto value) {
if (mutationRequestBuilder_ == null) { if (mutationRequestBuilder_ == null) {
if (value == null) { if (value == null) {
throw new NullPointerException(); throw new NullPointerException();
@ -477,7 +477,7 @@ public final class MultiRowMutation {
return this; return this;
} }
public Builder addMutationRequest( public Builder addMutationRequest(
int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate value) { int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto value) {
if (mutationRequestBuilder_ == null) { if (mutationRequestBuilder_ == null) {
if (value == null) { if (value == null) {
throw new NullPointerException(); throw new NullPointerException();
@ -491,7 +491,7 @@ public final class MultiRowMutation {
return this; return this;
} }
public Builder addMutationRequest( public Builder addMutationRequest(
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder builderForValue) { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder builderForValue) {
if (mutationRequestBuilder_ == null) { if (mutationRequestBuilder_ == null) {
ensureMutationRequestIsMutable(); ensureMutationRequestIsMutable();
mutationRequest_.add(builderForValue.build()); mutationRequest_.add(builderForValue.build());
@ -502,7 +502,7 @@ public final class MultiRowMutation {
return this; return this;
} }
public Builder addMutationRequest( public Builder addMutationRequest(
int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder builderForValue) { int index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder builderForValue) {
if (mutationRequestBuilder_ == null) { if (mutationRequestBuilder_ == null) {
ensureMutationRequestIsMutable(); ensureMutationRequestIsMutable();
mutationRequest_.add(index, builderForValue.build()); mutationRequest_.add(index, builderForValue.build());
@ -513,7 +513,7 @@ public final class MultiRowMutation {
return this; return this;
} }
public Builder addAllMutationRequest( public Builder addAllMutationRequest(
java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate> values) { java.lang.Iterable<? extends org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto> values) {
if (mutationRequestBuilder_ == null) { if (mutationRequestBuilder_ == null) {
ensureMutationRequestIsMutable(); ensureMutationRequestIsMutable();
super.addAll(values, mutationRequest_); super.addAll(values, mutationRequest_);
@ -543,18 +543,18 @@ public final class MultiRowMutation {
} }
return this; return this;
} }
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder getMutationRequestBuilder( public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder getMutationRequestBuilder(
int index) { int index) {
return getMutationRequestFieldBuilder().getBuilder(index); return getMutationRequestFieldBuilder().getBuilder(index);
} }
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder getMutationRequestOrBuilder( public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationRequestOrBuilder(
int index) { int index) {
if (mutationRequestBuilder_ == null) { if (mutationRequestBuilder_ == null) {
return mutationRequest_.get(index); } else { return mutationRequest_.get(index); } else {
return mutationRequestBuilder_.getMessageOrBuilder(index); return mutationRequestBuilder_.getMessageOrBuilder(index);
} }
} }
public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder> public java.util.List<? extends org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder>
getMutationRequestOrBuilderList() { getMutationRequestOrBuilderList() {
if (mutationRequestBuilder_ != null) { if (mutationRequestBuilder_ != null) {
return mutationRequestBuilder_.getMessageOrBuilderList(); return mutationRequestBuilder_.getMessageOrBuilderList();
@ -562,25 +562,25 @@ public final class MultiRowMutation {
return java.util.Collections.unmodifiableList(mutationRequest_); return java.util.Collections.unmodifiableList(mutationRequest_);
} }
} }
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder addMutationRequestBuilder() { public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder addMutationRequestBuilder() {
return getMutationRequestFieldBuilder().addBuilder( return getMutationRequestFieldBuilder().addBuilder(
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDefaultInstance()); org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance());
} }
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder addMutationRequestBuilder( public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder addMutationRequestBuilder(
int index) { int index) {
return getMutationRequestFieldBuilder().addBuilder( return getMutationRequestFieldBuilder().addBuilder(
index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.getDefaultInstance()); index, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance());
} }
public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder> public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder>
getMutationRequestBuilderList() { getMutationRequestBuilderList() {
return getMutationRequestFieldBuilder().getBuilderList(); return getMutationRequestFieldBuilder().getBuilderList();
} }
private com.google.protobuf.RepeatedFieldBuilder< private com.google.protobuf.RepeatedFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder> org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder>
getMutationRequestFieldBuilder() { getMutationRequestFieldBuilder() {
if (mutationRequestBuilder_ == null) { if (mutationRequestBuilder_ == null) {
mutationRequestBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< mutationRequestBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder>( org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder>(
mutationRequest_, mutationRequest_,
((bitField0_ & 0x00000001) == 0x00000001), ((bitField0_ & 0x00000001) == 0x00000001),
getParentForChildren(), getParentForChildren(),
@ -1141,13 +1141,13 @@ public final class MultiRowMutation {
descriptor; descriptor;
static { static {
java.lang.String[] descriptorData = { java.lang.String[] descriptorData = {
"\n\026MultiRowMutation.proto\032\014Client.proto\"6" + "\n\026MultiRowMutation.proto\032\014Client.proto\"=" +
"\n\022MultiMutateRequest\022 \n\017mutationRequest\030" + "\n\022MultiMutateRequest\022\'\n\017mutationRequest\030" +
"\001 \003(\0132\007.Mutate\"\025\n\023MultiMutateResponse2R\n" + "\001 \003(\0132\016.MutationProto\"\025\n\023MultiMutateResp" +
"\027MultiRowMutationService\0227\n\nmutateRows\022\023" + "onse2R\n\027MultiRowMutationService\0227\n\nmutat" +
".MultiMutateRequest\032\024.MultiMutateRespons" + "eRows\022\023.MultiMutateRequest\032\024.MultiMutate" +
"eBF\n*org.apache.hadoop.hbase.protobuf.ge" + "ResponseBF\n*org.apache.hadoop.hbase.prot" +
"neratedB\020MultiRowMutationH\001\210\001\001\240\001\001" "obuf.generatedB\020MultiRowMutationH\001\210\001\001\240\001\001"
}; };
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {

View File

@ -51,7 +51,16 @@ message Get {
} }
message Result { message Result {
repeated KeyValue keyValue = 1; // Result includes the Cells or else it just has a count of Cells
// that are carried otherwise.
repeated Cell cell = 1;
// The below count is set when the associated cells are
// not part of this protobuf message; they are passed alongside
// and then this Message is just a placeholder with metadata.
// The count is needed to know how many to peel off the block of Cells as
// ours. NOTE: This is different from the pb managed cellCount of the
// 'cell' field above which is non-null when the cells are pb'd.
optional int32 associatedCellCount = 2;
} }
/** /**
@ -118,24 +127,34 @@ message Condition {
required Comparator comparator = 5; required Comparator comparator = 5;
} }
/** /**
* A specific mutate inside a mutate request. * A specific mutation inside a mutate request.
* It can be an append, increment, put or delete based * It can be an append, increment, put or delete based
* on the mutate type. * on the mutation type. It can be fully filled in or
* only metadata present because data is being carried
* elsewhere outside of pb.
*/ */
message Mutate { message MutationProto {
required bytes row = 1; optional bytes row = 1;
required MutateType mutateType = 2; optional MutationType mutateType = 2;
repeated ColumnValue columnValue = 3; repeated ColumnValue columnValue = 3;
repeated NameBytesPair attribute = 4; optional uint64 timestamp = 4;
optional uint64 timestamp = 5; repeated NameBytesPair attribute = 5;
optional bool writeToWAL = 6 [default = true]; optional bool writeToWAL = 6 [default = true];
// For some mutate, result may be returned, in which case, // For some mutations, a result may be returned, in which case,
// time range can be specified for potential performance gain // time range can be specified for potential performance gain
optional TimeRange timeRange = 10; optional TimeRange timeRange = 7;
// The below count is set when the associated cells are NOT
// part of this protobuf message; they are passed alongside
// and then this Message is a placeholder with metadata. The
// count is needed to know how many to peel off the block of Cells as
// ours. NOTE: This is different from the pb managed cellCount of the
// 'cell' field above which is non-null when the cells are pb'd.
optional int32 associatedCellCount = 8;
enum MutateType { enum MutationType {
APPEND = 0; APPEND = 0;
INCREMENT = 1; INCREMENT = 1;
PUT = 2; PUT = 2;
@ -172,7 +191,7 @@ message Mutate {
*/ */
message MutateRequest { message MutateRequest {
required RegionSpecifier region = 1; required RegionSpecifier region = 1;
required Mutate mutate = 2; required MutationProto mutation = 2;
optional Condition condition = 3; optional Condition condition = 3;
} }
@ -281,7 +300,7 @@ message CoprocessorServiceResponse {
* This is a union type - exactly one of the fields will be set. * This is a union type - exactly one of the fields will be set.
*/ */
message MultiAction { message MultiAction {
optional Mutate mutate = 1; optional MutationProto mutation = 1;
optional Get get = 2; optional Get get = 2;
} }

View File

@ -23,7 +23,7 @@ option java_generic_services = true;
option optimize_for = SPEED; option optimize_for = SPEED;
message MultiMutateRequest { message MultiMutateRequest {
repeated Mutate mutationRequest = 1; repeated MutationProto mutationRequest = 1;
} }
message MultiMutateResponse { message MultiMutateResponse {

View File

@ -15,123 +15,117 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
/**
* Specification of (unsecure) HBase RPC:
*
* Client needs to set up a connection first to a server serving a certain
* HBase protocol (like ClientProtocol). Once the connection is set up, the
* client and server communicates on that channel for RPC requests/responses.
* The sections below describe the flow.
*
* As part of setting up a connection to a server, the client needs to send
* the ConnectionHeader header. At the data level, this looks like
* <"hrpc"-bytearray><'5'[byte]><length-of-serialized-ConnectionHeader-obj[int32]><ConnectionHeader-object serialized>
*
* For every RPC that the client makes it needs to send the following
* RpcRequestHeader and the RpcRequestBody. At the data level this looks like:
* <length-of-serialized-RpcRequestHeader + length-of-varint32-of-serialized-RpcRequestHeader +
* length-of-serialized-RpcRequestBody + length-of-varint32-of-serialized-RpcRequestBody>
* <RpcRequestHeader [serialized using Message.writeDelimitedTo]>
* <RpcRequestBody [serialized using Message.writeDelimitedTo]>
*
* On a success, the server's protobuf response looks like
* <RpcResponseHeader-object [serialized using Message.writeDelimitedTo]>
* <RpcResponseBody-object [serialized using Message.writeDelimitedTo]>
* On a failure, the server's protobuf response looks like
* <RpcResponseHeader-object [serialized using Message.writeDelimitedTo]>
* <RpcException-object [serialized using Message.writeDelimitedTo]>
*
* There is one special message that's sent from client to server -
* the Ping message. At the data level, this is just the bytes corresponding
* to integer -1.
*/
import "Tracing.proto"; import "Tracing.proto";
import "hbase.proto";
option java_package = "org.apache.hadoop.hbase.protobuf.generated"; option java_package = "org.apache.hadoop.hbase.protobuf.generated";
option java_outer_classname = "RPCProtos"; option java_outer_classname = "RPCProtos";
option java_generate_equals_and_hash = true; option java_generate_equals_and_hash = true;
option optimize_for = SPEED; option optimize_for = SPEED;
// See https://issues.apache.org/jira/browse/HBASE-7898 for high-level
// description of RPC specification.
//
// On connection setup, the client sends six bytes of preamble -- a four
// byte magic, a byte of version, and a byte of authentication type.
//
// We then send a "ConnectionHeader" protobuf of user information and the
// 'protocol' or 'service' that is to be run over this connection as well as
// info such as codecs and compression to use when we send cell blocks(see below).
// This connection header protobuf is prefaced by an int that holds the length
// of this connection header (this is NOT a varint). The pb connection header
// is sent with Message#writeTo. The server throws an exception if it doesn't
// like what it was sent noting what it is objecting too. Otherwise, the server
// says nothing and is open for business.
//
// Hereafter the client makes requests and the server returns responses.
//
// Requests look like this:
//
// <An int with the total length of the request>
// <RequestHeader Message written out using Message#writeDelimitedTo>
// <Optionally a Request Parameter Message written out using Message#writeDelimitedTo>
// <Optionally a Cell block>
//
// ...where the Request Parameter Message is whatever the method name stipulated
// in the RequestHeader expects; e.g. if the method is a scan, then the pb
// Request Message is a GetRequest, or a ScanRequest. A block of Cells
// optionally follows. The presence of a Request param Message and/or a
// block of Cells will be noted in the RequestHeader.
//
// Response is the mirror of the request:
//
// <An int with the total length of the response>
// <ResponseHeader Message written out using Message#writeDelimitedTo>
// <Optionally a Response Result Message written out using Message#writeDelimitedTo>
// <Optionally a Cell block>
//
// ...where the Response Message is the response type that goes with the
// method specified when making the request and the follow on Cell blocks may
// or may not be there -- read the response header to find out if one following.
// If an exception, it will be included inside the Response Header.
//
// Any time we write a pb, we do it with Message#writeDelimitedTo EXCEPT when
// the connection header is sent; this is prefaced by an int with its length
// and the pb connection header is then written with Message#writeTo.
//
// User Information proto. Included in ConnectionHeader on connection setup
message UserInformation { message UserInformation {
required string effectiveUser = 1; required string effectiveUser = 1;
optional string realUser = 2; optional string realUser = 2;
} }
// This is sent on connection setup after the connection preamble is sent.
message ConnectionHeader { message ConnectionHeader {
/** User Info beyond what is established at connection establishment
* (applies to secure HBase setup)
*/
optional UserInformation userInfo = 1; optional UserInformation userInfo = 1;
/** Protocol name for next rpc layer
* the client created a proxy with this protocol name
*/
optional string protocol = 2 [default = "org.apache.hadoop.hbase.client.ClientProtocol"]; optional string protocol = 2 [default = "org.apache.hadoop.hbase.client.ClientProtocol"];
// Cell block codec we will use sending over optional cell blocks. Server throws exception
// if cannot deal.
optional string cellBlockCodecClass = 3 [default = "org.apache.hadoop.hbase.codec.KeyValueCodec"];
// Compressor we will use if cell block is compressed. Server will throw exception if not supported.
// Class must implement hadoop's CompressionCodec Interface
optional string cellBlockCompressorClass = 4;
} }
// Optional Cell block Message. Included in client RequestHeader
/** message CellBlockMeta {
* The RPC request header // Length of the following cell block. Could calculate it but convenient having it too hand.
*/ optional uint32 length = 1;
message RpcRequestHeader {
/** Monotonically increasing callId, mostly to keep track of RPCs */
required uint32 callId = 1;
optional RPCTInfo tinfo = 2;
}
/**
* The RPC request body
*/
message RpcRequestBody {
/** Name of the RPC method */
required string methodName = 1;
/** Bytes corresponding to the client protobuf request. This is the actual
* bytes corresponding to the RPC request argument.
*/
optional bytes request = 2;
/** Some metainfo about the request. Helps us to treat RPCs with
* different priorities. For now this is just the classname of the request
* proto object.
*/
optional string requestClassName = 4;
} }
/** // At the RPC layer, this message is used to carry
* The RPC response header // the server side exception to the RPC client.
*/ message ExceptionResponse {
message RpcResponseHeader { // Class name of the exception thrown from the server
/** Echo back the callId the client sent */ optional string exceptionClassName = 1;
required uint32 callId = 1; // Exception stack trace from the server side
/** Did the RPC execution encounter an error at the server */
enum Status {
SUCCESS = 0;
ERROR = 1;
FATAL = 2;
}
required Status status = 2;
}
/**
* The RPC response body
*/
message RpcResponseBody {
/** Optional response bytes. This is the actual bytes corresponding to the
* return value of the invoked RPC.
*/
optional bytes response = 1;
}
/**
* At the RPC layer, this message is used to indicate
* the server side exception to the RPC client.
*
* HBase RPC client throws an exception indicated
* by exceptionName with the stackTrace.
*/
message RpcException {
/** Class name of the exception thrown from the server */
required string exceptionName = 1;
/** Exception stack trace from the server side */
optional string stackTrace = 2; optional string stackTrace = 2;
// Optional hostname. Filled in for some exceptions such as region moved
// where exception gives clue on where the region may have moved.
optional string hostname = 3;
optional int32 port = 4;
// Set if we are NOT to retry on receipt of this exception
optional bool doNotRetry = 5;
}
// Header sent making a request.
message RequestHeader {
// Monotonically increasing callId to keep track of RPC requests and their response
optional uint32 callId = 1;
optional RPCTInfo traceInfo = 2;
optional string methodName = 3;
// If true, then a pb Message param follows.
optional bool requestParam = 4;
// If present, then an encoded data block follows.
optional CellBlockMeta cellBlockMeta = 5;
// TODO: Have client specify priority
}
message ResponseHeader {
optional uint32 callId = 1;
// If present, then request threw an exception and no response message (else we presume one)
optional ExceptionResponse exception = 2;
// If present, then an encoded data block follows.
optional CellBlockMeta cellBlockMeta = 3;
} }

View File

@ -23,6 +23,33 @@ option java_outer_classname = "HBaseProtos";
option java_generate_equals_and_hash = true; option java_generate_equals_and_hash = true;
option optimize_for = SPEED; option optimize_for = SPEED;
/**
* The type of the key in a Cell
*/
enum CellType {
MINIMUM = 0;
PUT = 4;
DELETE = 8;
DELETE_COLUMN = 12;
DELETE_FAMILY = 14;
// MAXIMUM is used when searching; you look from maximum on down.
MAXIMUM = 255;
}
/**
* Protocol buffer version of Cell.
*/
message Cell {
optional bytes row = 1;
optional bytes family = 2;
optional bytes qualifier = 3;
optional uint64 timestamp = 4;
optional CellType cellType = 5;
optional bytes value = 6;
}
/** /**
* Table Schema * Table Schema
* Inspired by the rest TableSchema * Inspired by the rest TableSchema
@ -200,21 +227,6 @@ enum CompareType {
NO_OP = 6; NO_OP = 6;
} }
/**
* The type of the key in a KeyValue.
*/
enum KeyType {
MINIMUM = 0;
PUT = 4;
DELETE = 8;
DELETE_COLUMN = 12;
DELETE_FAMILY = 14;
// MAXIMUM is used when searching; you look from maximum on down.
MAXIMUM = 255;
}
/** /**
* Protocol buffer version of KeyValue. * Protocol buffer version of KeyValue.
* It doesn't have those transient parameters * It doesn't have those transient parameters
@ -224,7 +236,7 @@ message KeyValue {
required bytes family = 2; required bytes family = 2;
required bytes qualifier = 3; required bytes qualifier = 3;
optional uint64 timestamp = 4; optional uint64 timestamp = 4;
optional KeyType keyType = 5; optional CellType keyType = 5;
optional bytes value = 6; optional bytes value = 6;
} }

View File

@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiMutateRequest; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiMutateRequest;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiRowMutationService; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiRowMutationService;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -301,7 +301,7 @@ public class MetaEditor {
CoprocessorRpcChannel channel = table.coprocessorService(row); CoprocessorRpcChannel channel = table.coprocessorService(row);
MultiMutateRequest.Builder mmrBuilder = MultiMutateRequest.newBuilder(); MultiMutateRequest.Builder mmrBuilder = MultiMutateRequest.newBuilder();
for (Put put : puts) { for (Put put : puts) {
mmrBuilder.addMutationRequest(ProtobufUtil.toMutate(MutateType.PUT, put)); mmrBuilder.addMutationRequest(ProtobufUtil.toMutation(MutationType.PUT, put));
} }
MultiRowMutationService.BlockingInterface service = MultiRowMutationService.BlockingInterface service =

View File

@ -0,0 +1,96 @@
/**
* 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.codec;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.codec.BaseDecoder;
import org.apache.hadoop.hbase.codec.BaseEncoder;
import org.apache.hadoop.hbase.codec.Codec;
import org.apache.hadoop.hbase.codec.CodecException;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import com.google.protobuf.ByteString;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Codec that just writes out Cell as a protobuf Cell Message. Does not write the mvcc stamp.
* Use a different codec if you want that in the stream.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class MessageCodec implements Codec {
static class MessageEncoder extends BaseEncoder {
MessageEncoder(final OutputStream out) {
super(out);
}
@Override
public void write(Cell cell) throws IOException {
checkFlushed();
HBaseProtos.Cell.Builder builder = HBaseProtos.Cell.newBuilder();
// This copies bytes from Cell to ByteString. I don't see anyway around the copy.
// ByteString is final.
builder.setRow(ByteString.copyFrom(cell.getRowArray(), cell.getRowOffset(),
cell.getRowLength()));
builder.setFamily(ByteString.copyFrom(cell.getFamilyArray(), cell.getFamilyOffset(),
cell.getFamilyLength()));
builder.setQualifier(ByteString.copyFrom(cell.getQualifierArray(), cell.getQualifierOffset(),
cell.getQualifierLength()));
builder.setTimestamp(cell.getTimestamp());
builder.setCellType(HBaseProtos.CellType.valueOf(cell.getTypeByte()));
builder.setValue(ByteString.copyFrom(cell.getValueArray(), cell.getValueOffset(),
cell.getValueLength()));
HBaseProtos.Cell pbcell = builder.build();
try {
pbcell.writeDelimitedTo(this.out);
} catch (IOException e) {
throw new CodecException(e);
}
}
}
static class MessageDecoder extends BaseDecoder {
MessageDecoder(final InputStream in) {
super(in);
}
protected Cell parseCell() throws IOException {
HBaseProtos.Cell pbcell = HBaseProtos.Cell.parseDelimitedFrom(this.in);
return CellUtil.createCell(pbcell.getRow().toByteArray(),
pbcell.getFamily().toByteArray(), pbcell.getQualifier().toByteArray(),
pbcell.getTimestamp(), (byte)pbcell.getCellType().getNumber(),
pbcell.getValue().toByteArray());
}
}
@Override
public Decoder getDecoder(InputStream is) {
return new MessageDecoder(is);
}
@Override
public Encoder getEncoder(OutputStream os) {
return new MessageEncoder(os);
}
}

View File

@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.exceptions.WrongRegionException;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.ResponseConverter; import org.apache.hadoop.hbase.protobuf.ResponseConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiMutateRequest; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiMutateRequest;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiMutateResponse; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiMutateResponse;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiRowMutationService; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiRowMutationService;
@ -86,9 +86,9 @@ CoprocessorService, Coprocessor {
try { try {
// set of rows to lock, sorted to avoid deadlocks // set of rows to lock, sorted to avoid deadlocks
SortedSet<byte[]> rowsToLock = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR); SortedSet<byte[]> rowsToLock = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
List<Mutate> mutateRequestList = request.getMutationRequestList(); List<MutationProto> mutateRequestList = request.getMutationRequestList();
List<Mutation> mutations = new ArrayList<Mutation>(mutateRequestList.size()); List<Mutation> mutations = new ArrayList<Mutation>(mutateRequestList.size());
for (Mutate m : mutateRequestList) { for (MutationProto m : mutateRequestList) {
mutations.add(ProtobufUtil.toMutation(m)); mutations.add(ProtobufUtil.toMutation(m));
} }

View File

@ -23,36 +23,33 @@ import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method; import java.lang.reflect.Method;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.IpcProtocol; import org.apache.hadoop.hbase.IpcProtocol;
import org.apache.hadoop.hbase.client.Operation; import org.apache.hadoop.hbase.client.Operation;
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException; import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.security.HBasePolicyProvider; import org.apache.hadoop.hbase.security.HBasePolicyProvider;
import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager; import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.security.authorize.ServiceAuthorizationManager; import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
import org.codehaus.jackson.map.ObjectMapper; import org.codehaus.jackson.map.ObjectMapper;
import com.google.protobuf.Message; import com.google.protobuf.Message;
import com.google.protobuf.ServiceException; import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat;
/** /**
* The {@link RpcServerEngine} implementation for ProtoBuf-based RPCs. * The {@link RpcServerEngine} implementation for ProtoBuf-based RPCs.
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
class ProtobufRpcServerEngine implements RpcServerEngine { class ProtobufRpcServerEngine implements RpcServerEngine {
private static final Log LOG =
LogFactory.getLog("org.apache.hadoop.hbase.ipc.ProtobufRpcServerEngine");
ProtobufRpcServerEngine() { ProtobufRpcServerEngine() {
super(); super();
} }
@ -66,7 +63,6 @@ class ProtobufRpcServerEngine implements RpcServerEngine {
metaHandlerCount, verbose, highPriorityLevel); metaHandlerCount, verbose, highPriorityLevel);
} }
public static class Server extends HBaseServer { public static class Server extends HBaseServer {
boolean verbose; boolean verbose;
Object instance; Object instance;
@ -111,10 +107,6 @@ class ProtobufRpcServerEngine implements RpcServerEngine {
this.instance = instance; this.instance = instance;
this.implementation = instance.getClass(); this.implementation = instance.getClass();
} }
private static final Map<String, Message> methodArg =
new ConcurrentHashMap<String, Message>();
private static final Map<String, Method> methodInstances =
new ConcurrentHashMap<String, Method>();
private AuthenticationTokenSecretManager createSecretManager(){ private AuthenticationTokenSecretManager createSecretManager(){
if (!isSecurityEnabled || if (!isSecurityEnabled ||
@ -152,37 +144,20 @@ class ProtobufRpcServerEngine implements RpcServerEngine {
* the return response has protobuf response payload. On failure, the * the return response has protobuf response payload. On failure, the
* exception name and the stack trace are returned in the protobuf response. * exception name and the stack trace are returned in the protobuf response.
*/ */
public Message call(Class<? extends IpcProtocol> protocol, public Pair<Message, CellScanner> call(Class<? extends IpcProtocol> protocol,
RpcRequestBody rpcRequest, long receiveTime, MonitoredRPCHandler status) Method method, Message param, CellScanner cellScanner, long receiveTime,
MonitoredRPCHandler status)
throws IOException { throws IOException {
try { try {
String methodName = rpcRequest.getMethodName();
Method method = getMethod(protocol, methodName);
if (method == null) {
throw new UnknownProtocolException("Method " + methodName +
" doesn't exist in protocol " + protocol.getName());
}
/**
* RPCs for a particular interface (ie protocol) are done using a
* IPC connection that is setup using rpcProxy.
* The rpcProxy's has a declared protocol name that is
* sent form client to server at connection time.
*/
if (verbose) { if (verbose) {
LOG.info("Call: protocol name=" + protocol.getName() + LOG.info("callId: " + CurCall.get().id + " protocol: " + protocol.getName() +
", method=" + methodName); " method: " + method.getName());
} }
status.setRPC(method.getName(), new Object[]{param}, receiveTime);
status.setRPC(rpcRequest.getMethodName(), // TODO: Review after we add in encoded data blocks.
new Object[]{rpcRequest.getRequest()}, receiveTime); status.setRPCPacket(param);
status.setRPCPacket(rpcRequest);
status.resume("Servicing call"); status.resume("Servicing call");
//get an instance of the method arg type //get an instance of the method arg type
Message protoType = getMethodArgType(method);
Message param = protoType.newBuilderForType()
.mergeFrom(rpcRequest.getRequest()).build();
Message result; Message result;
Object impl = null; Object impl = null;
if (protocol.isAssignableFrom(this.implementation)) { if (protocol.isAssignableFrom(this.implementation)) {
@ -190,57 +165,53 @@ class ProtobufRpcServerEngine implements RpcServerEngine {
} else { } else {
throw new UnknownProtocolException(protocol); throw new UnknownProtocolException(protocol);
} }
PayloadCarryingRpcController controller = null;
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
if (method.getParameterTypes().length == 2) { if (method.getParameterTypes().length == 2) {
// RpcController + Message in the method args // Always create a controller. Some invocations may not pass data in but will pass
// (generated code from RPC bits in .proto files have RpcController) // data out and they'll need a controller instance to carry it for them.
result = (Message)method.invoke(impl, null, param); controller = new PayloadCarryingRpcController(cellScanner);
} else if (method.getParameterTypes().length == 1) { result = (Message)method.invoke(impl, controller, param);
// Message (hand written code usually has only a single argument)
result = (Message)method.invoke(impl, param);
} else { } else {
throw new ServiceException("Too many parameters for method: [" throw new ServiceException("Wrong number of parameters for method: [" +
+ method.getName() + "]" + ", allowed (at most): 2, Actual: " method.getName() + "]" + ", wanted: 2, actual: " + method.getParameterTypes().length);
+ method.getParameterTypes().length);
} }
int processingTime = (int) (System.currentTimeMillis() - startTime); int processingTime = (int) (System.currentTimeMillis() - startTime);
int qTime = (int) (startTime-receiveTime); int qTime = (int) (startTime-receiveTime);
if (TRACELOG.isDebugEnabled()) { if (LOG.isTraceEnabled()) {
TRACELOG.debug("Call #" + CurCall.get().id + LOG.trace(CurCall.get().toString() +
"; served=" + protocol.getSimpleName() + "#" + method.getName() + " response: " + TextFormat.shortDebugString(result) +
", queueTime=" + qTime + " served: " + protocol.getSimpleName() +
", processingTime=" + processingTime + " queueTime: " + qTime +
", request=" + param.toString() + " processingTime: " + processingTime);
" response=" + result.toString());
} }
metrics.dequeuedCall(qTime); metrics.dequeuedCall(qTime);
metrics.processedCall(processingTime); metrics.processedCall(processingTime);
if (verbose) { if (verbose) {
log("Return: "+result, LOG); log("Return " + TextFormat.shortDebugString(result), LOG);
} }
long responseSize = result.getSerializedSize(); long responseSize = result.getSerializedSize();
// log any RPC responses that are slower than the configured warn // log any RPC responses that are slower than the configured warn
// response time or larger than configured warning size // response time or larger than configured warning size
boolean tooSlow = (processingTime > warnResponseTime boolean tooSlow = (processingTime > warnResponseTime && warnResponseTime > -1);
&& warnResponseTime > -1); boolean tooLarge = (responseSize > warnResponseSize && warnResponseSize > -1);
boolean tooLarge = (responseSize > warnResponseSize
&& warnResponseSize > -1);
if (tooSlow || tooLarge) { if (tooSlow || tooLarge) {
// when tagging, we let TooLarge trump TooSmall to keep output simple // when tagging, we let TooLarge trump TooSmall to keep output simple
// note that large responses will often also be slow. // note that large responses will often also be slow.
// TOOD: This output is useless.... output the serialized pb as toString but do a
// short form, shorter than TextFormat.shortDebugString(proto).
StringBuilder buffer = new StringBuilder(256); StringBuilder buffer = new StringBuilder(256);
buffer.append(methodName); buffer.append(method.getName());
buffer.append("("); buffer.append("(");
buffer.append(param.getClass().getName()); buffer.append(param.getClass().getName());
buffer.append(")"); buffer.append(")");
logResponse(new Object[]{rpcRequest.getRequest()}, logResponse(new Object[]{param},
methodName, buffer.toString(), (tooLarge ? "TooLarge" : "TooSlow"), method.getName(), buffer.toString(), (tooLarge ? "TooLarge" : "TooSlow"),
status.getClient(), startTime, processingTime, qTime, status.getClient(), startTime, processingTime, qTime,
responseSize); responseSize);
} }
return result; return new Pair<Message, CellScanner>(result,
controller != null? controller.cellScanner(): null);
} catch (InvocationTargetException e) { } catch (InvocationTargetException e) {
Throwable target = e.getTargetException(); Throwable target = e.getTargetException();
if (target instanceof IOException) { if (target instanceof IOException) {
@ -262,48 +233,6 @@ class ProtobufRpcServerEngine implements RpcServerEngine {
} }
} }
static Method getMethod(Class<? extends IpcProtocol> protocol,
String methodName) {
Method method = methodInstances.get(methodName);
if (method != null) {
return method;
}
Method[] methods = protocol.getMethods();
for (Method m : methods) {
if (m.getName().equals(methodName)) {
m.setAccessible(true);
methodInstances.put(methodName, m);
return m;
}
}
return null;
}
static Message getMethodArgType(Method method) throws Exception {
Message protoType = methodArg.get(method.getName());
if (protoType != null) {
return protoType;
}
Class<?>[] args = method.getParameterTypes();
Class<?> arg;
if (args.length == 2) {
// RpcController + Message in the method args
// (generated code from RPC bits in .proto files have RpcController)
arg = args[1];
} else if (args.length == 1) {
arg = args[0];
} else {
//unexpected
return null;
}
//in the protobuf methods, args[1] is the only significant argument
Method newInstMethod = arg.getMethod("getDefaultInstance");
newInstMethod.setAccessible(true);
protoType = (Message) newInstMethod.invoke(null, (Object[]) null);
methodArg.put(method.getName(), protoType);
return protoType;
}
/** /**
* Logs an RPC response to the LOG file, producing valid JSON objects for * Logs an RPC response to the LOG file, producing valid JSON objects for
* client Operations. * client Operations.
@ -361,10 +290,12 @@ class ProtobufRpcServerEngine implements RpcServerEngine {
mapper.writeValueAsString(responseInfo)); mapper.writeValueAsString(responseInfo));
} }
} }
protected static void log(String value, Log LOG) { protected static void log(String value, Log LOG) {
String v = value; String v = value;
if (v != null && v.length() > 55) final int max = 100;
v = v.substring(0, 55)+"..."; if (v != null && v.length() > max)
v = v.substring(0, max) + "...";
LOG.info(v); LOG.info(v);
} }
} }

View File

@ -19,16 +19,19 @@
package org.apache.hadoop.hbase.ipc; package org.apache.hadoop.hbase.ipc;
import com.google.common.base.Function; import java.io.IOException;
import com.google.protobuf.Message; import java.lang.reflect.Method;
import java.net.InetSocketAddress;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.IpcProtocol; import org.apache.hadoop.hbase.IpcProtocol;
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
import org.apache.hadoop.hbase.util.Pair;
import java.io.IOException; import com.google.common.base.Function;
import java.net.InetSocketAddress; import com.google.protobuf.Message;
@InterfaceAudience.Private @InterfaceAudience.Private
public interface RpcServer { public interface RpcServer {
@ -47,19 +50,19 @@ public interface RpcServer {
InetSocketAddress getListenerAddress(); InetSocketAddress getListenerAddress();
/** Called for each call. /** Called for each call.
* @param method Method to invoke.
* @param param parameter * @param param parameter
* @param receiveTime time * @param receiveTime time
* @return Message Protobuf response Message * @param status
* @return Message Protobuf response Message and optionally the Cells that make up the response.
* @throws java.io.IOException e * @throws java.io.IOException e
*/ */
Message call(Class<? extends IpcProtocol> protocol, Pair<Message, CellScanner> call(Class<? extends IpcProtocol> protocol, Method method,
RpcRequestBody param, long receiveTime, MonitoredRPCHandler status) Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status)
throws IOException; throws IOException;
void setErrorHandler(HBaseRPCErrorHandler handler); void setErrorHandler(HBaseRPCErrorHandler handler);
void setQosFunction(Function<RpcRequestBody, Integer> newFunc);
void openServer(); void openServer();
void startThreads(); void startThreads();
@ -68,4 +71,6 @@ public interface RpcServer {
* Returns the metrics instance for reporting RPC call statistics * Returns the metrics instance for reporting RPC call statistics
*/ */
MetricsHBaseServer getMetrics(); MetricsHBaseServer getMetrics();
public void setQosFunction(Function<Pair<RequestHeader, Message>, Integer> newFunc);
} }

View File

@ -17,13 +17,13 @@
*/ */
package org.apache.hadoop.hbase.mapreduce; package org.apache.hadoop.hbase.mapreduce;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.OutputStream; import java.io.OutputStream;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.io.serializer.Deserializer; import org.apache.hadoop.io.serializer.Deserializer;
import org.apache.hadoop.io.serializer.Serialization; import org.apache.hadoop.io.serializer.Serialization;
import org.apache.hadoop.io.serializer.Serializer; import org.apache.hadoop.io.serializer.Serializer;
@ -45,43 +45,41 @@ public class KeyValueSerialization implements Serialization<KeyValue> {
} }
public static class KeyValueDeserializer implements Deserializer<KeyValue> { public static class KeyValueDeserializer implements Deserializer<KeyValue> {
private InputStream is; private DataInputStream dis;
@Override @Override
public void close() throws IOException { public void close() throws IOException {
this.is.close(); this.dis.close();
} }
@Override @Override
public KeyValue deserialize(KeyValue ignore) throws IOException { public KeyValue deserialize(KeyValue ignore) throws IOException {
// I can't overwrite the passed in KV, not from a proto kv, not just yet. TODO // I can't overwrite the passed in KV, not from a proto kv, not just yet. TODO
HBaseProtos.KeyValue proto = return KeyValue.create(this.dis);
HBaseProtos.KeyValue.parseDelimitedFrom(this.is);
return ProtobufUtil.toKeyValue(proto);
} }
@Override @Override
public void open(InputStream is) throws IOException { public void open(InputStream is) throws IOException {
this.is = is; this.dis = new DataInputStream(is);
} }
} }
public static class KeyValueSerializer implements Serializer<KeyValue> { public static class KeyValueSerializer implements Serializer<KeyValue> {
private OutputStream os; private DataOutputStream dos;
@Override @Override
public void close() throws IOException { public void close() throws IOException {
this.os.close(); this.dos.close();
} }
@Override @Override
public void open(OutputStream os) throws IOException { public void open(OutputStream os) throws IOException {
this.os = os; this.dos = new DataOutputStream(os);
} }
@Override @Override
public void serialize(KeyValue kv) throws IOException { public void serialize(KeyValue kv) throws IOException {
ProtobufUtil.toKeyValue(kv).writeDelimitedTo(this.os); KeyValue.write(kv, this.dos);
} }
} }
} }

View File

@ -25,8 +25,8 @@ import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.io.serializer.Deserializer; import org.apache.hadoop.io.serializer.Deserializer;
import org.apache.hadoop.io.serializer.Serialization; import org.apache.hadoop.io.serializer.Serialization;
import org.apache.hadoop.io.serializer.Serializer; import org.apache.hadoop.io.serializer.Serializer;
@ -57,7 +57,7 @@ public class MutationSerialization implements Serialization<Mutation> {
@Override @Override
public Mutation deserialize(Mutation mutation) throws IOException { public Mutation deserialize(Mutation mutation) throws IOException {
Mutate proto = Mutate.parseDelimitedFrom(in); MutationProto proto = MutationProto.parseDelimitedFrom(in);
return ProtobufUtil.toMutation(proto); return ProtobufUtil.toMutation(proto);
} }
@ -82,15 +82,15 @@ public class MutationSerialization implements Serialization<Mutation> {
@Override @Override
public void serialize(Mutation mutation) throws IOException { public void serialize(Mutation mutation) throws IOException {
MutateType type; MutationType type;
if (mutation instanceof Put) { if (mutation instanceof Put) {
type = MutateType.PUT; type = MutationType.PUT;
} else if (mutation instanceof Delete) { } else if (mutation instanceof Delete) {
type = MutateType.DELETE; type = MutationType.DELETE;
} else { } else {
throw new IllegalArgumentException("Only Put and Delete are supported"); throw new IllegalArgumentException("Only Put and Delete are supported");
} }
ProtobufUtil.toMutate(type, mutation).writeDelimitedTo(out); ProtobufUtil.toMutation(type, mutation).writeDelimitedTo(out);
} }
} }
} }

View File

@ -19,7 +19,8 @@
package org.apache.hadoop.hbase.monitoring; package org.apache.hadoop.hbase.monitoring;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody;
import com.google.protobuf.Message;
/** /**
* A MonitoredTask implementation optimized for use with RPC Handlers * A MonitoredTask implementation optimized for use with RPC Handlers
@ -37,8 +38,7 @@ public interface MonitoredRPCHandler extends MonitoredTask {
public abstract boolean isRPCRunning(); public abstract boolean isRPCRunning();
public abstract boolean isOperationRunning(); public abstract boolean isOperationRunning();
public abstract void setRPC(String methodName, Object [] params, public abstract void setRPC(String methodName, Object [] params, long queueTime);
long queueTime); public abstract void setRPCPacket(Message param);
public abstract void setRPCPacket(RpcRequestBody param);
public abstract void setConnection(String clientAddress, int remotePort); public abstract void setConnection(String clientAddress, int remotePort);
} }

View File

@ -18,20 +18,16 @@
*/ */
package org.apache.hadoop.hbase.monitoring; package org.apache.hadoop.hbase.monitoring;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Operation;
import org.apache.hadoop.hbase.io.WritableWithSize;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.Writable;
import org.codehaus.jackson.map.ObjectMapper;
import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Operation;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.protobuf.Message;
/** /**
* A MonitoredTask implementation designed for use with RPC Handlers * A MonitoredTask implementation designed for use with RPC Handlers
* handling frequent, short duration tasks. String concatenations and object * handling frequent, short duration tasks. String concatenations and object
@ -46,7 +42,7 @@ public class MonitoredRPCHandlerImpl extends MonitoredTaskImpl
private long rpcStartTime; private long rpcStartTime;
private String methodName = ""; private String methodName = "";
private Object [] params = {}; private Object [] params = {};
private RpcRequestBody packet; private Message packet;
public MonitoredRPCHandlerImpl() { public MonitoredRPCHandlerImpl() {
super(); super();
@ -201,7 +197,7 @@ public class MonitoredRPCHandlerImpl extends MonitoredTaskImpl
* that it can later compute its size if asked for it. * that it can later compute its size if asked for it.
* @param param The protobuf received by the RPC for this call * @param param The protobuf received by the RPC for this call
*/ */
public void setRPCPacket(RpcRequestBody param) { public void setRPCPacket(Message param) {
this.packet = param; this.packet = param;
} }

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hbase.errorhandling.ForeignException; import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
@ -205,8 +206,9 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs {
byte[] data = ZKUtil.getData(zkController.getWatcher(), path); byte[] data = ZKUtil.getData(zkController.getWatcher(), path);
LOG.debug("start proc data length is " + data.length); LOG.debug("start proc data length is " + data.length);
if (!ProtobufUtil.isPBMagicPrefix(data)) { if (!ProtobufUtil.isPBMagicPrefix(data)) {
String msg = "Data in for starting procuedure " + opName + " is illegally formatted. " String msg = "Data in for starting procuedure " + opName +
+ "Killing the procedure."; " is illegally formatted (no pb magic). " +
"Killing the procedure: " + Bytes.toString(data);
LOG.error(msg); LOG.error(msg);
throw new IllegalArgumentException(msg); throw new IllegalArgumentException(msg);
} }

View File

@ -25,7 +25,6 @@ import java.lang.annotation.RetentionPolicy;
import java.lang.management.ManagementFactory; import java.lang.management.ManagementFactory;
import java.lang.management.MemoryUsage; import java.lang.management.MemoryUsage;
import java.lang.reflect.Constructor; import java.lang.reflect.Constructor;
import java.lang.reflect.Method;
import java.net.BindException; import java.net.BindException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.util.ArrayList; import java.util.ArrayList;
@ -58,14 +57,17 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Chore; import org.apache.hadoop.hbase.Chore;
import org.apache.hadoop.hbase.exceptions.ClockOutOfSyncException; import org.apache.hadoop.hbase.exceptions.ClockOutOfSyncException;
import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException; import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HealthCheckChore; import org.apache.hadoop.hbase.HealthCheckChore;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
import org.apache.hadoop.hbase.exceptions.LeaseException; import org.apache.hadoop.hbase.exceptions.LeaseException;
import org.apache.hadoop.hbase.exceptions.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.exceptions.NoSuchColumnFamilyException;
import org.apache.hadoop.hbase.exceptions.NotServingRegionException; import org.apache.hadoop.hbase.exceptions.NotServingRegionException;
@ -107,6 +109,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.ipc.HBaseClientRPC; import org.apache.hadoop.hbase.ipc.HBaseClientRPC;
import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler; import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
import org.apache.hadoop.hbase.ipc.HBaseServerRPC; import org.apache.hadoop.hbase.ipc.HBaseServerRPC;
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
import org.apache.hadoop.hbase.ipc.ProtobufRpcClientEngine; import org.apache.hadoop.hbase.ipc.ProtobufRpcClientEngine;
import org.apache.hadoop.hbase.ipc.RpcClientEngine; import org.apache.hadoop.hbase.ipc.RpcClientEngine;
import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.ipc.RpcServer;
@ -156,10 +159,10 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate;
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.MutateRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; 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.ScanResponse;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@ -168,7 +171,6 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionLoad; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionLoad;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
@ -212,11 +214,11 @@ import org.apache.hadoop.util.StringUtils;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import org.cliffc.high_scale_lib.Counter; import org.cliffc.high_scale_lib.Counter;
import com.google.common.base.Function;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
import com.google.protobuf.Message; import com.google.protobuf.Message;
import com.google.protobuf.RpcController; import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException; import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat;
/** /**
* HRegionServer makes a set of HRegions available to clients. It checks in with * HRegionServer makes a set of HRegions available to clients. It checks in with
@ -458,8 +460,7 @@ public class HRegionServer implements ClientProtocol,
// Config'ed params // Config'ed params
this.numRetries = conf.getInt("hbase.client.retries.number", 10); this.numRetries = conf.getInt("hbase.client.retries.number", 10);
this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
10 * 1000);
this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000); this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000);
this.sleeper = new Sleeper(this.msgInterval, this); this.sleeper = new Sleeper(this.msgInterval, this);
@ -507,7 +508,7 @@ public class HRegionServer implements ClientProtocol,
this.isa = this.rpcServer.getListenerAddress(); this.isa = this.rpcServer.getListenerAddress();
this.rpcServer.setErrorHandler(this); this.rpcServer.setErrorHandler(this);
this.rpcServer.setQosFunction((qosFunction = new QosFunction())); this.rpcServer.setQosFunction((qosFunction = new QosFunction(this)));
this.startcode = System.currentTimeMillis(); this.startcode = System.currentTimeMillis();
// login the zookeeper client principal (if using security) // login the zookeeper client principal (if using security)
@ -566,152 +567,6 @@ public class HRegionServer implements ClientProtocol,
return null; return null;
} }
/**
* Utility used ensuring higher quality of service for priority rpcs; e.g.
* rpcs to .META., etc.
*/
class QosFunction implements Function<RpcRequestBody,Integer> {
private final Map<String, Integer> annotatedQos;
//We need to mock the regionserver instance for some unit tests (set via
//setRegionServer method.
//The field value is initially set to the enclosing instance of HRegionServer.
private HRegionServer hRegionServer = HRegionServer.this;
//The logic for figuring out high priority RPCs is as follows:
//1. if the method is annotated with a QosPriority of QOS_HIGH,
// that is honored
//2. parse out the protobuf message and see if the request is for meta
// region, and if so, treat it as a high priority RPC
//Some optimizations for (2) are done here -
//Clients send the argument classname as part of making the RPC. The server
//decides whether to deserialize the proto argument message based on the
//pre-established set of argument classes (knownArgumentClasses below).
//This prevents the server from having to deserialize all proto argument
//messages prematurely.
//All the argument classes declare a 'getRegion' method that returns a
//RegionSpecifier object. Methods can be invoked on the returned object
//to figure out whether it is a meta region or not.
@SuppressWarnings("unchecked")
private final Class<? extends Message>[] knownArgumentClasses = new Class[]{
GetRegionInfoRequest.class,
GetStoreFileRequest.class,
CloseRegionRequest.class,
FlushRegionRequest.class,
SplitRegionRequest.class,
CompactRegionRequest.class,
GetRequest.class,
MutateRequest.class,
ScanRequest.class,
MultiRequest.class
};
//Some caches for helping performance
private final Map<String, Class<? extends Message>> argumentToClassMap =
new HashMap<String, Class<? extends Message>>();
private final Map<String, Map<Class<? extends Message>, Method>>
methodMap = new HashMap<String, Map<Class<? extends Message>, Method>>();
public QosFunction() {
Map<String, Integer> qosMap = new HashMap<String, Integer>();
for (Method m : HRegionServer.class.getMethods()) {
QosPriority p = m.getAnnotation(QosPriority.class);
if (p != null) {
qosMap.put(m.getName(), p.priority());
}
}
annotatedQos = qosMap;
if (methodMap.get("parseFrom") == null) {
methodMap.put("parseFrom",
new HashMap<Class<? extends Message>, Method>());
}
if (methodMap.get("getRegion") == null) {
methodMap.put("getRegion",
new HashMap<Class<? extends Message>, Method>());
}
for (Class<? extends Message> cls : knownArgumentClasses) {
argumentToClassMap.put(cls.getCanonicalName(), cls);
try {
methodMap.get("parseFrom").put(cls,
cls.getDeclaredMethod("parseFrom",ByteString.class));
methodMap.get("getRegion").put(cls, cls.getDeclaredMethod("getRegion"));
} catch (Exception e) {
throw new RuntimeException(e);
}
}
}
void setRegionServer(HRegionServer server) {
this.hRegionServer = server;
}
public boolean isMetaRegion(byte[] regionName) {
HRegion region;
try {
region = hRegionServer.getRegion(regionName);
} catch (NotServingRegionException ignored) {
return false;
}
return region.getRegionInfo().isMetaRegion();
}
@Override
public Integer apply(RpcRequestBody from) {
String methodName = from.getMethodName();
Class<? extends Message> rpcArgClass = null;
if (from.hasRequestClassName()) {
String cls = from.getRequestClassName();
rpcArgClass = argumentToClassMap.get(cls);
}
Integer priorityByAnnotation = annotatedQos.get(methodName);
if (priorityByAnnotation != null) {
return priorityByAnnotation;
}
if (rpcArgClass == null || from.getRequest().isEmpty()) {
return HConstants.NORMAL_QOS;
}
Object deserializedRequestObj;
//check whether the request has reference to Meta region
try {
Method parseFrom = methodMap.get("parseFrom").get(rpcArgClass);
deserializedRequestObj = parseFrom.invoke(null, from.getRequest());
Method getRegion = methodMap.get("getRegion").get(rpcArgClass);
RegionSpecifier regionSpecifier =
(RegionSpecifier)getRegion.invoke(deserializedRequestObj,
(Object[])null);
HRegion region = hRegionServer.getRegion(regionSpecifier);
if (region.getRegionInfo().isMetaTable()) {
if (LOG.isDebugEnabled()) {
LOG.debug("High priority: " + from.toString());
}
return HConstants.HIGH_QOS;
}
} catch (Exception ex) {
throw new RuntimeException(ex);
}
if (methodName.equals("scan")) { // scanner methods...
ScanRequest request = (ScanRequest)deserializedRequestObj;
if (!request.hasScannerId()) {
return HConstants.NORMAL_QOS;
}
RegionScanner scanner = hRegionServer.getScanner(request.getScannerId());
if (scanner != null && scanner.getRegionInfo().isMetaTable()) {
if (LOG.isDebugEnabled()) {
LOG.debug("High priority scanner request: " + request.getScannerId());
}
return HConstants.HIGH_QOS;
}
}
if (LOG.isDebugEnabled()) {
LOG.debug("Low priority: " + from.toString());
}
return HConstants.NORMAL_QOS;
}
}
/** /**
* All initialization needed before we go register with Master. * All initialization needed before we go register with Master.
* *
@ -1749,15 +1604,15 @@ public class HRegionServer implements ClientProtocol,
if (cause != null) { if (cause != null) {
msg += "\nCause:\n" + StringUtils.stringifyException(cause); msg += "\nCause:\n" + StringUtils.stringifyException(cause);
} }
if (hbaseMaster != null) { // Report to the master but only if we have already registered with the master.
if (hbaseMaster != null && this.serverNameFromMasterPOV != null) {
ReportRSFatalErrorRequest.Builder builder = ReportRSFatalErrorRequest.Builder builder =
ReportRSFatalErrorRequest.newBuilder(); ReportRSFatalErrorRequest.newBuilder();
ServerName sn = ServerName sn =
ServerName.parseVersionedServerName(this.serverNameFromMasterPOV.getVersionedBytes()); ServerName.parseVersionedServerName(this.serverNameFromMasterPOV.getVersionedBytes());
builder.setServer(ProtobufUtil.toServerName(sn)); builder.setServer(ProtobufUtil.toServerName(sn));
builder.setErrorMessage(msg); builder.setErrorMessage(msg);
hbaseMaster.reportRSFatalError( hbaseMaster.reportRSFatalError(null, builder.build());
null,builder.build());
} }
} catch (Throwable t) { } catch (Throwable t) {
LOG.warn("Unable to report fatal error to master", t); LOG.warn("Unable to report fatal error to master", t);
@ -2805,33 +2660,39 @@ public class HRegionServer implements ClientProtocol,
/** /**
* Mutate data in a table. * Mutate data in a table.
* *
* @param controller the RPC controller * @param rpcc the RPC controller
* @param request the mutate request * @param request the mutate request
* @throws ServiceException * @throws ServiceException
*/ */
@Override @Override
public MutateResponse mutate(final RpcController controller, public MutateResponse mutate(final RpcController rpcc,
final MutateRequest request) throws ServiceException { final MutateRequest request) throws ServiceException {
// rpc controller is how we bring in data via the back door; it is unprotobuf'ed data.
// It is also the conduit via which we pass back data.
PayloadCarryingRpcController controller = (PayloadCarryingRpcController)rpcc;
CellScanner cellScanner = controller != null? controller.cellScanner(): null;
// Clear scanner so we are not holding on to reference across call.
controller.setCellScanner(null);
try { try {
requestCount.increment(); requestCount.increment();
HRegion region = getRegion(request.getRegion()); HRegion region = getRegion(request.getRegion());
MutateResponse.Builder builder = MutateResponse.newBuilder(); MutateResponse.Builder builder = MutateResponse.newBuilder();
Mutate mutate = request.getMutate(); MutationProto mutation = request.getMutation();
if (!region.getRegionInfo().isMetaTable()) { if (!region.getRegionInfo().isMetaTable()) {
cacheFlusher.reclaimMemStoreMemory(); cacheFlusher.reclaimMemStoreMemory();
} }
Result r = null; Result r = null;
Boolean processed = null; Boolean processed = null;
MutateType type = mutate.getMutateType(); MutationType type = mutation.getMutateType();
switch (type) { switch (type) {
case APPEND: case APPEND:
r = append(region, mutate); r = append(region, mutation, cellScanner);
break; break;
case INCREMENT: case INCREMENT:
r = increment(region, mutate); r = increment(region, mutation, cellScanner);
break; break;
case PUT: case PUT:
Put put = ProtobufUtil.toPut(mutate); Put put = ProtobufUtil.toPut(mutation, cellScanner);
if (request.hasCondition()) { if (request.hasCondition()) {
Condition condition = request.getCondition(); Condition condition = request.getCondition();
byte[] row = condition.getRow().toByteArray(); byte[] row = condition.getRow().toByteArray();
@ -2859,7 +2720,7 @@ public class HRegionServer implements ClientProtocol,
} }
break; break;
case DELETE: case DELETE:
Delete delete = ProtobufUtil.toDelete(mutate); Delete delete = ProtobufUtil.toDelete(mutation, cellScanner);
if (request.hasCondition()) { if (request.hasCondition()) {
Condition condition = request.getCondition(); Condition condition = request.getCondition();
byte[] row = condition.getRow().toByteArray(); byte[] row = condition.getRow().toByteArray();
@ -2890,10 +2751,15 @@ public class HRegionServer implements ClientProtocol,
throw new DoNotRetryIOException( throw new DoNotRetryIOException(
"Unsupported mutate type: " + type.name()); "Unsupported mutate type: " + type.name());
} }
CellScannable cellsToReturn = null;
if (processed != null) { if (processed != null) {
builder.setProcessed(processed.booleanValue()); builder.setProcessed(processed.booleanValue());
} else if (r != null) { } else if (r != null) {
builder.setResult(ProtobufUtil.toResult(r)); builder.setResult(ProtobufUtil.toResultNoData(r));
cellsToReturn = r;
}
if (cellsToReturn != null) {
controller.setCellScanner(cellsToReturn.cellScanner());
} }
return builder.build(); return builder.build();
} catch (IOException ie) { } catch (IOException ie) {
@ -3006,7 +2872,8 @@ public class HRegionServer implements ClientProtocol,
if (rsh != null) { if (rsh != null) {
if (request.getNextCallSeq() != rsh.nextCallSeq) { if (request.getNextCallSeq() != rsh.nextCallSeq) {
throw new OutOfOrderScannerNextException("Expected nextCallSeq: " + rsh.nextCallSeq throw new OutOfOrderScannerNextException("Expected nextCallSeq: " + rsh.nextCallSeq
+ " But the nextCallSeq got from client: " + request.getNextCallSeq()); + " But the nextCallSeq got from client: " + request.getNextCallSeq() +
"; request=" + TextFormat.shortDebugString(request));
} }
// Increment the nextCallSeq value which is the next expected from client. // Increment the nextCallSeq value which is the next expected from client.
rsh.nextCallSeq++; rsh.nextCallSeq++;
@ -3208,47 +3075,61 @@ public class HRegionServer implements ClientProtocol,
/** /**
* Execute multiple actions on a table: get, mutate, and/or execCoprocessor * Execute multiple actions on a table: get, mutate, and/or execCoprocessor
* *
* @param controller the RPC controller * @param rpcc the RPC controller
* @param request the multi request * @param request the multi request
* @throws ServiceException * @throws ServiceException
*/ */
@Override @Override
public MultiResponse multi(final RpcController controller, public MultiResponse multi(final RpcController rpcc, final MultiRequest request)
final MultiRequest request) throws ServiceException { throws ServiceException {
// rpc controller is how we bring in data via the back door; it is unprotobuf'ed data.
// It is also the conduit via which we pass back data.
PayloadCarryingRpcController controller = (PayloadCarryingRpcController)rpcc;
CellScanner cellScanner = controller != null? controller.cellScanner(): null;
// Clear scanner so we are not holding on to reference across call.
controller.setCellScanner(null);
List<CellScannable> cellsToReturn = null;
try { try {
HRegion region = getRegion(request.getRegion()); HRegion region = getRegion(request.getRegion());
MultiResponse.Builder builder = MultiResponse.newBuilder(); MultiResponse.Builder builder = MultiResponse.newBuilder();
List<MutationProto> mutations = new ArrayList<MutationProto>(request.getActionCount());
// Do a bunch of mutations atomically. Mutations are Puts and Deletes. NOT Gets.
if (request.hasAtomic() && request.getAtomic()) { if (request.hasAtomic() && request.getAtomic()) {
List<Mutate> mutates = new ArrayList<Mutate>(); // MultiAction is union type. Has a Get or a Mutate.
for (ClientProtos.MultiAction actionUnion : request.getActionList()) { for (ClientProtos.MultiAction actionUnion : request.getActionList()) {
if (actionUnion.hasMutate()) { if (actionUnion.hasMutation()) {
mutates.add(actionUnion.getMutate()); mutations.add(actionUnion.getMutation());
} else { } else {
throw new DoNotRetryIOException( throw new DoNotRetryIOException("Unsupported atomic action type: " + actionUnion);
"Unsupported atomic action type: " + actionUnion);
} }
} }
mutateRows(region, mutates); // TODO: We are not updating a metric here. Should we up requestCount?
if (!mutations.isEmpty()) mutateRows(region, mutations, cellScanner);
} else { } else {
// Do a bunch of Actions.
ActionResult.Builder resultBuilder = null; ActionResult.Builder resultBuilder = null;
List<Mutate> mutates = new ArrayList<Mutate>(); cellsToReturn = new ArrayList<CellScannable>(request.getActionCount());
for (ClientProtos.MultiAction actionUnion : request.getActionList()) { for (ClientProtos.MultiAction actionUnion : request.getActionList()) {
requestCount.increment(); this.requestCount.increment();
try {
ClientProtos.Result result = null; ClientProtos.Result result = null;
try {
if (actionUnion.hasGet()) { if (actionUnion.hasGet()) {
Get get = ProtobufUtil.toGet(actionUnion.getGet()); Get get = ProtobufUtil.toGet(actionUnion.getGet());
Result r = region.get(get); Result r = region.get(get);
if (r != null) { if (r != null) {
result = ProtobufUtil.toResult(r); // Get a result with no data. The data will be carried alongside pbs, not as pbs.
result = ProtobufUtil.toResultNoData(r);
// Add the Result to controller so it gets serialized apart from pb. Get
// Results could be big so good if they are not serialized as pb.
cellsToReturn.add(r);
} }
} else if (actionUnion.hasMutate()) { } else if (actionUnion.hasMutation()) {
Mutate mutate = actionUnion.getMutate(); MutationProto mutation = actionUnion.getMutation();
MutateType type = mutate.getMutateType(); MutationType type = mutation.getMutateType();
if (type != MutateType.PUT && type != MutateType.DELETE) { if (type != MutationType.PUT && type != MutationType.DELETE) {
if (!mutates.isEmpty()) { if (!mutations.isEmpty()) {
doBatchOp(builder, region, mutates); doBatchOp(builder, region, mutations, cellScanner);
mutates.clear(); mutations.clear();
} else if (!region.getRegionInfo().isMetaTable()) { } else if (!region.getRegionInfo().isMetaTable()) {
cacheFlusher.reclaimMemStoreMemory(); cacheFlusher.reclaimMemStoreMemory();
} }
@ -3256,22 +3137,23 @@ public class HRegionServer implements ClientProtocol,
Result r = null; Result r = null;
switch (type) { switch (type) {
case APPEND: case APPEND:
r = append(region, mutate); r = append(region, mutation, cellScanner);
break; break;
case INCREMENT: case INCREMENT:
r = increment(region, mutate); r = increment(region, mutation, cellScanner);
break; break;
case PUT: case PUT:
mutates.add(mutate);
break;
case DELETE: case DELETE:
mutates.add(mutate); mutations.add(mutation);
break; break;
default: default:
throw new DoNotRetryIOException("Unsupported mutate type: " + type.name()); throw new DoNotRetryIOException("Unsupported mutate type: " + type.name());
} }
if (r != null) { if (r != null) {
result = ProtobufUtil.toResult(r); // Put the data into the cellsToReturn and the metadata about the result is all that
// we will pass back in the protobuf result.
result = ProtobufUtil.toResultNoData(r);
cellsToReturn.add(r);
} }
} else { } else {
LOG.warn("Error: invalid action: " + actionUnion + ". " LOG.warn("Error: invalid action: " + actionUnion + ". "
@ -3292,10 +3174,14 @@ public class HRegionServer implements ClientProtocol,
builder.addResult(ResponseConverter.buildActionResult(ie)); builder.addResult(ResponseConverter.buildActionResult(ie));
} }
} }
if (!mutates.isEmpty()) { if (!mutations.isEmpty()) {
doBatchOp(builder, region, mutates); doBatchOp(builder, region, mutations, cellScanner);
} }
} }
// Load the controller with the Cells to return.
if (cellsToReturn != null && !cellsToReturn.isEmpty()) {
controller.setCellScanner(CellUtil.createCellScanner(cellsToReturn));
}
return builder.build(); return builder.build();
} catch (IOException ie) { } catch (IOException ie) {
throw new ServiceException(ie); throw new ServiceException(ie);
@ -3758,15 +3644,16 @@ public class HRegionServer implements ClientProtocol,
* Execute an append mutation. * Execute an append mutation.
* *
* @param region * @param region
* @param mutate * @param m
* @param cellScanner
* @return result to return to client if default operation should be * @return result to return to client if default operation should be
* bypassed as indicated by RegionObserver, null otherwise * bypassed as indicated by RegionObserver, null otherwise
* @throws IOException * @throws IOException
*/ */
protected Result append(final HRegion region, protected Result append(final HRegion region,
final Mutate mutate) throws IOException { final MutationProto m, final CellScanner cellScanner) throws IOException {
long before = EnvironmentEdgeManager.currentTimeMillis(); long before = EnvironmentEdgeManager.currentTimeMillis();
Append append = ProtobufUtil.toAppend(mutate); Append append = ProtobufUtil.toAppend(m, cellScanner);
Result r = null; Result r = null;
if (region.getCoprocessorHost() != null) { if (region.getCoprocessorHost() != null) {
r = region.getCoprocessorHost().preAppend(append); r = region.getCoprocessorHost().preAppend(append);
@ -3785,14 +3672,15 @@ public class HRegionServer implements ClientProtocol,
* Execute an increment mutation. * Execute an increment mutation.
* *
* @param region * @param region
* @param mutate * @param mutation
* @return the Result * @return the Result
* @throws IOException * @throws IOException
*/ */
protected Result increment(final HRegion region, protected Result increment(final HRegion region, final MutationProto mutation,
final Mutate mutate) throws IOException { final CellScanner cells)
throws IOException {
long before = EnvironmentEdgeManager.currentTimeMillis(); long before = EnvironmentEdgeManager.currentTimeMillis();
Increment increment = ProtobufUtil.toIncrement(mutate); Increment increment = ProtobufUtil.toIncrement(mutation, cells);
Result r = null; Result r = null;
if (region.getCoprocessorHost() != null) { if (region.getCoprocessorHost() != null) {
r = region.getCoprocessorHost().preIncrement(increment); r = region.getCoprocessorHost().preIncrement(increment);
@ -3812,12 +3700,12 @@ public class HRegionServer implements ClientProtocol,
* *
* @param builder * @param builder
* @param region * @param region
* @param mutates * @param mutations
*/ */
protected void doBatchOp(final MultiResponse.Builder builder, protected void doBatchOp(final MultiResponse.Builder builder, final HRegion region,
final HRegion region, final List<Mutate> mutates) { final List<MutationProto> mutations, final CellScanner cells) {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
Pair<Mutation, Integer>[] mutationsWithLocks = new Pair[mutates.size()]; Pair<Mutation, Integer>[] mutationsWithLocks = new Pair[mutations.size()];
long before = EnvironmentEdgeManager.currentTimeMillis(); long before = EnvironmentEdgeManager.currentTimeMillis();
boolean batchContainsPuts = false, batchContainsDelete = false; boolean batchContainsPuts = false, batchContainsDelete = false;
try { try {
@ -3825,21 +3713,20 @@ public class HRegionServer implements ClientProtocol,
resultBuilder.setValue(ClientProtos.Result.newBuilder().build()); resultBuilder.setValue(ClientProtos.Result.newBuilder().build());
ActionResult result = resultBuilder.build(); ActionResult result = resultBuilder.build();
int i = 0; int i = 0;
for (Mutate m : mutates) { for (MutationProto m : mutations) {
Mutation mutation; Mutation mutation;
if (m.getMutateType() == MutateType.PUT) { if (m.getMutateType() == MutationType.PUT) {
mutation = ProtobufUtil.toPut(m); mutation = ProtobufUtil.toPut(m, cells);
batchContainsPuts = true; batchContainsPuts = true;
} else { } else {
mutation = ProtobufUtil.toDelete(m); mutation = ProtobufUtil.toDelete(m, cells);
batchContainsDelete = true; batchContainsDelete = true;
} }
mutationsWithLocks[i++] = new Pair<Mutation, Integer>(mutation, null); mutationsWithLocks[i++] = new Pair<Mutation, Integer>(mutation, null);
builder.addResult(result); builder.addResult(result);
} }
requestCount.add(mutations.size());
requestCount.add(mutates.size());
if (!region.getRegionInfo().isMetaTable()) { if (!region.getRegionInfo().isMetaTable()) {
cacheFlusher.reclaimMemStoreMemory(); cacheFlusher.reclaimMemStoreMemory();
} }
@ -3871,7 +3758,7 @@ public class HRegionServer implements ClientProtocol,
} }
} catch (IOException ie) { } catch (IOException ie) {
ActionResult result = ResponseConverter.buildActionResult(ie); ActionResult result = ResponseConverter.buildActionResult(ie);
for (int i = 0, n = mutates.size(); i < n; i++) { for (int i = 0; i < mutations.size(); i++) {
builder.setResult(i, result); builder.setResult(i, result);
} }
} }
@ -3888,25 +3775,27 @@ public class HRegionServer implements ClientProtocol,
* Mutate a list of rows atomically. * Mutate a list of rows atomically.
* *
* @param region * @param region
* @param mutates * @param mutations
* @param cellScanner if non-null, the mutation data -- the Cell content.
* @throws IOException * @throws IOException
*/ */
protected void mutateRows(final HRegion region, protected void mutateRows(final HRegion region, final List<MutationProto> mutations,
final List<Mutate> mutates) throws IOException { final CellScanner cellScanner)
Mutate firstMutate = mutates.get(0); throws IOException {
MutationProto firstMutate = mutations.get(0);
if (!region.getRegionInfo().isMetaTable()) { if (!region.getRegionInfo().isMetaTable()) {
cacheFlusher.reclaimMemStoreMemory(); cacheFlusher.reclaimMemStoreMemory();
} }
byte [] row = firstMutate.getRow().toByteArray(); byte [] row = firstMutate.getRow().toByteArray();
RowMutations rm = new RowMutations(row); RowMutations rm = new RowMutations(row);
for (Mutate mutate: mutates) { for (MutationProto mutate: mutations) {
MutateType type = mutate.getMutateType(); MutationType type = mutate.getMutateType();
switch (mutate.getMutateType()) { switch (mutate.getMutateType()) {
case PUT: case PUT:
rm.add(ProtobufUtil.toPut(mutate)); rm.add(ProtobufUtil.toPut(mutate, cellScanner));
break; break;
case DELETE: case DELETE:
rm.add(ProtobufUtil.toDelete(mutate)); rm.add(ProtobufUtil.toDelete(mutate, cellScanner));
break; break;
default: default:
throw new DoNotRetryIOException( throw new DoNotRetryIOException(

View File

@ -0,0 +1,196 @@
/**
* 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.regionserver;
import java.lang.reflect.Method;
import java.util.HashMap;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.exceptions.NotServingRegionException;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetStoreFileRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
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.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
import org.apache.hadoop.hbase.regionserver.HRegionServer.QosPriority;
import org.apache.hadoop.hbase.util.Pair;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.protobuf.Message;
import com.google.protobuf.TextFormat;
/**
* A guava function that will return a priority for use by QoS facility in regionserver; e.g.
* rpcs to .META. and -ROOT-, etc., get priority.
*/
// TODO: Remove. This is doing way too much work just to figure a priority. Do as Elliott
// suggests and just have the client specify a priority.
//The logic for figuring out high priority RPCs is as follows:
//1. if the method is annotated with a QosPriority of QOS_HIGH,
// that is honored
//2. parse out the protobuf message and see if the request is for meta
// region, and if so, treat it as a high priority RPC
//Some optimizations for (2) are done here -
//Clients send the argument classname as part of making the RPC. The server
//decides whether to deserialize the proto argument message based on the
//pre-established set of argument classes (knownArgumentClasses below).
//This prevents the server from having to deserialize all proto argument
//messages prematurely.
//All the argument classes declare a 'getRegion' method that returns a
//RegionSpecifier object. Methods can be invoked on the returned object
//to figure out whether it is a meta region or not.
class QosFunction implements Function<Pair<RequestHeader, Message>, Integer> {
public static final Log LOG = LogFactory.getLog(QosFunction.class.getName());
private final Map<String, Integer> annotatedQos;
//We need to mock the regionserver instance for some unit tests (set via
//setRegionServer method.
private HRegionServer hRegionServer;
@SuppressWarnings("unchecked")
private final Class<? extends Message>[] knownArgumentClasses = new Class[]{
GetRegionInfoRequest.class,
GetStoreFileRequest.class,
CloseRegionRequest.class,
FlushRegionRequest.class,
SplitRegionRequest.class,
CompactRegionRequest.class,
GetRequest.class,
MutateRequest.class,
ScanRequest.class,
MultiRequest.class
};
// Some caches for helping performance
private final Map<String, Class<? extends Message>> argumentToClassMap =
new HashMap<String, Class<? extends Message>>();
private final Map<String, Map<Class<? extends Message>, Method>> methodMap =
new HashMap<String, Map<Class<? extends Message>, Method>>();
QosFunction(final HRegionServer hrs) {
this.hRegionServer = hrs;
Map<String, Integer> qosMap = new HashMap<String, Integer>();
for (Method m : HRegionServer.class.getMethods()) {
QosPriority p = m.getAnnotation(QosPriority.class);
if (p != null) {
qosMap.put(m.getName(), p.priority());
}
}
this.annotatedQos = qosMap;
if (methodMap.get("getRegion") == null) {
methodMap.put("hasRegion", new HashMap<Class<? extends Message>, Method>());
methodMap.put("getRegion", new HashMap<Class<? extends Message>, Method>());
}
for (Class<? extends Message> cls : knownArgumentClasses) {
argumentToClassMap.put(cls.getName(), cls);
try {
methodMap.get("hasRegion").put(cls, cls.getDeclaredMethod("hasRegion"));
methodMap.get("getRegion").put(cls, cls.getDeclaredMethod("getRegion"));
} catch (Exception e) {
throw new RuntimeException(e);
}
}
}
public boolean isMetaRegion(byte[] regionName) {
HRegion region;
try {
region = hRegionServer.getRegion(regionName);
} catch (NotServingRegionException ignored) {
return false;
}
return region.getRegionInfo().isMetaTable();
}
@Override
public Integer apply(Pair<RequestHeader, Message> headerAndParam) {
RequestHeader header = headerAndParam.getFirst();
String methodName = header.getMethodName();
Integer priorityByAnnotation = annotatedQos.get(header.getMethodName());
if (priorityByAnnotation != null) {
return priorityByAnnotation;
}
Message param = headerAndParam.getSecond();
if (param == null) {
return HConstants.NORMAL_QOS;
}
String cls = param.getClass().getName();
Class<? extends Message> rpcArgClass = argumentToClassMap.get(cls);
RegionSpecifier regionSpecifier = null;
//check whether the request has reference to meta region or now.
try {
// Check if the param has a region specifier; the pb methods are hasRegion and getRegion if
// hasRegion returns true. Not all listed methods have region specifier each time. For
// example, the ScanRequest has it on setup but thereafter relies on the scannerid rather than
// send the region over every time.
Method hasRegion = methodMap.get("hasRegion").get(rpcArgClass);
if (hasRegion != null && (Boolean)hasRegion.invoke(param, (Object[])null)) {
Method getRegion = methodMap.get("getRegion").get(rpcArgClass);
regionSpecifier = (RegionSpecifier)getRegion.invoke(param, (Object[])null);
HRegion region = hRegionServer.getRegion(regionSpecifier);
if (region.getRegionInfo().isMetaTable()) {
if (LOG.isTraceEnabled()) {
LOG.trace("High priority: " + TextFormat.shortDebugString(param));
}
return HConstants.HIGH_QOS;
}
}
} catch (Exception ex) {
// Not good throwing an exception out of here, a runtime anyways. Let the query go into the
// server and have it throw the exception if still an issue. Just mark it normal priority.
if (LOG.isDebugEnabled()) LOG.debug("Marking normal priority after getting exception=" + ex);
return HConstants.NORMAL_QOS;
}
if (methodName.equals("scan")) { // scanner methods...
ScanRequest request = (ScanRequest)param;
if (!request.hasScannerId()) {
return HConstants.NORMAL_QOS;
}
RegionScanner scanner = hRegionServer.getScanner(request.getScannerId());
if (scanner != null && scanner.getRegionInfo().isMetaRegion()) {
if (LOG.isTraceEnabled()) {
LOG.trace("High priority scanner request: " + TextFormat.shortDebugString(request));
}
return HConstants.HIGH_QOS;
}
}
if (LOG.isTraceEnabled()) {
LOG.trace("Low priority: " + TextFormat.shortDebugString(param));
}
return HConstants.NORMAL_QOS;
}
@VisibleForTesting
void setRegionServer(final HRegionServer hrs) {
this.hRegionServer = hrs;
}
}

View File

@ -279,7 +279,7 @@ public class IncrementCoalescer implements IncrementCoalescerMBean {
LOG.error("FAILED_ICV: " + Bytes.toString(row.getTable()) + ", " LOG.error("FAILED_ICV: " + Bytes.toString(row.getTable()) + ", "
+ Bytes.toStringBinary(row.getRowKey()) + ", " + Bytes.toStringBinary(row.getRowKey()) + ", "
+ Bytes.toStringBinary(row.getFamily()) + ", " + Bytes.toStringBinary(row.getFamily()) + ", "
+ Bytes.toStringBinary(row.getQualifier()) + ", " + counter); + Bytes.toStringBinary(row.getQualifier()) + ", " + counter, e);
} }
} }

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.util; package org.apache.hadoop.hbase.util;
import java.io.IOException; import java.io.IOException;
import java.io.PrintWriter;
import java.lang.reflect.Constructor; import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException; import java.lang.reflect.InvocationTargetException;
import java.util.List; import java.util.List;
@ -31,6 +32,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.ShutdownHook; import org.apache.hadoop.hbase.regionserver.ShutdownHook;
import org.apache.hadoop.util.ReflectionUtils;
/** /**
* Utility used running a cluster all in the one JVM. * Utility used running a cluster all in the one JVM.
@ -201,13 +203,22 @@ public class JVMClusterUtil {
// Wait for an active master to be initialized (implies being master) // Wait for an active master to be initialized (implies being master)
// with this, when we return the cluster is complete // with this, when we return the cluster is complete
startTime = System.currentTimeMillis(); startTime = System.currentTimeMillis();
final int maxwait = 200000;
while (true) { while (true) {
JVMClusterUtil.MasterThread t = findActiveMaster(masters); JVMClusterUtil.MasterThread t = findActiveMaster(masters);
if (t != null && t.master.isInitialized()) { if (t != null && t.master.isInitialized()) {
return t.master.getServerName().toString(); return t.master.getServerName().toString();
} }
if (System.currentTimeMillis() > startTime + 200000) { // REMOVE
throw new RuntimeException("Master not initialized after 200 seconds"); if (System.currentTimeMillis() > startTime + 10000) {
Threads.sleep(1000);
}
if (System.currentTimeMillis() > startTime + maxwait) {
String msg = "Master not initialized after " + maxwait + "ms seconds";
ReflectionUtils.printThreadInfo(new PrintWriter(System.out),
"Thread dump because: " + msg);
throw new RuntimeException(msg);
} }
try { try {
Thread.sleep(100); Thread.sleep(100);
@ -279,8 +290,6 @@ public class JVMClusterUtil {
} }
} }
if (masters != null) { if (masters != null) {
for (JVMClusterUtil.MasterThread t : masters) { for (JVMClusterUtil.MasterThread t : masters) {
while (t.master.isAlive() && !wasInterrupted) { while (t.master.isAlive() && !wasInterrupted) {

View File

@ -38,6 +38,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.catalog.CatalogTracker; import org.apache.hadoop.hbase.catalog.CatalogTracker;
@ -50,6 +51,8 @@ import org.apache.hadoop.hbase.exceptions.TableNotEnabledException;
import org.apache.hadoop.hbase.exceptions.TableNotFoundException; import org.apache.hadoop.hbase.exceptions.TableNotFoundException;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException; import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.ipc.HBaseClient;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -60,6 +63,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZKTableReadOnly; import org.apache.hadoop.hbase.zookeeper.ZKTableReadOnly;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.log4j.Level;
import org.junit.*; import org.junit.*;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
@ -79,6 +83,9 @@ public class TestAdmin {
@BeforeClass @BeforeClass
public static void setUpBeforeClass() throws Exception { public static void setUpBeforeClass() throws Exception {
((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
TEST_UTIL.getConfiguration().setBoolean("hbase.online.schema.update.enable", true); TEST_UTIL.getConfiguration().setBoolean("hbase.online.schema.update.enable", true);
TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100); TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250); TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);

View File

@ -23,14 +23,18 @@ import java.io.IOException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer; import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer;
import org.apache.hadoop.hbase.ipc.HBaseClient;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; 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.ScanResponse;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.log4j.Level;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
@ -55,6 +59,9 @@ public class TestClientScannerRPCTimeout {
@BeforeClass @BeforeClass
public static void setUpBeforeClass() throws Exception { public static void setUpBeforeClass() throws Exception {
((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
Configuration conf = TEST_UTIL.getConfiguration(); Configuration conf = TEST_UTIL.getConfiguration();
conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, rpcTimeout); conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, rpcTimeout);
conf.setStrings(HConstants.REGION_SERVER_IMPL, RegionServerWithScanTimeout.class.getName()); conf.setStrings(HConstants.REGION_SERVER_IMPL, RegionServerWithScanTimeout.class.getName());

View File

@ -47,9 +47,9 @@ import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
@ -78,19 +78,23 @@ import org.apache.hadoop.hbase.filter.WhileMatchFilter;
import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.ipc.HBaseClient;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiMutateRequest; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiMutateRequest;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiRowMutationService; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiRowMutationService;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
import org.apache.hadoop.hbase.exceptions.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.exceptions.NoSuchColumnFamilyException;
import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.log4j.Level;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Before; import org.junit.Before;
@ -120,6 +124,9 @@ public class TestFromClientSide {
*/ */
@BeforeClass @BeforeClass
public static void setUpBeforeClass() throws Exception { public static void setUpBeforeClass() throws Exception {
((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
Configuration conf = TEST_UTIL.getConfiguration(); Configuration conf = TEST_UTIL.getConfiguration();
conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
MultiRowMutationEndpoint.class.getName()); MultiRowMutationEndpoint.class.getName());
@ -4148,11 +4155,11 @@ public class TestFromClientSide {
HTable t = TEST_UTIL.createTable(TABLENAME, FAMILY); HTable t = TEST_UTIL.createTable(TABLENAME, FAMILY);
Put p = new Put(ROW); Put p = new Put(ROW);
p.add(FAMILY, QUALIFIER, VALUE); p.add(FAMILY, QUALIFIER, VALUE);
Mutate m1 = ProtobufUtil.toMutate(MutateType.PUT, p); MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, p);
p = new Put(ROW1); p = new Put(ROW1);
p.add(FAMILY, QUALIFIER, VALUE); p.add(FAMILY, QUALIFIER, VALUE);
Mutate m2 = ProtobufUtil.toMutate(MutateType.PUT, p); MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, p);
MultiMutateRequest.Builder mrmBuilder = MultiMutateRequest.newBuilder(); MultiMutateRequest.Builder mrmBuilder = MultiMutateRequest.newBuilder();
mrmBuilder.addMutationRequest(m1); mrmBuilder.addMutationRequest(m1);
@ -4195,6 +4202,8 @@ public class TestFromClientSide {
Delete d = new Delete(ROW); Delete d = new Delete(ROW);
d.deleteColumns(FAMILY, QUALIFIERS[0]); d.deleteColumns(FAMILY, QUALIFIERS[0]);
arm.add(d); arm.add(d);
// TODO: Trying mutateRow again. The batch was failing with a one try only.
// t.mutateRow(arm);
t.batch(Arrays.asList((Row)arm)); t.batch(Arrays.asList((Row)arm));
r = t.get(g); r = t.get(g);
assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[1]))); assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[1])));

View File

@ -660,6 +660,8 @@ public class TestHCM {
} }
} }
assertNotNull(otherRow); assertNotNull(otherRow);
// If empty row, set it to first row.-f
if (otherRow.length <= 0) otherRow = Bytes.toBytes("aaa");
Put put2 = new Put(otherRow); Put put2 = new Put(otherRow);
put2.add(FAM_NAM, otherRow, otherRow); put2.add(FAM_NAM, otherRow, otherRow);
table.put(put2); //cache put2's location table.put(put2); //cache put2's location

View File

@ -68,6 +68,9 @@ public class TestMultiParallel {
private static final int slaves = 2; // also used for testing HTable pool size private static final int slaves = 2; // also used for testing HTable pool size
@BeforeClass public static void beforeClass() throws Exception { @BeforeClass public static void beforeClass() throws Exception {
((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
UTIL.startMiniCluster(slaves); UTIL.startMiniCluster(slaves);
HTable t = UTIL.createTable(Bytes.toBytes(TEST_TABLE), Bytes.toBytes(FAMILY)); HTable t = UTIL.createTable(Bytes.toBytes(TEST_TABLE), Bytes.toBytes(FAMILY));
UTIL.createMultiRegions(t, Bytes.toBytes(FAMILY)); UTIL.createMultiRegions(t, Bytes.toBytes(FAMILY));

View File

@ -0,0 +1,137 @@
/**
* 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.codec;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
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.Cell;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.codec.CellCodec;
import org.apache.hadoop.hbase.codec.Codec;
import org.apache.hadoop.hbase.codec.KeyValueCodec;
import org.apache.hadoop.hbase.codec.MessageCodec;
import org.apache.hadoop.hbase.io.CellOutputStream;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Do basic codec performance eval.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class CodecPerformance {
public static final Log LOG = LogFactory.getLog(CodecPerformance.class);
static Cell [] getCells(final int howMany) {
Cell [] cells = new Cell[howMany];
for (int i = 0; i < howMany; i++) {
byte [] index = Bytes.toBytes(i);
KeyValue kv = new KeyValue(index, Bytes.toBytes("f"), index, index);
cells[i] = kv;
}
return cells;
}
static int getRoughSize(final Cell [] cells) {
int size = 0;
for (Cell c: cells) {
size += c.getRowLength() + c.getFamilyLength() + c.getQualifierLength() + c.getValueLength();
size += Bytes.SIZEOF_LONG + Bytes.SIZEOF_BYTE;
}
return size;
}
static byte [] runEncoderTest(final int index, final int initialBufferSize,
final ByteArrayOutputStream baos, final CellOutputStream encoder, final Cell [] cells)
throws IOException {
long startTime = System.currentTimeMillis();
for (int i = 0; i < cells.length; i++) {
encoder.write(cells[i]);
}
encoder.flush();
LOG.info("" + index + " encoded count=" + cells.length + " in " +
(System.currentTimeMillis() - startTime) + "ms for encoder " + encoder);
// Ensure we did not have to grow the backing buffer.
assertTrue(baos.size() < initialBufferSize);
return baos.toByteArray();
}
static Cell [] runDecoderTest(final int index, final int count, final CellScanner decoder)
throws IOException {
Cell [] cells = new Cell[count];
long startTime = System.currentTimeMillis();
for (int i = 0; decoder.advance(); i++) {
cells[i] = decoder.current();
}
LOG.info("" + index + " decoded count=" + cells.length + " in " +
(System.currentTimeMillis() - startTime) + "ms for decoder " + decoder);
// Ensure we did not have to grow the backing buffer.
assertTrue(cells.length == count);
return cells;
}
static void verifyCells(final Cell [] input, final Cell [] output) {
assertEquals(input.length, output.length);
for (int i = 0; i < input.length; i ++) {
input[i].equals(output[i]);
}
}
static void doCodec(final Codec codec, final Cell [] cells, final int cycles, final int count,
final int initialBufferSize)
throws IOException {
byte [] bytes = null;
Cell [] cellsDecoded = null;
for (int i = 0; i < cycles; i++) {
ByteArrayOutputStream baos = new ByteArrayOutputStream(initialBufferSize);
Codec.Encoder encoder = codec.getEncoder(baos);
bytes = runEncoderTest(i, initialBufferSize, baos, encoder, cells);
}
for (int i = 0; i < cycles; i++) {
ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
Codec.Decoder decoder = codec.getDecoder(bais);
cellsDecoded = CodecPerformance.runDecoderTest(i, count, decoder);
}
verifyCells(cells, cellsDecoded);
}
public static void main(String[] args) throws IOException {
// How many Cells to encode/decode on each cycle.
final int count = 100000;
// How many times to do an operation; repeat gives hotspot chance to warm up.
final int cycles = 30;
Cell [] cells = getCells(count);
int size = getRoughSize(cells);
int initialBufferSize = 2 * size; // Multiply by 2 to ensure we don't have to grow buffer
// Test KeyValue codec.
doCodec(new KeyValueCodec(), cells, cycles, count, initialBufferSize);
doCodec(new CellCodec(), cells, cycles, count, initialBufferSize);
doCodec(new MessageCodec(), cells, cycles, count, initialBufferSize);
}
}

View File

@ -0,0 +1,122 @@
/**
* 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.codec;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.codec.Codec;
import org.apache.hadoop.hbase.codec.MessageCodec;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import com.google.common.io.CountingInputStream;
import com.google.common.io.CountingOutputStream;
@Category(SmallTests.class)
public class TestCellMessageCodec {
public static final Log LOG = LogFactory.getLog(TestCellMessageCodec.class);
@Test
public void testEmptyWorks() throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
CountingOutputStream cos = new CountingOutputStream(baos);
DataOutputStream dos = new DataOutputStream(cos);
MessageCodec cmc = new MessageCodec();
Codec.Encoder encoder = cmc.getEncoder(dos);
encoder.flush();
dos.close();
long offset = cos.getCount();
assertEquals(0, offset);
CountingInputStream cis = new CountingInputStream(new ByteArrayInputStream(baos.toByteArray()));
DataInputStream dis = new DataInputStream(cis);
Codec.Decoder decoder = cmc.getDecoder(dis);
assertFalse(decoder.advance());
dis.close();
assertEquals(0, cis.getCount());
}
@Test
public void testOne() throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
CountingOutputStream cos = new CountingOutputStream(baos);
DataOutputStream dos = new DataOutputStream(cos);
MessageCodec cmc = new MessageCodec();
Codec.Encoder encoder = cmc.getEncoder(dos);
final KeyValue kv =
new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"), Bytes.toBytes("v"));
encoder.write(kv);
encoder.flush();
dos.close();
long offset = cos.getCount();
CountingInputStream cis = new CountingInputStream(new ByteArrayInputStream(baos.toByteArray()));
DataInputStream dis = new DataInputStream(cis);
Codec.Decoder decoder = cmc.getDecoder(dis);
assertTrue(decoder.advance()); // First read should pull in the KV
assertFalse(decoder.advance()); // Second read should trip over the end-of-stream marker and return false
dis.close();
assertEquals(offset, cis.getCount());
}
@Test
public void testThree() throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
CountingOutputStream cos = new CountingOutputStream(baos);
DataOutputStream dos = new DataOutputStream(cos);
MessageCodec cmc = new MessageCodec();
Codec.Encoder encoder = cmc.getEncoder(dos);
final KeyValue kv1 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("1"), Bytes.toBytes("1"));
final KeyValue kv2 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("2"), Bytes.toBytes("2"));
final KeyValue kv3 = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("3"), Bytes.toBytes("3"));
encoder.write(kv1);
encoder.write(kv2);
encoder.write(kv3);
encoder.flush();
dos.close();
long offset = cos.getCount();
CountingInputStream cis = new CountingInputStream(new ByteArrayInputStream(baos.toByteArray()));
DataInputStream dis = new DataInputStream(cis);
Codec.Decoder decoder = cmc.getDecoder(dis);
assertTrue(decoder.advance());
Cell c = decoder.current();
assertTrue(CellComparator.equals(c, kv1));
assertTrue(decoder.advance());
c = decoder.current();
assertTrue(CellComparator.equals(c, kv2));
assertTrue(decoder.advance());
c = decoder.current();
assertTrue(CellComparator.equals(c, kv3));
assertFalse(decoder.advance());
dis.close();
assertEquals(offset, cis.getCount());
}
}

View File

@ -25,6 +25,7 @@ import java.util.List;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
@ -33,12 +34,15 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.exceptions.MasterNotRunningException; import org.apache.hadoop.hbase.exceptions.MasterNotRunningException;
import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException; import org.apache.hadoop.hbase.exceptions.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.ipc.HBaseClient;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.ScannerCallable;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass; import org.junit.AfterClass;
@ -46,6 +50,7 @@ import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.MediumTests;
import org.apache.log4j.Level;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
/** /**
@ -172,7 +177,9 @@ public class TestFilterWithScanLimits {
@BeforeClass @BeforeClass
public static void setUp() throws Exception { public static void setUp() throws Exception {
Configuration config = TEST_UTIL.getConfiguration(); ((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
TEST_UTIL.startMiniCluster(1); TEST_UTIL.startMiniCluster(1);
initialize(TEST_UTIL.getConfiguration()); initialize(TEST_UTIL.getConfiguration());
} }

View File

@ -44,6 +44,8 @@ import org.apache.log4j.spi.LoggingEvent;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import com.google.protobuf.ServiceException;
/** /**
* Test that delayed RPCs work. Fire up three calls, the first of which should * Test that delayed RPCs work. Fire up three calls, the first of which should
* be delayed. Check that the last two, which are undelayed, return before the * be delayed. Check that the last two, which are undelayed, return before the
@ -100,8 +102,7 @@ public class TestDelayedRpc {
assertEquals(UNDELAYED, results.get(0).intValue()); assertEquals(UNDELAYED, results.get(0).intValue());
assertEquals(UNDELAYED, results.get(1).intValue()); assertEquals(UNDELAYED, results.get(1).intValue());
assertEquals(results.get(2).intValue(), delayReturnValue ? DELAYED : assertEquals(results.get(2).intValue(), delayReturnValue ? DELAYED : 0xDEADBEEF);
0xDEADBEEF);
} finally { } finally {
clientEngine.close(); clientEngine.close();
} }
@ -182,7 +183,7 @@ public class TestDelayedRpc {
} }
public interface TestRpc extends IpcProtocol { public interface TestRpc extends IpcProtocol {
TestResponse test(TestArg delay); TestResponse test(final Object rpcController, TestArg delay) throws ServiceException;
} }
private static class TestRpcImpl implements TestRpc { private static class TestRpcImpl implements TestRpc {
@ -201,7 +202,8 @@ public class TestDelayedRpc {
} }
@Override @Override
public TestResponse test(final TestArg testArg) { public TestResponse test(final Object rpcController, final TestArg testArg)
throws ServiceException {
boolean delay = testArg.getDelay(); boolean delay = testArg.getDelay();
TestResponse.Builder responseBuilder = TestResponse.newBuilder(); TestResponse.Builder responseBuilder = TestResponse.newBuilder();
if (!delay) { if (!delay) {
@ -243,9 +245,8 @@ public class TestDelayedRpc {
@Override @Override
public void run() { public void run() {
try { try {
Integer result = Integer result = new Integer(server.test(null, TestArg.newBuilder().setDelay(delay).
new Integer(server.test(TestArg.newBuilder() build()).getResponse());
.setDelay(delay).build()).getResponse());
if (results != null) { if (results != null) {
synchronized (results) { synchronized (results) {
results.add(result); results.add(result);
@ -276,7 +277,7 @@ public class TestDelayedRpc {
int result = 0xDEADBEEF; int result = 0xDEADBEEF;
try { try {
result = client.test(TestArg.newBuilder().setDelay(false).build()).getResponse(); result = client.test(null, TestArg.newBuilder().setDelay(false).build()).getResponse();
} catch (Exception e) { } catch (Exception e) {
fail("No exception should have been thrown."); fail("No exception should have been thrown.");
} }
@ -284,7 +285,7 @@ public class TestDelayedRpc {
boolean caughtException = false; boolean caughtException = false;
try { try {
result = client.test(TestArg.newBuilder().setDelay(true).build()).getResponse(); result = client.test(null, TestArg.newBuilder().setDelay(true).build()).getResponse();
} catch(Exception e) { } catch(Exception e) {
// Exception thrown by server is enclosed in a RemoteException. // Exception thrown by server is enclosed in a RemoteException.
if (e.getCause().getMessage().contains( if (e.getCause().getMessage().contains(
@ -303,7 +304,7 @@ public class TestDelayedRpc {
*/ */
private static class FaultyTestRpc implements TestRpc { private static class FaultyTestRpc implements TestRpc {
@Override @Override
public TestResponse test(TestArg arg) { public TestResponse test(Object rpcController, TestArg arg) {
if (!arg.getDelay()) if (!arg.getDelay())
return TestResponse.newBuilder().setResponse(UNDELAYED).build(); return TestResponse.newBuilder().setResponse(UNDELAYED).build();
Delayable call = HBaseServer.getCurrentCall(); Delayable call = HBaseServer.getCurrentCall();

View File

@ -25,21 +25,35 @@ import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.spy; import static org.mockito.Mockito.spy;
import java.io.IOException; import java.io.IOException;
import java.lang.reflect.Method;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.net.Socket; import java.net.Socket;
import java.util.ArrayList;
import java.util.List;
import javax.net.SocketFactory; import javax.net.SocketFactory;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.IpcProtocol; import org.apache.hadoop.hbase.IpcProtocol;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.SmallTests; import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody; import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.junit.Test; import org.junit.Test;
@ -53,6 +67,8 @@ import com.google.protobuf.Message;
@Category(SmallTests.class) @Category(SmallTests.class)
public class TestIPC { public class TestIPC {
public static final Log LOG = LogFactory.getLog(TestIPC.class); public static final Log LOG = LogFactory.getLog(TestIPC.class);
static byte [] CELL_BYTES = Bytes.toBytes("xyz");
static Cell CELL = new KeyValue(CELL_BYTES, CELL_BYTES, CELL_BYTES, CELL_BYTES);
private static class TestRpcServer extends HBaseServer { private static class TestRpcServer extends HBaseServer {
TestRpcServer() throws IOException { TestRpcServer() throws IOException {
@ -60,10 +76,53 @@ public class TestIPC {
} }
@Override @Override
public Message call(Class<? extends IpcProtocol> protocol, public Pair<Message, CellScanner> call(Class<? extends IpcProtocol> protocol, Method method,
RpcRequestBody param, long receiveTime, MonitoredRPCHandler status) Message param, final CellScanner cells, long receiveTime, MonitoredRPCHandler status)
throws IOException { throws IOException {
return param; /*
List<Cell> cellsOut = new ArrayList<Cell>();
while (cells.advance()) {
Cell cell = cells.current();
Bytes.equals(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
CELL_BYTES, 0, CELL_BYTES.length);
cellsOut.add(cell);
}
return new Pair<Message, CellScanner>(param, CellUtil.createCellScanner(cellsOut));
*/
return new Pair<Message, CellScanner>(param, null);
}
}
/**
* A nothing protocol used in test below.
*/
interface NothingProtocol extends IpcProtocol {
void doNothing();
}
public static class DoNothing implements NothingProtocol {
public void doNothing() {}
}
@Test
public void testCompressCellBlock()
throws IOException, InterruptedException, SecurityException, NoSuchMethodException {
Configuration conf = HBaseConfiguration.create();
conf.set("hbase.client.rpc.compressor", GzipCodec.class.getCanonicalName());
TestRpcServer rpcServer = new TestRpcServer();
HBaseClient client = new HBaseClient(conf, HConstants.CLUSTER_ID_DEFAULT);
List<Cell> cells = new ArrayList<Cell>();
cells.add(CELL);
try {
rpcServer.start();
InetSocketAddress address = rpcServer.getListenerAddress();
// Get any method name... just so it is not null
Method m = NothingProtocol.class.getMethod("doNothing");
client.call(m, null, CellUtil.createCellScanner(cells), address, NothingProtocol.class,
User.getCurrent(), 0);
} finally {
client.stop();
rpcServer.stop();
} }
} }
@ -81,18 +140,73 @@ public class TestIPC {
}).when(spyFactory).createSocket(); }).when(spyFactory).createSocket();
TestRpcServer rpcServer = new TestRpcServer(); TestRpcServer rpcServer = new TestRpcServer();
rpcServer.start(); HBaseClient client = new HBaseClient(conf, HConstants.CLUSTER_ID_DEFAULT, spyFactory);
HBaseClient client = new HBaseClient(
conf, HConstants.CLUSTER_ID_DEFAULT, spyFactory);
InetSocketAddress address = rpcServer.getListenerAddress();
try { try {
client.call(RpcRequestBody.getDefaultInstance(), address, User.getCurrent(), 0); rpcServer.start();
InetSocketAddress address = rpcServer.getListenerAddress();
client.call(null, null, null, address, null, User.getCurrent(), 0);
fail("Expected an exception to have been thrown!"); fail("Expected an exception to have been thrown!");
} catch (Exception e) { } catch (Exception e) {
LOG.info("Caught expected exception: " + e.toString()); LOG.info("Caught expected exception: " + e.toString());
assertTrue(StringUtils.stringifyException(e).contains("Injected fault")); assertTrue(StringUtils.stringifyException(e).contains("Injected fault"));
} finally {
client.stop();
rpcServer.stop();
}
}
public static void main(String[] args)
throws IOException, SecurityException, NoSuchMethodException, InterruptedException {
if (args.length != 2) {
System.out.println("Usage: TestIPC <CYCLES> <CELLS_PER_CYCLE>");
return;
}
// ((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.INFO);
// ((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.INFO);
int cycles = Integer.parseInt(args[0]);
int cellcount = Integer.parseInt(args[1]);
Configuration conf = HBaseConfiguration.create();
TestRpcServer rpcServer = new TestRpcServer();
HBaseClient client = new HBaseClient(conf, HConstants.CLUSTER_ID_DEFAULT);
KeyValue kv = KeyValueUtil.ensureKeyValue(CELL);
Put p = new Put(kv.getRow());
for (int i = 0; i < cellcount; i++) {
p.add(kv);
}
RowMutations rm = new RowMutations(kv.getRow());
rm.add(p);
try {
rpcServer.start();
InetSocketAddress address = rpcServer.getListenerAddress();
// Get any method name... just so it is not null
Method m = NothingProtocol.class.getMethod("doNothing");
long startTime = System.currentTimeMillis();
User user = User.getCurrent();
for (int i = 0; i < cycles; i++) {
List<CellScannable> cells = new ArrayList<CellScannable>();
// Message param = RequestConverter.buildMultiRequest(HConstants.EMPTY_BYTE_ARRAY, rm);
Message param = RequestConverter.buildNoDataMultiRequest(HConstants.EMPTY_BYTE_ARRAY, rm, cells);
CellScanner cellScanner = CellUtil.createCellScanner(cells);
if (i % 1000 == 0) {
LOG.info("" + i);
// Uncomment this for a thread dump every so often.
// ReflectionUtils.printThreadInfo(new PrintWriter(System.out),
// "Thread dump " + Thread.currentThread().getName());
}
Pair<Message, CellScanner> response =
client.call(m, param, cellScanner, address, NothingProtocol.class, user, 0);
/*
int count = 0;
while (p.getSecond().advance()) {
count++;
}
assertEquals(cells.size(), count);*/
}
LOG.info("Cycled " + cycles + " time(s) with " + cellcount + " cell(s) in " +
(System.currentTimeMillis() - startTime) + "ms");
} finally {
client.stop();
rpcServer.stop();
} }
} }
} }

View File

@ -22,13 +22,15 @@ import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.IpcProtocol; import org.apache.hadoop.hbase.IpcProtocol;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoRequestProto;
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EchoResponseProto;
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto;
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto;
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto; import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.junit.Before; import org.junit.Before;
@ -53,8 +55,7 @@ public class TestProtoBufRpc {
private static Configuration conf; private static Configuration conf;
private static RpcServer server; private static RpcServer server;
public interface TestRpcService public interface TestRpcService extends TestProtobufRpcProto.BlockingInterface, IpcProtocol {
extends TestProtobufRpcProto.BlockingInterface, IpcProtocol {
public long VERSION = 1; public long VERSION = 1;
} }
@ -83,7 +84,10 @@ public class TestProtoBufRpc {
@Before @Before
public void setUp() throws IOException { // Setup server for both protocols public void setUp() throws IOException { // Setup server for both protocols
conf = new Configuration(); conf = new Configuration();
Logger log = Logger.getLogger("org.apache.hadoop.ipc.HBaseServer");
log.setLevel(Level.DEBUG);
log = Logger.getLogger("org.apache.hadoop.ipc.HBaseServer.trace");
log.setLevel(Level.TRACE);
// Create server side implementation // Create server side implementation
PBServerImpl serverImpl = new PBServerImpl(); PBServerImpl serverImpl = new PBServerImpl();
// Get RPC server for server side implementation // Get RPC server for server side implementation
@ -94,7 +98,6 @@ public class TestProtoBufRpc {
server.start(); server.start();
} }
@After @After
public void tearDown() throws Exception { public void tearDown() throws Exception {
server.stop(); server.stop();
@ -105,14 +108,13 @@ public class TestProtoBufRpc {
ProtobufRpcClientEngine clientEngine = ProtobufRpcClientEngine clientEngine =
new ProtobufRpcClientEngine(conf, HConstants.CLUSTER_ID_DEFAULT); new ProtobufRpcClientEngine(conf, HConstants.CLUSTER_ID_DEFAULT);
try { try {
TestRpcService client = clientEngine.getProxy(TestRpcService.class, addr, conf, 10000); TestRpcService client = clientEngine.getProxy(TestRpcService.class, addr, conf, 100000);
// Test ping method // Test ping method
EmptyRequestProto emptyRequest = EmptyRequestProto.newBuilder().build(); EmptyRequestProto emptyRequest = EmptyRequestProto.newBuilder().build();
client.ping(null, emptyRequest); client.ping(null, emptyRequest);
// Test echo method // Test echo method
EchoRequestProto echoRequest = EchoRequestProto.newBuilder() EchoRequestProto echoRequest = EchoRequestProto.newBuilder().setMessage("hello").build();
.setMessage("hello").build();
EchoResponseProto echoResponse = client.echo(null, echoRequest); EchoResponseProto echoResponse = client.echo(null, echoRequest);
Assert.assertEquals(echoResponse.getMessage(), "hello"); Assert.assertEquals(echoResponse.getMessage(), "hello");

View File

@ -31,11 +31,11 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; 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.Column;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.DeleteType;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
@ -88,9 +88,9 @@ public class TestProtobufUtil {
*/ */
@Test @Test
public void testAppend() throws IOException { public void testAppend() throws IOException {
Mutate.Builder mutateBuilder = Mutate.newBuilder(); MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
mutateBuilder.setRow(ByteString.copyFromUtf8("row")); mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
mutateBuilder.setMutateType(MutateType.APPEND); mutateBuilder.setMutateType(MutationType.APPEND);
mutateBuilder.setTimestamp(111111); mutateBuilder.setTimestamp(111111);
ColumnValue.Builder valueBuilder = ColumnValue.newBuilder(); ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
valueBuilder.setFamily(ByteString.copyFromUtf8("f1")); valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
@ -103,29 +103,27 @@ public class TestProtobufUtil {
valueBuilder.addQualifierValue(qualifierBuilder.build()); valueBuilder.addQualifierValue(qualifierBuilder.build());
mutateBuilder.addColumnValue(valueBuilder.build()); mutateBuilder.addColumnValue(valueBuilder.build());
Mutate proto = mutateBuilder.build(); MutationProto proto = mutateBuilder.build();
// default fields // default fields
assertEquals(true, proto.getWriteToWAL()); assertEquals(true, proto.getWriteToWAL());
// set the default value for equal comparison // set the default value for equal comparison
mutateBuilder = Mutate.newBuilder(proto); mutateBuilder = MutationProto.newBuilder(proto);
mutateBuilder.setWriteToWAL(true); mutateBuilder.setWriteToWAL(true);
Append append = ProtobufUtil.toAppend(proto); Append append = ProtobufUtil.toAppend(proto, null);
// append always use the latest timestamp, // append always use the latest timestamp,
// add the timestamp to the original mutate // add the timestamp to the original mutate
long timestamp = append.getTimeStamp(); long timestamp = append.getTimeStamp();
mutateBuilder.setTimestamp(timestamp); mutateBuilder.setTimestamp(timestamp);
for (ColumnValue.Builder column: for (ColumnValue.Builder column: mutateBuilder.getColumnValueBuilderList()) {
mutateBuilder.getColumnValueBuilderList()) {
for (QualifierValue.Builder qualifier: for (QualifierValue.Builder qualifier:
column.getQualifierValueBuilderList()) { column.getQualifierValueBuilderList()) {
qualifier.setTimestamp(timestamp); qualifier.setTimestamp(timestamp);
} }
} }
assertEquals(mutateBuilder.build(), assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.APPEND, append));
ProtobufUtil.toMutate(MutateType.APPEND, append));
} }
/** /**
@ -135,9 +133,9 @@ public class TestProtobufUtil {
*/ */
@Test @Test
public void testDelete() throws IOException { public void testDelete() throws IOException {
Mutate.Builder mutateBuilder = Mutate.newBuilder(); MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
mutateBuilder.setRow(ByteString.copyFromUtf8("row")); mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
mutateBuilder.setMutateType(MutateType.DELETE); mutateBuilder.setMutateType(MutationType.DELETE);
mutateBuilder.setTimestamp(111111); mutateBuilder.setTimestamp(111111);
ColumnValue.Builder valueBuilder = ColumnValue.newBuilder(); ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
valueBuilder.setFamily(ByteString.copyFromUtf8("f1")); valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
@ -152,12 +150,12 @@ public class TestProtobufUtil {
valueBuilder.addQualifierValue(qualifierBuilder.build()); valueBuilder.addQualifierValue(qualifierBuilder.build());
mutateBuilder.addColumnValue(valueBuilder.build()); mutateBuilder.addColumnValue(valueBuilder.build());
Mutate proto = mutateBuilder.build(); MutationProto proto = mutateBuilder.build();
// default fields // default fields
assertEquals(true, proto.getWriteToWAL()); assertEquals(true, proto.getWriteToWAL());
// set the default value for equal comparison // set the default value for equal comparison
mutateBuilder = Mutate.newBuilder(proto); mutateBuilder = MutationProto.newBuilder(proto);
mutateBuilder.setWriteToWAL(true); mutateBuilder.setWriteToWAL(true);
Delete delete = ProtobufUtil.toDelete(proto); Delete delete = ProtobufUtil.toDelete(proto);
@ -172,7 +170,7 @@ public class TestProtobufUtil {
} }
} }
assertEquals(mutateBuilder.build(), assertEquals(mutateBuilder.build(),
ProtobufUtil.toMutate(MutateType.DELETE, delete)); ProtobufUtil.toMutation(MutationType.DELETE, delete));
} }
/** /**
@ -182,9 +180,9 @@ public class TestProtobufUtil {
*/ */
@Test @Test
public void testIncrement() throws IOException { public void testIncrement() throws IOException {
Mutate.Builder mutateBuilder = Mutate.newBuilder(); MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
mutateBuilder.setRow(ByteString.copyFromUtf8("row")); mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
mutateBuilder.setMutateType(MutateType.INCREMENT); mutateBuilder.setMutateType(MutationType.INCREMENT);
ColumnValue.Builder valueBuilder = ColumnValue.newBuilder(); ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
valueBuilder.setFamily(ByteString.copyFromUtf8("f1")); valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder(); QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
@ -196,16 +194,16 @@ public class TestProtobufUtil {
valueBuilder.addQualifierValue(qualifierBuilder.build()); valueBuilder.addQualifierValue(qualifierBuilder.build());
mutateBuilder.addColumnValue(valueBuilder.build()); mutateBuilder.addColumnValue(valueBuilder.build());
Mutate proto = mutateBuilder.build(); MutationProto proto = mutateBuilder.build();
// default fields // default fields
assertEquals(true, proto.getWriteToWAL()); assertEquals(true, proto.getWriteToWAL());
// set the default value for equal comparison // set the default value for equal comparison
mutateBuilder = Mutate.newBuilder(proto); mutateBuilder = MutationProto.newBuilder(proto);
mutateBuilder.setWriteToWAL(true); mutateBuilder.setWriteToWAL(true);
Increment increment = ProtobufUtil.toIncrement(proto); Increment increment = ProtobufUtil.toIncrement(proto, null);
assertEquals(mutateBuilder.build(), ProtobufUtil.toMutate(increment)); assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(increment));
} }
/** /**
@ -215,9 +213,9 @@ public class TestProtobufUtil {
*/ */
@Test @Test
public void testPut() throws IOException { public void testPut() throws IOException {
Mutate.Builder mutateBuilder = Mutate.newBuilder(); MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
mutateBuilder.setRow(ByteString.copyFromUtf8("row")); mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
mutateBuilder.setMutateType(MutateType.PUT); mutateBuilder.setMutateType(MutationType.PUT);
mutateBuilder.setTimestamp(111111); mutateBuilder.setTimestamp(111111);
ColumnValue.Builder valueBuilder = ColumnValue.newBuilder(); ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
valueBuilder.setFamily(ByteString.copyFromUtf8("f1")); valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
@ -231,12 +229,12 @@ public class TestProtobufUtil {
valueBuilder.addQualifierValue(qualifierBuilder.build()); valueBuilder.addQualifierValue(qualifierBuilder.build());
mutateBuilder.addColumnValue(valueBuilder.build()); mutateBuilder.addColumnValue(valueBuilder.build());
Mutate proto = mutateBuilder.build(); MutationProto proto = mutateBuilder.build();
// default fields // default fields
assertEquals(true, proto.getWriteToWAL()); assertEquals(true, proto.getWriteToWAL());
// set the default value for equal comparison // set the default value for equal comparison
mutateBuilder = Mutate.newBuilder(proto); mutateBuilder = MutationProto.newBuilder(proto);
mutateBuilder.setWriteToWAL(true); mutateBuilder.setWriteToWAL(true);
Put put = ProtobufUtil.toPut(proto); Put put = ProtobufUtil.toPut(proto);
@ -255,7 +253,7 @@ public class TestProtobufUtil {
} }
} }
assertEquals(mutateBuilder.build(), assertEquals(mutateBuilder.build(),
ProtobufUtil.toMutate(MutateType.PUT, put)); ProtobufUtil.toMutation(MutationType.PUT, put));
} }
/** /**

View File

@ -18,23 +18,21 @@
*/ */
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId; import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
/** /**
* Test metrics incremented on region server operations. * Test metrics incremented on region server operations.

View File

@ -22,37 +22,25 @@ import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import org.junit.Assert;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.filter.SingleColumnValueExcludeFilter;
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
import org.apache.hadoop.hbase.filter.FilterList;
import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.RowFilter;
import org.apache.hadoop.hbase.filter.RegexStringComparator;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
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.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
@ -69,7 +57,6 @@ public class TestJoinedScanners {
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static final String DIR = TEST_UTIL.getDataTestDir("TestJoinedScanners").toString(); private static final String DIR = TEST_UTIL.getDataTestDir("TestJoinedScanners").toString();
private static final byte[] tableName = Bytes.toBytes("testTable");
private static final byte[] cf_essential = Bytes.toBytes("essential"); private static final byte[] cf_essential = Bytes.toBytes("essential");
private static final byte[] cf_joined = Bytes.toBytes("joined"); private static final byte[] cf_joined = Bytes.toBytes("joined");
private static final byte[] col_name = Bytes.toBytes("a"); private static final byte[] col_name = Bytes.toBytes("a");

View File

@ -28,58 +28,56 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
import org.apache.hadoop.hbase.regionserver.HRegionServer.QosFunction; import org.apache.hadoop.hbase.util.Pair;
import org.junit.BeforeClass; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import org.mockito.Mockito; import org.mockito.Mockito;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
import com.google.protobuf.Message;
/** /**
* Tests that verify certain RPCs get a higher QoS. * Tests that verify certain RPCs get a higher QoS.
*/ */
@Category(MediumTests.class) @Category(MediumTests.class)
public class TestPriorityRpc { public class TestPriorityRpc {
static HRegionServer regionServer = null; private HRegionServer regionServer = null;
static QosFunction qosFunction = null; private QosFunction qosFunction = null;
@BeforeClass
public static void onetimeSetup() { @Before
public void setup() {
Configuration conf = HBaseConfiguration.create(); Configuration conf = HBaseConfiguration.create();
regionServer = regionServer = HRegionServer.constructRegionServer(HRegionServer.class, conf);
HRegionServer.constructRegionServer(HRegionServer.class, conf);
qosFunction = regionServer.getQosFunction(); qosFunction = regionServer.getQosFunction();
} }
@Test @Test
public void testQosFunctionForMeta() throws IOException { public void testQosFunctionForMeta() throws IOException {
qosFunction = regionServer.getQosFunction(); qosFunction = regionServer.getQosFunction();
RpcRequestBody.Builder rpcRequestBuilder = RpcRequestBody.newBuilder(); RequestHeader.Builder headerBuilder = RequestHeader.newBuilder();
//create a rpc request that has references to META region and also //create a rpc request that has references to META region and also
//uses one of the known argument classes (known argument classes are //uses one of the known argument classes (known argument classes are
//listed in HRegionServer.QosFunction.knownArgumentClasses) //listed in HRegionServer.QosFunction.knownArgumentClasses)
rpcRequestBuilder = RpcRequestBody.newBuilder(); headerBuilder.setMethodName("foo");
rpcRequestBuilder.setMethodName("foo");
GetRequest.Builder getRequestBuilder = GetRequest.newBuilder(); GetRequest.Builder getRequestBuilder = GetRequest.newBuilder();
RegionSpecifier.Builder regionSpecifierBuilder = RegionSpecifier.newBuilder(); RegionSpecifier.Builder regionSpecifierBuilder = RegionSpecifier.newBuilder();
regionSpecifierBuilder.setType(RegionSpecifierType.REGION_NAME); regionSpecifierBuilder.setType(RegionSpecifierType.REGION_NAME);
ByteString name = ByteString name = ByteString.copyFrom(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
ByteString.copyFrom(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
regionSpecifierBuilder.setValue(name); regionSpecifierBuilder.setValue(name);
RegionSpecifier regionSpecifier = regionSpecifierBuilder.build(); RegionSpecifier regionSpecifier = regionSpecifierBuilder.build();
getRequestBuilder.setRegion(regionSpecifier); getRequestBuilder.setRegion(regionSpecifier);
Get.Builder getBuilder = Get.newBuilder(); Get.Builder getBuilder = Get.newBuilder();
getBuilder.setRow(ByteString.copyFrom("somerow".getBytes())); getBuilder.setRow(ByteString.copyFrom("somerow".getBytes()));
getRequestBuilder.setGet(getBuilder.build()); getRequestBuilder.setGet(getBuilder.build());
rpcRequestBuilder.setRequest(getRequestBuilder.build().toByteString()); GetRequest getRequest = getRequestBuilder.build();
rpcRequestBuilder.setRequestClassName(GetRequest.class.getCanonicalName()); RequestHeader header = headerBuilder.build();
RpcRequestBody rpcRequest = rpcRequestBuilder.build();
HRegion mockRegion = Mockito.mock(HRegion.class); HRegion mockRegion = Mockito.mock(HRegion.class);
HRegionServer mockRS = Mockito.mock(HRegionServer.class); HRegionServer mockRS = Mockito.mock(HRegionServer.class);
HRegionInfo mockRegionInfo = Mockito.mock(HRegionInfo.class); HRegionInfo mockRegionInfo = Mockito.mock(HRegionInfo.class);
@ -87,7 +85,8 @@ public class TestPriorityRpc {
Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo); Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
Mockito.when(mockRegionInfo.isMetaTable()).thenReturn(true); Mockito.when(mockRegionInfo.isMetaTable()).thenReturn(true);
qosFunction.setRegionServer(mockRS); qosFunction.setRegionServer(mockRS);
assertTrue (qosFunction.apply(rpcRequest) == HConstants.HIGH_QOS); assertTrue (qosFunction.apply(new Pair<RequestHeader, Message>(header, getRequest)) ==
HConstants.HIGH_QOS);
} }
@Test @Test
@ -96,51 +95,53 @@ public class TestPriorityRpc {
//known argument classes (it uses one random request class) //known argument classes (it uses one random request class)
//(known argument classes are listed in //(known argument classes are listed in
//HRegionServer.QosFunction.knownArgumentClasses) //HRegionServer.QosFunction.knownArgumentClasses)
RpcRequestBody.Builder rpcRequestBuilder = RpcRequestBody.newBuilder(); RequestHeader.Builder headerBuilder = RequestHeader.newBuilder();
rpcRequestBuilder.setMethodName("foo"); headerBuilder.setMethodName("foo");
rpcRequestBuilder.setRequestClassName(GetOnlineRegionRequest.class.getCanonicalName()); RequestHeader header = headerBuilder.build();
RpcRequestBody rpcRequest = rpcRequestBuilder.build();
QosFunction qosFunc = regionServer.getQosFunction(); QosFunction qosFunc = regionServer.getQosFunction();
assertTrue (qosFunc.apply(rpcRequest) == HConstants.NORMAL_QOS); assertTrue (qosFunc.apply(new Pair<RequestHeader, Message>(header, null)) ==
HConstants.NORMAL_QOS);
} }
@Test @Test
public void testQosFunctionForScanMethod() throws IOException { public void testQosFunctionForScanMethod() throws IOException {
RpcRequestBody.Builder rpcRequestBuilder = RpcRequestBody.newBuilder(); RequestHeader.Builder headerBuilder = RequestHeader.newBuilder();
rpcRequestBuilder.setMethodName("scan"); headerBuilder.setMethodName("scan");
RequestHeader header = headerBuilder.build();
//build an empty scan request //build an empty scan request
ScanRequest.Builder scanBuilder = ScanRequest.newBuilder(); ScanRequest.Builder scanBuilder = ScanRequest.newBuilder();
ByteString requestBody = scanBuilder.build().toByteString(); ScanRequest scanRequest = scanBuilder.build();
rpcRequestBuilder.setRequest(requestBody); HRegion mockRegion = Mockito.mock(HRegion.class);
RpcRequestBody rpcRequest = rpcRequestBuilder.build(); HRegionServer mockRS = Mockito.mock(HRegionServer.class);
assertTrue (qosFunction.apply(rpcRequest) == HConstants.NORMAL_QOS); HRegionInfo mockRegionInfo = Mockito.mock(HRegionInfo.class);
Mockito.when(mockRS.getRegion((RegionSpecifier)Mockito.any())).thenReturn(mockRegion);
Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
Mockito.when(mockRegionInfo.isMetaRegion()).thenReturn(false);
qosFunction.setRegionServer(mockRS);
int qos = qosFunction.apply(new Pair<RequestHeader, Message>(header, scanRequest));
assertTrue ("" + qos, qos == HConstants.NORMAL_QOS);
//build a scan request with scannerID //build a scan request with scannerID
scanBuilder = ScanRequest.newBuilder(); scanBuilder = ScanRequest.newBuilder();
scanBuilder.setScannerId(12345); scanBuilder.setScannerId(12345);
requestBody = scanBuilder.build().toByteString(); scanRequest = scanBuilder.build();
rpcRequestBuilder.setRequest(requestBody);
rpcRequestBuilder.setRequestClassName(ScanRequest.class.getCanonicalName());
rpcRequest = rpcRequestBuilder.build();
//mock out a high priority type handling and see the QoS returned //mock out a high priority type handling and see the QoS returned
HRegionServer mockRS = Mockito.mock(HRegionServer.class);
RegionScanner mockRegionScanner = Mockito.mock(RegionScanner.class); RegionScanner mockRegionScanner = Mockito.mock(RegionScanner.class);
HRegionInfo mockRegionInfo = Mockito.mock(HRegionInfo.class);
HRegion mockRegion = Mockito.mock(HRegion.class);
Mockito.when(mockRS.getScanner(12345)).thenReturn(mockRegionScanner); Mockito.when(mockRS.getScanner(12345)).thenReturn(mockRegionScanner);
Mockito.when(mockRegionScanner.getRegionInfo()).thenReturn(mockRegionInfo); Mockito.when(mockRegionScanner.getRegionInfo()).thenReturn(mockRegionInfo);
Mockito.when(mockRS.getRegion((RegionSpecifier)Mockito.any())).thenReturn(mockRegion); Mockito.when(mockRS.getRegion((RegionSpecifier)Mockito.any())).thenReturn(mockRegion);
Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo); Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
Mockito.when(mockRegionInfo.isMetaTable()).thenReturn(true); Mockito.when(mockRegionInfo.isMetaRegion()).thenReturn(true);
qosFunction.setRegionServer(mockRS); qosFunction.setRegionServer(mockRS);
assertTrue (qosFunction.apply(rpcRequest) == HConstants.HIGH_QOS); assertTrue (qosFunction.apply(new Pair<RequestHeader, Message>(header, scanRequest)) ==
HConstants.HIGH_QOS);
//the same as above but with non-meta region //the same as above but with non-meta region
Mockito.when(mockRegionInfo.isMetaTable()).thenReturn(false); Mockito.when(mockRegionInfo.isMetaRegion()).thenReturn(false);
assertTrue (qosFunction.apply(rpcRequest) == HConstants.NORMAL_QOS); assertTrue (qosFunction.apply(new Pair<RequestHeader, Message>(header, scanRequest)) ==
HConstants.NORMAL_QOS);
} }
} }

View File

@ -29,6 +29,7 @@ import java.util.concurrent.CountDownLatch;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -39,8 +40,11 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.LargeTests; import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.exceptions.SnapshotCreationException; import org.apache.hadoop.hbase.exceptions.SnapshotCreationException;
import org.apache.hadoop.hbase.exceptions.TableNotFoundException; import org.apache.hadoop.hbase.exceptions.TableNotFoundException;
import org.apache.hadoop.hbase.ipc.HBaseClient;
import org.apache.hadoop.hbase.ipc.HBaseServer;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.ScannerCallable;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
@ -53,6 +57,7 @@ import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HBaseFsck; import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.log4j.Level;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Before; import org.junit.Before;
@ -83,6 +88,9 @@ public class TestFlushSnapshotFromClient {
*/ */
@BeforeClass @BeforeClass
public static void setupCluster() throws Exception { public static void setupCluster() throws Exception {
((Log4JLogger)HBaseServer.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)HBaseClient.LOG).getLogger().setLevel(Level.ALL);
((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
setupConf(UTIL.getConfiguration()); setupConf(UTIL.getConfiguration());
UTIL.startMiniCluster(NUM_RS); UTIL.startMiniCluster(NUM_RS);
} }

View File

@ -248,6 +248,8 @@ public class TestThriftServer {
Thread.sleep(1000); Thread.sleep(1000);
long lv = handler.get(tableAname, rowAname, columnAname, null).get(0).value.getLong(); long lv = handler.get(tableAname, rowAname, columnAname, null).get(0).value.getLong();
// Wait on all increments being flushed
while (handler.coalescer.getQueueSize() != 0) Threads.sleep(10);
assertEquals((100 + (2 * numIncrements)), lv ); assertEquals((100 + (2 * numIncrements)), lv );

View File

@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.thrift.generated.Hbase;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge; import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.protocol.TBinaryProtocol;
import org.apache.thrift.protocol.TCompactProtocol; import org.apache.thrift.protocol.TCompactProtocol;
import org.apache.thrift.protocol.TProtocol; import org.apache.thrift.protocol.TProtocol;