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:
parent
75105027a1
commit
5fde90f68f
|
@ -90,7 +90,6 @@ import org.apache.hadoop.hbase.util.Pair;
|
|||
import org.apache.hadoop.hbase.util.SoftValueSortedMap;
|
||||
import org.apache.hadoop.hbase.util.Triple;
|
||||
import org.apache.hadoop.hbase.zookeeper.*;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
|
@ -523,7 +522,6 @@ public class HConnectionManager {
|
|||
// package protected for the tests
|
||||
ClusterStatusListener clusterStatusListener;
|
||||
|
||||
private final Object metaRegionLock = new Object();
|
||||
private final Object userRegionLock = new Object();
|
||||
|
||||
// We have a single lock for master & zk to prevent deadlocks. Having
|
||||
|
@ -645,7 +643,7 @@ public class HConnectionManager {
|
|||
* @return
|
||||
*/
|
||||
public String toString(){
|
||||
return "hconnection 0x" + Integer.toHexString( hashCode() );
|
||||
return "hconnection-0x" + Integer.toHexString(hashCode());
|
||||
}
|
||||
|
||||
private String clusterId = null;
|
||||
|
@ -882,7 +880,7 @@ public class HConnectionManager {
|
|||
MetaScanner.metaScan(conf, visitor, tableName);
|
||||
return available.get() && (regionCount.get() > 0);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean isTableAvailable(final byte[] tableName, final byte[][] splitKeys)
|
||||
throws IOException {
|
||||
|
@ -1011,13 +1009,16 @@ public class HConnectionManager {
|
|||
if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
|
||||
ZooKeeperKeepAliveConnection zkw = getKeepAliveZooKeeperWatcher();
|
||||
try {
|
||||
LOG.debug("Looking up meta region location in ZK," +
|
||||
" connection=" + this);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Looking up meta region location in ZK," + " connection=" + this);
|
||||
}
|
||||
ServerName servername =
|
||||
MetaRegionTracker.blockUntilAvailable(zkw, this.rpcTimeout);
|
||||
|
||||
LOG.debug("Looked up meta region location, connection=" + this +
|
||||
"; serverName=" + ((servername == null) ? "null" : servername));
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.debug("Looked up meta region location, connection=" + this +
|
||||
"; serverName=" + ((servername == null) ? "null" : servername));
|
||||
}
|
||||
if (servername == null) return null;
|
||||
return new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, servername, 0);
|
||||
} catch (InterruptedException e) {
|
||||
|
@ -1821,26 +1822,17 @@ public class HConnectionManager {
|
|||
}
|
||||
|
||||
@Deprecated
|
||||
private <R> Callable<MultiResponse> createCallable(
|
||||
final HRegionLocation loc, final MultiAction<R> multi,
|
||||
final byte [] tableName) {
|
||||
// TODO: This does not belong in here!!! St.Ack HConnections should
|
||||
private <R> Callable<MultiResponse> createCallable(final HRegionLocation loc,
|
||||
final MultiAction<R> multi, final byte[] tableName) {
|
||||
// TODO: This does not belong in here!!! St.Ack HConnections should
|
||||
// not be dealing in Callables; Callables have HConnections, not other
|
||||
// way around.
|
||||
final HConnection connection = this;
|
||||
return new Callable<MultiResponse>() {
|
||||
public MultiResponse call() throws IOException {
|
||||
@Override
|
||||
public MultiResponse call() throws Exception {
|
||||
ServerCallable<MultiResponse> callable =
|
||||
new ServerCallable<MultiResponse>(connection, tableName, null) {
|
||||
public MultiResponse call() throws IOException {
|
||||
return ProtobufUtil.multi(server, multi);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void connect(boolean reload) throws IOException {
|
||||
server = connection.getClient(loc.getServerName());
|
||||
}
|
||||
};
|
||||
new MultiServerCallable<R>(connection, tableName, loc, multi);
|
||||
return callable.withoutRetries();
|
||||
}
|
||||
};
|
||||
|
@ -2162,8 +2154,7 @@ public class HConnectionManager {
|
|||
} else // success
|
||||
if (callback != null) {
|
||||
this.callback.update(resultsForRS.getKey(),
|
||||
this.rows.get(regionResult.getFirst()).getRow(),
|
||||
(R) result);
|
||||
this.rows.get(regionResult.getFirst()).getRow(), (R) result);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2222,8 +2213,6 @@ public class HConnectionManager {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* 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.
|
||||
|
|
|
@ -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.filter.BinaryComparator;
|
||||
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.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
|
@ -728,9 +729,11 @@ public class HTable implements HTableInterface {
|
|||
try {
|
||||
MutateRequest request = RequestConverter.buildMutateRequest(
|
||||
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;
|
||||
return ProtobufUtil.toResult(response.getResult());
|
||||
return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
|
@ -752,8 +755,9 @@ public class HTable implements HTableInterface {
|
|||
try {
|
||||
MutateRequest request = RequestConverter.buildMutateRequest(
|
||||
location.getRegionInfo().getRegionName(), increment);
|
||||
MutateResponse response = server.mutate(null, request);
|
||||
return ProtobufUtil.toResult(response.getResult());
|
||||
PayloadCarryingRpcController rpcContoller = new PayloadCarryingRpcController();
|
||||
MutateResponse response = server.mutate(rpcContoller, request);
|
||||
return ProtobufUtil.toResult(response.getResult(), rpcContoller.cellScanner());
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
|
@ -796,8 +800,10 @@ public class HTable implements HTableInterface {
|
|||
MutateRequest request = RequestConverter.buildMutateRequest(
|
||||
location.getRegionInfo().getRegionName(), row, family,
|
||||
qualifier, amount, writeToWAL);
|
||||
MutateResponse response = server.mutate(null, request);
|
||||
Result result = ProtobufUtil.toResult(response.getResult());
|
||||
PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
|
||||
MutateResponse response = server.mutate(rpcController, request);
|
||||
Result result =
|
||||
ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
|
||||
return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier)));
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -19,13 +19,11 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
|
@ -52,10 +50,8 @@ public class RowMutations implements Row {
|
|||
* @param row row key
|
||||
*/
|
||||
public RowMutations(byte [] row) {
|
||||
if(row == null || row.length > HConstants.MAX_ROW_LENGTH) {
|
||||
throw new IllegalArgumentException("Row key is invalid");
|
||||
}
|
||||
this.row = Arrays.copyOf(row, row.length);
|
||||
Mutation.checkRow(row);
|
||||
this.row = Bytes.copy(row);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -78,10 +74,10 @@ public class RowMutations implements Row {
|
|||
|
||||
private void internalAdd(Mutation m) throws IOException {
|
||||
int res = Bytes.compareTo(this.row, m.getRow());
|
||||
if(res != 0) {
|
||||
throw new IOException("The row in the recently added Put/Delete " +
|
||||
Bytes.toStringBinary(m.getRow()) + " doesn't match the original one " +
|
||||
Bytes.toStringBinary(this.row));
|
||||
if (res != 0) {
|
||||
throw new WrongRowIOException("The row in the recently added Put/Delete <" +
|
||||
Bytes.toStringBinary(m.getRow()) + "> doesn't match the original one <" +
|
||||
Bytes.toStringBinary(this.row) + ">");
|
||||
}
|
||||
mutations.add(m);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -19,24 +19,50 @@
|
|||
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import com.google.protobuf.CodedOutputStream;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.Message.Builder;
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.BufferedOutputStream;
|
||||
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.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.IpcProtocol;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos;
|
||||
import org.apache.hadoop.hbase.codec.Codec;
|
||||
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.RpcException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestHeader;
|
||||
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.ExceptionResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.Tracing.RPCTInfo;
|
||||
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.io.IOUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.io.compress.CompressionCodec;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
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.Trace;
|
||||
|
||||
import javax.net.SocketFactory;
|
||||
import javax.security.sasl.SaslException;
|
||||
import java.io.BufferedInputStream;
|
||||
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;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.Message.Builder;
|
||||
import com.google.protobuf.TextFormat;
|
||||
|
||||
|
||||
/** 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.
|
||||
*
|
||||
* <p>This is the org.apache.hadoop.ipc.Client renamed as HBaseClient and
|
||||
* moved into this package so can access package-private methods.
|
||||
*
|
||||
* See HBaseServer
|
||||
* <p>See HBaseServer
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
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;
|
||||
private static final Map<String, Method> methodInstances =
|
||||
new ConcurrentHashMap<String, Method>();
|
||||
private ReflectionCache reflectionCache = new ReflectionCache();
|
||||
|
||||
protected int counter; // counter for call ids
|
||||
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 socketTimeout; // socket timeout
|
||||
protected FailedServers failedServers;
|
||||
private final Codec codec;
|
||||
private final CompressionCodec compressor;
|
||||
private final IPCUtil ipcUtil;
|
||||
|
||||
protected final SocketFactory socketFactory; // how to create sockets
|
||||
protected String clusterId;
|
||||
|
@ -187,9 +186,8 @@ public class HBaseClient {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("serial")
|
||||
public static class FailedServerException extends IOException {
|
||||
private static final long serialVersionUID = -4744376109431464127L;
|
||||
|
||||
public FailedServerException(String s) {
|
||||
super(s);
|
||||
}
|
||||
|
@ -201,6 +199,8 @@ public class HBaseClient {
|
|||
* @param conf Configuration
|
||||
* @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) {
|
||||
conf.setInt(PING_INTERVAL_NAME, pingInterval);
|
||||
}
|
||||
|
@ -235,20 +235,34 @@ public class HBaseClient {
|
|||
/** A call waiting for a value. */
|
||||
protected class Call {
|
||||
final int id; // call id
|
||||
final RpcRequestBody param; // rpc request object
|
||||
Message value; // value, null if error
|
||||
final Message param; // rpc request method param object
|
||||
/**
|
||||
* 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
|
||||
boolean done; // true when call is done
|
||||
long startTime;
|
||||
final Method method;
|
||||
|
||||
protected Call(RpcRequestBody param) {
|
||||
protected Call(final Method method, Message param, final CellScanner cells) {
|
||||
this.param = param;
|
||||
this.method = method;
|
||||
this.cells = cells;
|
||||
this.startTime = System.currentTimeMillis();
|
||||
synchronized (HBaseClient.this) {
|
||||
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
|
||||
* value or error are available. Notifies by default. */
|
||||
protected synchronized void callComplete() {
|
||||
|
@ -269,10 +283,12 @@ public class HBaseClient {
|
|||
/** Set the return value when there is no error.
|
||||
* 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) {
|
||||
this.value = value;
|
||||
public synchronized void setResponse(Message response, final CellScanner cells) {
|
||||
this.response = response;
|
||||
this.cells = cells;
|
||||
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>>();
|
||||
static {
|
||||
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.
|
||||
* @param remoteId - the ConnectionId to use for the connection creation.
|
||||
*/
|
||||
protected Connection createConnection(ConnectionId remoteId) throws IOException {
|
||||
return new Connection(remoteId);
|
||||
protected Connection createConnection(ConnectionId remoteId, final Codec codec,
|
||||
final CompressionCodec compressor)
|
||||
throws IOException {
|
||||
return new Connection(remoteId, codec, compressor);
|
||||
}
|
||||
|
||||
/** Thread that reads responses and notifies callers. Each connection owns a
|
||||
|
@ -312,6 +330,8 @@ public class HBaseClient {
|
|||
private Token<? extends TokenIdentifier> token;
|
||||
private HBaseSaslRpcClient saslRpcClient;
|
||||
private int reloginMaxBackoff; // max pause before relogin on sasl failure
|
||||
private final Codec codec;
|
||||
private final CompressionCodec compressor;
|
||||
|
||||
// currently active calls
|
||||
protected final ConcurrentSkipListMap<Integer, Call> calls =
|
||||
|
@ -322,12 +342,14 @@ public class HBaseClient {
|
|||
new AtomicBoolean(); // indicate if the connection is closed
|
||||
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()) {
|
||||
throw new UnknownHostException("unknown host: " +
|
||||
remoteId.getAddress().getHostName());
|
||||
throw new UnknownHostException("unknown host: " + remoteId.getAddress().getHostName());
|
||||
}
|
||||
this.server = remoteId.getAddress();
|
||||
this.codec = codec;
|
||||
this.compressor = compressor;
|
||||
|
||||
UserGroupInformation ticket = remoteId.getTicket().getUGI();
|
||||
Class<?> protocol = remoteId.getProtocol();
|
||||
|
@ -368,29 +390,33 @@ public class HBaseClient {
|
|||
authMethod = AuthMethod.KERBEROS;
|
||||
}
|
||||
|
||||
if (LOG.isDebugEnabled())
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Use " + authMethod + " authentication for protocol "
|
||||
+ protocol.getSimpleName());
|
||||
|
||||
}
|
||||
reloginMaxBackoff = conf.getInt("hbase.security.relogin.maxbackoff", 5000);
|
||||
this.remoteId = remoteId;
|
||||
|
||||
ConnectionHeader.Builder builder = ConnectionHeader.newBuilder();
|
||||
builder.setProtocol(protocol == null ? "" : protocol.getName());
|
||||
UserInformation userInfoPB;
|
||||
if ((userInfoPB = getUserInfoPB(ticket)) != null) {
|
||||
if ((userInfoPB = getUserInfo(ticket)) != null) {
|
||||
builder.setUserInfo(userInfoPB);
|
||||
}
|
||||
builder.setCellBlockCodecClass(this.codec.getClass().getCanonicalName());
|
||||
if (this.compressor != null) {
|
||||
builder.setCellBlockCompressorClass(this.compressor.getClass().getCanonicalName());
|
||||
}
|
||||
this.header = builder.build();
|
||||
|
||||
this.setName("IPC Client (" + socketFactory.hashCode() +") connection to " +
|
||||
remoteId.getAddress().toString() +
|
||||
((ticket==null)?" from an unknown user": (" from "
|
||||
((ticket==null)?" from an unknown user": (" from "
|
||||
+ ticket.getUserName())));
|
||||
this.setDaemon(true);
|
||||
}
|
||||
|
||||
private UserInformation getUserInfoPB(UserGroupInformation ugi) {
|
||||
private UserInformation getUserInfo(UserGroupInformation ugi) {
|
||||
if (ugi == null || authMethod == AuthMethod.DIGEST) {
|
||||
// Don't send user for token auth
|
||||
return null;
|
||||
|
@ -582,8 +608,7 @@ public class HBaseClient {
|
|||
*/
|
||||
protected synchronized boolean waitForWork() {
|
||||
if (calls.isEmpty() && !shouldCloseConnection.get() && running.get()) {
|
||||
long timeout = maxIdleTime-
|
||||
(System.currentTimeMillis()-lastActivity.get());
|
||||
long timeout = maxIdleTime - (System.currentTimeMillis()-lastActivity.get());
|
||||
if (timeout>0) {
|
||||
try {
|
||||
wait(timeout);
|
||||
|
@ -613,6 +638,7 @@ public class HBaseClient {
|
|||
* since last I/O activity is equal to or greater than the ping interval
|
||||
*/
|
||||
protected synchronized void sendPing() throws IOException {
|
||||
// Can we do tcp keepalive instead of this pinging?
|
||||
long curTime = System.currentTimeMillis();
|
||||
if ( curTime - lastActivity.get() >= pingInterval) {
|
||||
lastActivity.set(curTime);
|
||||
|
@ -626,24 +652,23 @@ public class HBaseClient {
|
|||
|
||||
@Override
|
||||
public void run() {
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug(getName() + ": starting, having connections "
|
||||
+ connections.size());
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(getName() + ": starting, connections " + connections.size());
|
||||
}
|
||||
|
||||
try {
|
||||
while (waitForWork()) {//wait here for work - read or close connection
|
||||
receiveResponse();
|
||||
readResponse();
|
||||
}
|
||||
} 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));
|
||||
}
|
||||
|
||||
close();
|
||||
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug(getName() + ": stopped, remaining connections "
|
||||
+ connections.size());
|
||||
LOG.debug(getName() + ": stopped, connections " + connections.size());
|
||||
}
|
||||
|
||||
private synchronized void disposeSasl() {
|
||||
|
@ -691,7 +716,7 @@ public class HBaseClient {
|
|||
* method. In case when the user doesn't have valid credentials, we don't
|
||||
* need to retry (from cache or ticket). In such cases, it is prudent to
|
||||
* throw a runtime exception when we receive a SaslException from the
|
||||
* underlying authentication implementation, so there is no retry from
|
||||
* underlying authentication implementation, so there is no retry from
|
||||
* other high level (for eg, HCM or HBaseAdmin).
|
||||
* </p>
|
||||
*/
|
||||
|
@ -766,7 +791,7 @@ public class HBaseClient {
|
|||
|
||||
try {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Connecting to "+server);
|
||||
LOG.debug("Connecting to " + server);
|
||||
}
|
||||
short numRetries = 0;
|
||||
final short MAX_RETRIES = 5;
|
||||
|
@ -775,7 +800,8 @@ public class HBaseClient {
|
|||
setupConnection();
|
||||
InputStream inStream = NetUtils.getInputStream(socket);
|
||||
OutputStream outStream = NetUtils.getOutputStream(socket);
|
||||
writeRpcHeader(outStream);
|
||||
// Write out the preamble -- MAGIC, version, and auth to use.
|
||||
writeConnectionHeaderPreamble(outStream);
|
||||
if (useSasl) {
|
||||
final InputStream in2 = inStream;
|
||||
final OutputStream out2 = outStream;
|
||||
|
@ -787,19 +813,22 @@ public class HBaseClient {
|
|||
}
|
||||
boolean continueSasl = false;
|
||||
try {
|
||||
continueSasl =
|
||||
ticket.doAs(new PrivilegedExceptionAction<Boolean>() {
|
||||
@Override
|
||||
public Boolean run() throws IOException {
|
||||
return setupSaslConnection(in2, out2);
|
||||
}
|
||||
});
|
||||
if (ticket == null) {
|
||||
throw new NullPointerException("ticket is null");
|
||||
} else {
|
||||
continueSasl =
|
||||
ticket.doAs(new PrivilegedExceptionAction<Boolean>() {
|
||||
@Override
|
||||
public Boolean run() throws IOException {
|
||||
return setupSaslConnection(in2, out2);
|
||||
}
|
||||
});
|
||||
}
|
||||
} catch (Exception ex) {
|
||||
if (rand == null) {
|
||||
rand = new Random();
|
||||
}
|
||||
handleSaslConnectionFailure(numRetries++, MAX_RETRIES, ex, rand,
|
||||
ticket);
|
||||
handleSaslConnectionFailure(numRetries++, MAX_RETRIES, ex, rand, ticket);
|
||||
continue;
|
||||
}
|
||||
if (continueSasl) {
|
||||
|
@ -812,11 +841,10 @@ public class HBaseClient {
|
|||
useSasl = false;
|
||||
}
|
||||
}
|
||||
this.in = new DataInputStream(new BufferedInputStream
|
||||
(new PingInputStream(inStream)));
|
||||
this.out = new DataOutputStream
|
||||
(new BufferedOutputStream(outStream));
|
||||
writeHeader();
|
||||
this.in = new DataInputStream(new BufferedInputStream(new PingInputStream(inStream)));
|
||||
this.out = new DataOutputStream(new BufferedOutputStream(outStream));
|
||||
// Now write out the connection header
|
||||
writeConnectionHeader();
|
||||
|
||||
// update last activity time
|
||||
touch();
|
||||
|
@ -840,30 +868,38 @@ public class HBaseClient {
|
|||
}
|
||||
}
|
||||
|
||||
/* Write the RPC header */
|
||||
private void writeRpcHeader(OutputStream outStream) throws IOException {
|
||||
DataOutputStream out = new DataOutputStream(new BufferedOutputStream(outStream));
|
||||
// Write out the header, version and authentication method
|
||||
out.write(HConstants.RPC_HEADER.array());
|
||||
out.write(HConstants.CURRENT_VERSION);
|
||||
authMethod.write(out);
|
||||
out.flush();
|
||||
/**
|
||||
* Write the RPC header: <MAGIC WORD -- 'HBas'> <ONEBYTE_VERSION> <ONEBYTE_AUTH_TYPE>
|
||||
*/
|
||||
private void writeConnectionHeaderPreamble(OutputStream outStream) throws IOException {
|
||||
// Assemble the preamble up in a buffer first and then send it. Writing individual elements,
|
||||
// they are getting sent across piecemeal according to wireshark and then server is messing
|
||||
// up the reading on occasion (the passed in stream is not buffered yet).
|
||||
|
||||
// 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.
|
||||
*/
|
||||
private void writeHeader() throws IOException {
|
||||
// Write out the ConnectionHeader
|
||||
out.writeInt(header.getSerializedSize());
|
||||
header.writeTo(out);
|
||||
private void writeConnectionHeader() throws IOException {
|
||||
this.out.writeInt(this.header.getSerializedSize());
|
||||
this.header.writeTo(this.out);
|
||||
this.out.flush();
|
||||
}
|
||||
|
||||
/** Close the connection. */
|
||||
protected synchronized void close() {
|
||||
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;
|
||||
}
|
||||
|
||||
|
@ -883,8 +919,7 @@ public class HBaseClient {
|
|||
// clean up all calls
|
||||
if (closeException == null) {
|
||||
if (!calls.isEmpty()) {
|
||||
LOG.warn(
|
||||
"A connection is closed for no cause and calls are not empty. " +
|
||||
LOG.warn(getName() + ": connection is closed for no cause and calls are not empty. " +
|
||||
"#Calls: " + calls.size());
|
||||
|
||||
// clean up calls anyway
|
||||
|
@ -894,7 +929,7 @@ public class HBaseClient {
|
|||
} else {
|
||||
// log the info
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("closing ipc connection to " + server + ": " +
|
||||
LOG.debug(getName() + ": closing ipc connection to " + server + ": " +
|
||||
closeException.getMessage(),closeException);
|
||||
}
|
||||
|
||||
|
@ -905,126 +940,100 @@ public class HBaseClient {
|
|||
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
|
||||
* threads.
|
||||
* @param call
|
||||
* @see #readResponse()
|
||||
*/
|
||||
protected void sendParam(Call call) {
|
||||
if (shouldCloseConnection.get()) {
|
||||
return;
|
||||
}
|
||||
protected void writeRequest(Call call) {
|
||||
if (shouldCloseConnection.get()) return;
|
||||
try {
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug(getName() + " sending #" + call.id);
|
||||
|
||||
RpcRequestHeader.Builder headerBuilder = RPCProtos.RpcRequestHeader.newBuilder();
|
||||
headerBuilder.setCallId(call.id);
|
||||
|
||||
RequestHeader.Builder builder = RequestHeader.newBuilder();
|
||||
builder.setCallId(call.id);
|
||||
if (Trace.isTracing()) {
|
||||
Span s = Trace.currentTrace();
|
||||
headerBuilder.setTinfo(RPCTInfo.newBuilder()
|
||||
.setParentId(s.getSpanId())
|
||||
.setTraceId(s.getTraceId()));
|
||||
builder.setTraceInfo(RPCTInfo.newBuilder().
|
||||
setParentId(s.getSpanId()).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
|
||||
RequestHeader header = builder.build();
|
||||
synchronized (this.out) { // FindBugs IS2_INCONSISTENT_SYNC
|
||||
RpcRequestHeader header = headerBuilder.build();
|
||||
int serializedHeaderSize = header.getSerializedSize();
|
||||
int requestSerializedSize = call.param.getSerializedSize();
|
||||
this.out.writeInt(serializedHeaderSize +
|
||||
CodedOutputStream.computeRawVarint32Size(serializedHeaderSize) +
|
||||
requestSerializedSize +
|
||||
CodedOutputStream.computeRawVarint32Size(requestSerializedSize));
|
||||
header.writeDelimitedTo(this.out);
|
||||
call.param.writeDelimitedTo(this.out);
|
||||
this.out.flush();
|
||||
IPCUtil.write(this.out, header, call.param, cellBlock);
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(getName() + ": wrote request header " + TextFormat.shortDebugString(header));
|
||||
}
|
||||
} catch(IOException 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.
|
||||
* Because only one receiver, so no synchronization on in.
|
||||
*/
|
||||
protected void receiveResponse() {
|
||||
if (shouldCloseConnection.get()) {
|
||||
return;
|
||||
}
|
||||
protected void readResponse() {
|
||||
if (shouldCloseConnection.get()) return;
|
||||
touch();
|
||||
|
||||
try {
|
||||
// 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.
|
||||
RpcResponseHeader response = RpcResponseHeader.parseDelimitedFrom(in);
|
||||
if (response == null) {
|
||||
// When the stream is closed, protobuf doesn't raise an EOFException,
|
||||
// instead, it returns a null message object.
|
||||
throw new EOFException();
|
||||
// Total size of the response. Unused. But have to read it in anyways.
|
||||
/*int totalSize =*/ in.readInt();
|
||||
|
||||
// Read the header
|
||||
ResponseHeader responseHeader = ResponseHeader.parseDelimitedFrom(in);
|
||||
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);
|
||||
|
||||
Status status = response.getStatus();
|
||||
if (status == Status.SUCCESS) {
|
||||
if (responseHeader.hasException()) {
|
||||
ExceptionResponse exceptionResponse = responseHeader.getException();
|
||||
RemoteException re = createRemoteException(exceptionResponse);
|
||||
if (isFatalConnectionException(exceptionResponse)) {
|
||||
markClosed(re);
|
||||
} else {
|
||||
if (call != null) call.setException(re);
|
||||
}
|
||||
} else {
|
||||
Message rpcResponseType;
|
||||
try {
|
||||
rpcResponseType = ProtobufRpcClientEngine.Invoker.getReturnProtoType(
|
||||
getMethod(remoteId.getProtocol(),
|
||||
call.param.getMethodName()));
|
||||
// TODO: Why pb engine pollution in here in this class? FIX.
|
||||
rpcResponseType =
|
||||
ProtobufRpcClientEngine.Invoker.getReturnProtoType(
|
||||
reflectionCache.getMethod(remoteId.getProtocol(), call.method.getName()));
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e); //local exception
|
||||
}
|
||||
Builder builder = rpcResponseType.newBuilderForType();
|
||||
builder.mergeDelimitedFrom(in);
|
||||
Message value = builder.build();
|
||||
Message value = null;
|
||||
if (rpcResponseType != null) {
|
||||
Builder builder = rpcResponseType.newBuilderForType();
|
||||
builder.mergeDelimitedFrom(in);
|
||||
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
|
||||
// timeout, so check if it still exists before setting the value.
|
||||
if (call != null) {
|
||||
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) call.setResponse(value, cellBlockScanner);
|
||||
}
|
||||
if (call != null) calls.remove(id);
|
||||
} catch (IOException e) {
|
||||
if (e instanceof SocketTimeoutException && remoteId.rpcTimeout > 0) {
|
||||
// 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) {
|
||||
if (shouldCloseConnection.compareAndSet(false, true)) {
|
||||
closeException = e;
|
||||
|
@ -1103,53 +1136,13 @@ public class HBaseClient {
|
|||
/**
|
||||
* Client-side call timeout
|
||||
*/
|
||||
@SuppressWarnings("serial")
|
||||
public static class CallTimeoutException extends IOException {
|
||||
public CallTimeoutException(final String 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}
|
||||
* class.
|
||||
|
@ -1165,9 +1158,12 @@ public class HBaseClient {
|
|||
this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true);
|
||||
this.pingInterval = getPingInterval(conf);
|
||||
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.codec = getCodec(conf);
|
||||
this.compressor = getCompressor(conf);
|
||||
this.socketFactory = factory;
|
||||
this.clusterId = clusterId != null ? clusterId : HConstants.CLUSTER_ID_DEFAULT;
|
||||
this.connections = new PoolMap<ConnectionId, Connection>(
|
||||
|
@ -1175,6 +1171,35 @@ public class HBaseClient {
|
|||
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
|
||||
* @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
|
||||
* <code>address</code> which is servicing the <code>protocol</code> protocol,
|
||||
* with the <code>ticket</code> credentials, returning the value.
|
||||
* Throws exceptions if there are network problems or if the remote code
|
||||
* threw an exception. */
|
||||
public Message call(RpcRequestBody param, InetSocketAddress addr,
|
||||
Class<? extends IpcProtocol> protocol,
|
||||
User ticket, int rpcTimeout)
|
||||
* threw an exception.
|
||||
* @param method
|
||||
* @param param
|
||||
* @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 {
|
||||
Call call = new Call(param);
|
||||
Connection connection = getConnection(addr, protocol, ticket, rpcTimeout, call);
|
||||
connection.sendParam(call); // send the parameter
|
||||
Call call = new Call(method, param, cells);
|
||||
Connection connection =
|
||||
getConnection(addr, protocol, ticket, rpcTimeout, call, this.codec, this.compressor);
|
||||
connection.writeRequest(call); // send the parameter
|
||||
boolean interrupted = false;
|
||||
//noinspection SynchronizationOnLocalVariableOrMethodParameter
|
||||
synchronized (call) {
|
||||
|
@ -1305,7 +1324,7 @@ public class HBaseClient {
|
|||
// local exception
|
||||
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)
|
||||
.initCause(exception);
|
||||
} else if (exception instanceof SocketTimeoutException) {
|
||||
return (SocketTimeoutException)new SocketTimeoutException(
|
||||
"Call to " + addr + " failed on socket timeout exception: "
|
||||
+ exception).initCause(exception);
|
||||
return (SocketTimeoutException)new SocketTimeoutException("Call to " + addr +
|
||||
" failed on socket timeout exception: " + exception).initCause(exception);
|
||||
} else {
|
||||
return (IOException)new IOException(
|
||||
"Call to " + addr + " failed on local exception: " + exception)
|
||||
.initCause(exception);
|
||||
|
||||
return (IOException)new IOException("Call to " + addr + " failed on local exception: " +
|
||||
exception).initCause(exception);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1364,51 +1380,11 @@ 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
|
||||
* pool. Connections to a given host/port are reused. */
|
||||
protected Connection getConnection(InetSocketAddress addr,
|
||||
Class<? extends IpcProtocol> protocol,
|
||||
User ticket,
|
||||
int rpcTimeout,
|
||||
Call call)
|
||||
throws IOException, InterruptedException {
|
||||
protected Connection getConnection(InetSocketAddress addr, Class<? extends IpcProtocol> protocol,
|
||||
User ticket, int rpcTimeout, Call call, final Codec codec, final CompressionCodec compressor)
|
||||
throws IOException, InterruptedException {
|
||||
if (!running.get()) {
|
||||
// the client is stopped
|
||||
throw new IOException("The client is stopped");
|
||||
|
@ -1422,7 +1398,7 @@ public class HBaseClient {
|
|||
synchronized (connections) {
|
||||
connection = connections.get(remoteId);
|
||||
if (connection == null) {
|
||||
connection = createConnection(remoteId);
|
||||
connection = createConnection(remoteId, this.codec, this.compressor);
|
||||
connections.put(remoteId, connection);
|
||||
}
|
||||
}
|
||||
|
@ -1471,6 +1447,12 @@ public class HBaseClient {
|
|||
return ticket;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return this.address.toString() + "/" + this.protocol + "/" + this.ticket + "/" +
|
||||
this.rpcTimeout;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (obj instanceof ConnectionId) {
|
||||
|
@ -1484,9 +1466,9 @@ public class HBaseClient {
|
|||
|
||||
@Override // simply use the default Object#hashcode() ?
|
||||
public int hashCode() {
|
||||
return (address.hashCode() + PRIME * (
|
||||
PRIME * System.identityHashCode(protocol) ^
|
||||
(ticket == null ? 0 : ticket.hashCode()) )) ^ rpcTimeout;
|
||||
int hashcode = (address.hashCode() + PRIME * (PRIME * System.identityHashCode(protocol) ^
|
||||
(ticket == null ? 0 : ticket.hashCode()) )) ^ rpcTimeout;
|
||||
return hashcode;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -24,9 +24,10 @@ import com.google.protobuf.ServiceException;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
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.util.Pair;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -77,7 +78,8 @@ public class ProtobufRpcClientEngine implements RpcClientEngine {
|
|||
final private int rpcTimeout;
|
||||
|
||||
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.address = addr;
|
||||
this.ticket = ticket;
|
||||
|
@ -85,30 +87,6 @@ public class ProtobufRpcClientEngine implements RpcClientEngine {
|
|||
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
|
||||
* ServiceException, since the invocation proxy expects only
|
||||
|
@ -122,33 +100,51 @@ public class ProtobufRpcClientEngine implements RpcClientEngine {
|
|||
* set as cause in ServiceException</li>
|
||||
* </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
|
||||
* cause is RemoteException, then unwrap it to get the exception thrown by
|
||||
* the server.
|
||||
*/
|
||||
@Override
|
||||
public Object invoke(Object proxy, Method method, Object[] args)
|
||||
throws ServiceException {
|
||||
throws ServiceException {
|
||||
long startTime = 0;
|
||||
if (LOG.isDebugEnabled()) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
startTime = System.currentTimeMillis();
|
||||
}
|
||||
|
||||
RpcRequestBody rpcRequest = constructRpcRequest(method, args);
|
||||
Message val = null;
|
||||
if (args.length != 2) {
|
||||
throw new ServiceException(method.getName() + " didn't get two args: " + args.length);
|
||||
}
|
||||
// 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 {
|
||||
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;
|
||||
if (LOG.isTraceEnabled()) LOG.trace("Call: " + method.getName() + " " + callTime);
|
||||
}
|
||||
return val;
|
||||
return val.getFirst();
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof RemoteException) {
|
||||
Throwable cause = ((RemoteException)e).unwrapRemoteException();
|
||||
throw new ServiceException(cause);
|
||||
throw new ServiceException("methodName=" + method.getName(), cause);
|
||||
}
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
|
@ -158,8 +154,8 @@ public class ProtobufRpcClientEngine implements RpcClientEngine {
|
|||
if (returnTypes.containsKey(method.getName())) {
|
||||
return returnTypes.get(method.getName());
|
||||
}
|
||||
|
||||
Class<?> returnType = method.getReturnType();
|
||||
if (returnType.getName().equals("void")) return null;
|
||||
Method newInstMethod = returnType.getMethod("getDefaultInstance");
|
||||
newInstMethod.setAccessible(true);
|
||||
Message protoType = (Message) newInstMethod.invoke(null, (Object[]) null);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.protobuf;
|
||||
|
||||
|
||||
import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
|
@ -34,7 +35,21 @@ import java.util.Map;
|
|||
import java.util.Map.Entry;
|
||||
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.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
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.MasterAdminProtocol;
|
||||
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.Append;
|
||||
import org.apache.hadoop.hbase.client.ClientProtocol;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Increment;
|
||||
import org.apache.hadoop.hbase.client.MultiAction;
|
||||
import org.apache.hadoop.hbase.client.MultiResponse;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
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.metrics.ScanMetrics;
|
||||
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.GetRequest;
|
||||
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.Mutate;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.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.DeleteType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||
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.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.
|
||||
*/
|
||||
|
@ -342,42 +341,74 @@ 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
|
||||
* @return the converted client Put
|
||||
* @throws DoNotRetryIOException
|
||||
* @param proto The protocol buffer MutationProto to convert
|
||||
* @return A client Put.
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Put toPut(
|
||||
final Mutate proto) throws DoNotRetryIOException {
|
||||
MutateType type = proto.getMutateType();
|
||||
assert type == MutateType.PUT : type.name();
|
||||
byte[] row = proto.getRow().toByteArray();
|
||||
long timestamp = HConstants.LATEST_TIMESTAMP;
|
||||
if (proto.hasTimestamp()) {
|
||||
timestamp = proto.getTimestamp();
|
||||
public static Put toPut(final MutationProto proto)
|
||||
throws IOException {
|
||||
return toPut(proto, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a protocol buffer Mutate to a Put.
|
||||
*
|
||||
* @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()) {
|
||||
byte[] family = column.getFamily().toByteArray();
|
||||
for (QualifierValue qv: column.getQualifierValueList()) {
|
||||
byte[] qualifier = qv.getQualifier().toByteArray();
|
||||
if (!qv.hasValue()) {
|
||||
throw new DoNotRetryIOException(
|
||||
"Missing required field: qualifer value");
|
||||
}
|
||||
byte[] value = qv.getValue().toByteArray();
|
||||
long ts = timestamp;
|
||||
if (qv.hasTimestamp()) {
|
||||
ts = qv.getTimestamp();
|
||||
}
|
||||
put.add(family, qualifier, ts, value);
|
||||
}
|
||||
}
|
||||
}
|
||||
Put put = new Put(row, timestamp);
|
||||
put.setWriteToWAL(proto.getWriteToWAL());
|
||||
for (NameBytesPair attribute: proto.getAttributeList()) {
|
||||
put.setAttribute(attribute.getName(),
|
||||
attribute.getValue().toByteArray());
|
||||
}
|
||||
for (ColumnValue column: proto.getColumnValueList()) {
|
||||
byte[] family = column.getFamily().toByteArray();
|
||||
for (QualifierValue qv: column.getQualifierValueList()) {
|
||||
byte[] qualifier = qv.getQualifier().toByteArray();
|
||||
if (!qv.hasValue()) {
|
||||
throw new DoNotRetryIOException(
|
||||
"Missing required field: qualifer value");
|
||||
}
|
||||
byte[] value = qv.getValue().toByteArray();
|
||||
long ts = timestamp;
|
||||
if (qv.hasTimestamp()) {
|
||||
ts = qv.getTimestamp();
|
||||
}
|
||||
put.add(family, qualifier, ts, value);
|
||||
}
|
||||
put.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
|
||||
}
|
||||
return put;
|
||||
}
|
||||
|
@ -387,74 +418,130 @@ public final class ProtobufUtil {
|
|||
*
|
||||
* @param proto the protocol buffer Mutate to convert
|
||||
* @return the converted client Delete
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Delete toDelete(final Mutate proto) {
|
||||
MutateType type = proto.getMutateType();
|
||||
assert type == MutateType.DELETE : type.name();
|
||||
byte[] row = proto.getRow().toByteArray();
|
||||
public static Delete toDelete(final MutationProto proto)
|
||||
throws IOException {
|
||||
return toDelete(proto, null);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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;
|
||||
if (proto.hasTimestamp()) {
|
||||
timestamp = proto.getTimestamp();
|
||||
}
|
||||
Delete delete = new Delete(row, timestamp);
|
||||
delete.setWriteToWAL(proto.getWriteToWAL());
|
||||
for (NameBytesPair attribute: proto.getAttributeList()) {
|
||||
delete.setAttribute(attribute.getName(),
|
||||
attribute.getValue().toByteArray());
|
||||
}
|
||||
for (ColumnValue column: proto.getColumnValueList()) {
|
||||
byte[] family = column.getFamily().toByteArray();
|
||||
for (QualifierValue qv: column.getQualifierValueList()) {
|
||||
DeleteType deleteType = qv.getDeleteType();
|
||||
byte[] qualifier = null;
|
||||
if (qv.hasQualifier()) {
|
||||
qualifier = qv.getQualifier().toByteArray();
|
||||
Delete delete = 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));
|
||||
}
|
||||
long ts = HConstants.LATEST_TIMESTAMP;
|
||||
if (qv.hasTimestamp()) {
|
||||
ts = qv.getTimestamp();
|
||||
Cell cell = cellScanner.current();
|
||||
if (delete == null) {
|
||||
delete =
|
||||
new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), timestamp);
|
||||
}
|
||||
if (deleteType == DeleteType.DELETE_ONE_VERSION) {
|
||||
delete.deleteColumn(family, qualifier, ts);
|
||||
} else if (deleteType == DeleteType.DELETE_MULTIPLE_VERSIONS) {
|
||||
delete.deleteColumns(family, qualifier, ts);
|
||||
} else {
|
||||
delete.deleteFamily(family, ts);
|
||||
delete.addDeleteMarker(KeyValueUtil.ensureKeyValue(cell));
|
||||
}
|
||||
} else {
|
||||
delete = new Delete(row, timestamp);
|
||||
for (ColumnValue column: proto.getColumnValueList()) {
|
||||
byte[] family = column.getFamily().toByteArray();
|
||||
for (QualifierValue qv: column.getQualifierValueList()) {
|
||||
DeleteType deleteType = qv.getDeleteType();
|
||||
byte[] qualifier = null;
|
||||
if (qv.hasQualifier()) {
|
||||
qualifier = qv.getQualifier().toByteArray();
|
||||
}
|
||||
long ts = HConstants.LATEST_TIMESTAMP;
|
||||
if (qv.hasTimestamp()) {
|
||||
ts = qv.getTimestamp();
|
||||
}
|
||||
if (deleteType == DeleteType.DELETE_ONE_VERSION) {
|
||||
delete.deleteColumn(family, qualifier, ts);
|
||||
} else if (deleteType == DeleteType.DELETE_MULTIPLE_VERSIONS) {
|
||||
delete.deleteColumns(family, qualifier, ts);
|
||||
} else {
|
||||
delete.deleteFamily(family, ts);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
delete.setWriteToWAL(proto.getWriteToWAL());
|
||||
for (NameBytesPair attribute: proto.getAttributeList()) {
|
||||
delete.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
|
||||
}
|
||||
return delete;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a protocol buffer Mutate to an Append
|
||||
*
|
||||
* @param cellScanner
|
||||
* @param proto the protocol buffer Mutate to convert
|
||||
* @return the converted client Append
|
||||
* @throws DoNotRetryIOException
|
||||
*/
|
||||
public static Append toAppend(
|
||||
final Mutate proto) throws DoNotRetryIOException {
|
||||
MutateType type = proto.getMutateType();
|
||||
assert type == MutateType.APPEND : type.name();
|
||||
byte[] row = proto.getRow().toByteArray();
|
||||
Append append = new Append(row);
|
||||
public static Append toAppend(final MutationProto proto, final CellScanner cellScanner)
|
||||
throws DoNotRetryIOException {
|
||||
MutationType type = proto.getMutateType();
|
||||
assert type == MutationType.APPEND : type.name();
|
||||
byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
|
||||
Append append = 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 (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()) {
|
||||
byte[] family = column.getFamily().toByteArray();
|
||||
for (QualifierValue qv: column.getQualifierValueList()) {
|
||||
byte[] qualifier = qv.getQualifier().toByteArray();
|
||||
if (!qv.hasValue()) {
|
||||
throw new DoNotRetryIOException(
|
||||
"Missing required field: qualifer value");
|
||||
}
|
||||
byte[] value = qv.getValue().toByteArray();
|
||||
append.add(family, qualifier, value);
|
||||
}
|
||||
}
|
||||
}
|
||||
append.setWriteToWAL(proto.getWriteToWAL());
|
||||
for (NameBytesPair attribute: proto.getAttributeList()) {
|
||||
append.setAttribute(attribute.getName(),
|
||||
attribute.getValue().toByteArray());
|
||||
}
|
||||
for (ColumnValue column: proto.getColumnValueList()) {
|
||||
byte[] family = column.getFamily().toByteArray();
|
||||
for (QualifierValue qv: column.getQualifierValueList()) {
|
||||
byte[] qualifier = qv.getQualifier().toByteArray();
|
||||
if (!qv.hasValue()) {
|
||||
throw new DoNotRetryIOException(
|
||||
"Missing required field: qualifer value");
|
||||
}
|
||||
byte[] value = qv.getValue().toByteArray();
|
||||
append.add(family, qualifier, value);
|
||||
}
|
||||
append.setAttribute(attribute.getName(), attribute.getValue().toByteArray());
|
||||
}
|
||||
return append;
|
||||
}
|
||||
|
@ -466,18 +553,18 @@ public final class ProtobufUtil {
|
|||
* @return the converted Mutation
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Mutation toMutation(final Mutate proto) throws IOException {
|
||||
MutateType type = proto.getMutateType();
|
||||
if (type == MutateType.APPEND) {
|
||||
return toAppend(proto);
|
||||
public static Mutation toMutation(final MutationProto proto) throws IOException {
|
||||
MutationType type = proto.getMutateType();
|
||||
if (type == MutationType.APPEND) {
|
||||
return toAppend(proto, null);
|
||||
}
|
||||
if (type == MutateType.DELETE) {
|
||||
return toDelete(proto);
|
||||
if (type == MutationType.DELETE) {
|
||||
return toDelete(proto, null);
|
||||
}
|
||||
if (type == MutateType.PUT) {
|
||||
return toPut(proto);
|
||||
if (type == MutationType.PUT) {
|
||||
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
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Increment toIncrement(
|
||||
final Mutate proto) throws IOException {
|
||||
MutateType type = proto.getMutateType();
|
||||
assert type == MutateType.INCREMENT : type.name();
|
||||
byte[] row = proto.getRow().toByteArray();
|
||||
Increment increment = new Increment(row);
|
||||
increment.setWriteToWAL(proto.getWriteToWAL());
|
||||
public static Increment toIncrement(final MutationProto proto, final CellScanner cellScanner)
|
||||
throws IOException {
|
||||
MutationType type = proto.getMutateType();
|
||||
assert type == MutationType.INCREMENT : type.name();
|
||||
byte [] row = proto.hasRow()? proto.getRow().toByteArray(): null;
|
||||
Increment increment = 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 (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()) {
|
||||
HBaseProtos.TimeRange timeRange = proto.getTimeRange();
|
||||
long minStamp = 0;
|
||||
|
@ -506,18 +624,7 @@ public final class ProtobufUtil {
|
|||
}
|
||||
increment.setTimeRange(minStamp, maxStamp);
|
||||
}
|
||||
for (ColumnValue column: proto.getColumnValueList()) {
|
||||
byte[] family = column.getFamily().toByteArray();
|
||||
for (QualifierValue qv: column.getQualifierValueList()) {
|
||||
byte[] qualifier = qv.getQualifier().toByteArray();
|
||||
if (!qv.hasValue()) {
|
||||
throw new DoNotRetryIOException(
|
||||
"Missing required field: qualifer value");
|
||||
}
|
||||
long value = Bytes.toLong(qv.getValue().toByteArray());
|
||||
increment.addColumn(family, qualifier, value);
|
||||
}
|
||||
}
|
||||
increment.setWriteToWAL(proto.getWriteToWAL());
|
||||
return increment;
|
||||
}
|
||||
|
||||
|
@ -733,10 +840,10 @@ public final class ProtobufUtil {
|
|||
* @param increment
|
||||
* @return the converted mutate
|
||||
*/
|
||||
public static Mutate toMutate(final Increment increment) {
|
||||
Mutate.Builder builder = Mutate.newBuilder();
|
||||
public static MutationProto toMutation(final Increment increment) {
|
||||
MutationProto.Builder builder = MutationProto.newBuilder();
|
||||
builder.setRow(ByteString.copyFrom(increment.getRow()));
|
||||
builder.setMutateType(MutateType.INCREMENT);
|
||||
builder.setMutateType(MutationType.INCREMENT);
|
||||
builder.setWriteToWAL(increment.getWriteToWAL());
|
||||
TimeRange timeRange = increment.getTimeRange();
|
||||
if (!timeRange.isAllTime()) {
|
||||
|
@ -768,27 +875,14 @@ public final class ProtobufUtil {
|
|||
/**
|
||||
* Create a protocol buffer Mutate based on a client Mutation
|
||||
*
|
||||
* @param mutateType
|
||||
* @param type
|
||||
* @param mutation
|
||||
* @return a mutate
|
||||
* @return a protobuf'd Mutation
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Mutate toMutate(final MutateType mutateType,
|
||||
final Mutation mutation) throws IOException {
|
||||
Mutate.Builder mutateBuilder = Mutate.newBuilder();
|
||||
mutateBuilder.setRow(ByteString.copyFrom(mutation.getRow()));
|
||||
mutateBuilder.setMutateType(mutateType);
|
||||
mutateBuilder.setWriteToWAL(mutation.getWriteToWAL());
|
||||
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());
|
||||
}
|
||||
}
|
||||
public static MutationProto toMutation(final MutationType type, final Mutation mutation)
|
||||
throws IOException {
|
||||
MutationProto.Builder builder = getMutationBuilderAndSetCommonFields(type, mutation);
|
||||
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
|
||||
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
|
||||
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.setValue(ByteString.copyFrom(kv.getValue()));
|
||||
valueBuilder.setTimestamp(kv.getTimestamp());
|
||||
if (mutateType == MutateType.DELETE) {
|
||||
if (type == MutationType.DELETE) {
|
||||
KeyValue.Type keyValueType = KeyValue.Type.codeToType(kv.getType());
|
||||
valueBuilder.setDeleteType(toDeleteType(keyValueType));
|
||||
}
|
||||
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();
|
||||
if (cells != null) {
|
||||
for (Cell c : cells) {
|
||||
builder.addKeyValue(toKeyValue(c));
|
||||
builder.addCell(toCell(c));
|
||||
}
|
||||
}
|
||||
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
|
||||
*
|
||||
|
@ -834,12 +982,40 @@ public final class ProtobufUtil {
|
|||
* @return the converted client Result
|
||||
*/
|
||||
public static Result toResult(final ClientProtos.Result proto) {
|
||||
List<HBaseProtos.KeyValue> values = proto.getKeyValueList();
|
||||
List<KeyValue> keyValues = new ArrayList<KeyValue>(values.size());
|
||||
for (HBaseProtos.KeyValue kv: values) {
|
||||
keyValues.add(toKeyValue(kv));
|
||||
List<HBaseProtos.Cell> values = proto.getCellList();
|
||||
List<Cell> cells = new ArrayList<Cell>(values.size());
|
||||
for (HBaseProtos.Cell c: values) {
|
||||
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.
|
||||
*
|
||||
|
@ -1731,33 +1858,31 @@ public final class ProtobufUtil {
|
|||
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.
|
||||
// St.Ack 20121205
|
||||
// TODO: Do a Cell version
|
||||
HBaseProtos.KeyValue.Builder kvbuilder = HBaseProtos.KeyValue.newBuilder();
|
||||
HBaseProtos.Cell.Builder kvbuilder = HBaseProtos.Cell.newBuilder();
|
||||
kvbuilder.setRow(ByteString.copyFrom(kv.getRowArray(), kv.getRowOffset(),
|
||||
kv.getRowLength()));
|
||||
kvbuilder.setFamily(ByteString.copyFrom(kv.getFamilyArray(),
|
||||
kv.getFamilyOffset(), kv.getFamilyLength()));
|
||||
kvbuilder.setQualifier(ByteString.copyFrom(kv.getQualifierArray(),
|
||||
kv.getQualifierOffset(), kv.getQualifierLength()));
|
||||
kvbuilder.setKeyType(HBaseProtos.KeyType.valueOf(kv.getTypeByte()));
|
||||
kvbuilder.setCellType(HBaseProtos.CellType.valueOf(kv.getTypeByte()));
|
||||
kvbuilder.setTimestamp(kv.getTimestamp());
|
||||
kvbuilder.setValue(ByteString.copyFrom(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
|
||||
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.
|
||||
// St.Ack 20121205
|
||||
// TODO: Do a Cell version
|
||||
return new KeyValue(kv.getRow().toByteArray(),
|
||||
kv.getFamily().toByteArray(),
|
||||
kv.getQualifier().toByteArray(),
|
||||
kv.getTimestamp(),
|
||||
KeyValue.Type.codeToType((byte)kv.getKeyType().getNumber()),
|
||||
kv.getValue().toByteArray());
|
||||
return CellUtil.createCell(cell.getRow().toByteArray(),
|
||||
cell.getFamily().toByteArray(),
|
||||
cell.getQualifier().toByteArray(),
|
||||
cell.getTimestamp(),
|
||||
(byte)cell.getCellType().getNumber(),
|
||||
cell.getValue().toByteArray());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -17,8 +17,11 @@
|
|||
*/
|
||||
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.hbase.CellScannable;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
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.MultiGetRequest;
|
||||
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.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.HBaseProtos.CompareType;
|
||||
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.Pair;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
/**
|
||||
* Helper utility to build protocol buffer requests,
|
||||
|
@ -206,9 +208,9 @@ public final class RequestConverter {
|
|||
RegionSpecifierType.REGION_NAME, regionName);
|
||||
builder.setRegion(region);
|
||||
|
||||
Mutate.Builder mutateBuilder = Mutate.newBuilder();
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
|
||||
mutateBuilder.setRow(ByteString.copyFrom(row));
|
||||
mutateBuilder.setMutateType(MutateType.INCREMENT);
|
||||
mutateBuilder.setMutateType(MutationType.INCREMENT);
|
||||
mutateBuilder.setWriteToWAL(writeToWAL);
|
||||
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
|
||||
columnBuilder.setFamily(ByteString.copyFrom(family));
|
||||
|
@ -217,8 +219,7 @@ public final class RequestConverter {
|
|||
valueBuilder.setQualifier(ByteString.copyFrom(qualifier));
|
||||
columnBuilder.addQualifierValue(valueBuilder.build());
|
||||
mutateBuilder.addColumnValue(columnBuilder.build());
|
||||
|
||||
builder.setMutate(mutateBuilder.build());
|
||||
builder.setMutation(mutateBuilder.build());
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -245,7 +246,7 @@ public final class RequestConverter {
|
|||
builder.setRegion(region);
|
||||
Condition condition = buildCondition(
|
||||
row, family, qualifier, comparator, compareType);
|
||||
builder.setMutate(ProtobufUtil.toMutate(MutateType.PUT, put));
|
||||
builder.setMutation(ProtobufUtil.toMutation(MutationType.PUT, put));
|
||||
builder.setCondition(condition);
|
||||
return builder.build();
|
||||
}
|
||||
|
@ -273,7 +274,7 @@ public final class RequestConverter {
|
|||
builder.setRegion(region);
|
||||
Condition condition = buildCondition(
|
||||
row, family, qualifier, comparator, compareType);
|
||||
builder.setMutate(ProtobufUtil.toMutate(MutateType.DELETE, delete));
|
||||
builder.setMutation(ProtobufUtil.toMutation(MutationType.DELETE, delete));
|
||||
builder.setCondition(condition);
|
||||
return builder.build();
|
||||
}
|
||||
|
@ -292,7 +293,7 @@ public final class RequestConverter {
|
|||
RegionSpecifier region = buildRegionSpecifier(
|
||||
RegionSpecifierType.REGION_NAME, regionName);
|
||||
builder.setRegion(region);
|
||||
builder.setMutate(ProtobufUtil.toMutate(MutateType.PUT, put));
|
||||
builder.setMutation(ProtobufUtil.toMutation(MutationType.PUT, put));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -310,7 +311,7 @@ public final class RequestConverter {
|
|||
RegionSpecifier region = buildRegionSpecifier(
|
||||
RegionSpecifierType.REGION_NAME, regionName);
|
||||
builder.setRegion(region);
|
||||
builder.setMutate(ProtobufUtil.toMutate(MutateType.APPEND, append));
|
||||
builder.setMutation(ProtobufUtil.toMutation(MutationType.APPEND, append));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -327,7 +328,7 @@ public final class RequestConverter {
|
|||
RegionSpecifier region = buildRegionSpecifier(
|
||||
RegionSpecifierType.REGION_NAME, regionName);
|
||||
builder.setRegion(region);
|
||||
builder.setMutate(ProtobufUtil.toMutate(increment));
|
||||
builder.setMutation(ProtobufUtil.toMutation(increment));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -345,7 +346,7 @@ public final class RequestConverter {
|
|||
RegionSpecifier region = buildRegionSpecifier(
|
||||
RegionSpecifierType.REGION_NAME, regionName);
|
||||
builder.setRegion(region);
|
||||
builder.setMutate(ProtobufUtil.toMutate(MutateType.DELETE, delete));
|
||||
builder.setMutation(ProtobufUtil.toMutation(MutationType.DELETE, delete));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -358,29 +359,64 @@ public final class RequestConverter {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static MultiRequest buildMultiRequest(final byte[] regionName,
|
||||
final RowMutations rowMutations) throws IOException {
|
||||
MultiRequest.Builder builder = MultiRequest.newBuilder();
|
||||
RegionSpecifier region = buildRegionSpecifier(
|
||||
RegionSpecifierType.REGION_NAME, regionName);
|
||||
builder.setRegion(region);
|
||||
builder.setAtomic(true);
|
||||
final RowMutations rowMutations)
|
||||
throws IOException {
|
||||
MultiRequest.Builder builder = getMultiRequestBuilderWithRegionAndAtomicSet(regionName, true);
|
||||
for (Mutation mutation: rowMutations.getMutations()) {
|
||||
MutateType mutateType = null;
|
||||
MutationType mutateType = null;
|
||||
if (mutation instanceof Put) {
|
||||
mutateType = MutateType.PUT;
|
||||
mutateType = MutationType.PUT;
|
||||
} else if (mutation instanceof Delete) {
|
||||
mutateType = MutateType.DELETE;
|
||||
mutateType = MutationType.DELETE;
|
||||
} else {
|
||||
throw new DoNotRetryIOException(
|
||||
"RowMutations supports only put and delete, not "
|
||||
+ mutation.getClass().getName());
|
||||
throw new DoNotRetryIOException("RowMutations supports only put and delete, not " +
|
||||
mutation.getClass().getName());
|
||||
}
|
||||
Mutate mutate = ProtobufUtil.toMutate(mutateType, mutation);
|
||||
builder.addAction(MultiAction.newBuilder().setMutate(mutate).build());
|
||||
MutationProto mp = ProtobufUtil.toMutation(mutateType, mutation);
|
||||
builder.addAction(MultiAction.newBuilder().setMutation(mp).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
|
||||
*
|
||||
|
@ -475,25 +511,22 @@ public final class RequestConverter {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static <R> MultiRequest buildMultiRequest(final byte[] regionName,
|
||||
final List<Action<R>> actions) throws IOException {
|
||||
MultiRequest.Builder builder = MultiRequest.newBuilder();
|
||||
RegionSpecifier region = buildRegionSpecifier(
|
||||
RegionSpecifierType.REGION_NAME, regionName);
|
||||
builder.setRegion(region);
|
||||
final List<Action<R>> actions)
|
||||
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) {
|
||||
protoAction.setGet(ProtobufUtil.toGet((Get)row));
|
||||
} 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) {
|
||||
protoAction.setMutate(ProtobufUtil.toMutate(MutateType.DELETE, (Delete)row));
|
||||
protoAction.setMutation(ProtobufUtil.toMutation(MutationType.DELETE, (Delete)row));
|
||||
} 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) {
|
||||
protoAction.setMutate(ProtobufUtil.toMutate((Increment)row));
|
||||
protoAction.setMutation(ProtobufUtil.toMutation((Increment)row));
|
||||
} else if (row instanceof RowMutations) {
|
||||
continue; // ignore RowMutations
|
||||
} else {
|
||||
|
@ -505,6 +538,68 @@ public final class RequestConverter {
|
|||
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
|
||||
//Start utilities for Admin
|
||||
|
||||
|
|
|
@ -17,9 +17,12 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.protobuf;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.RpcController;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
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.util.StringUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.RpcController;
|
||||
|
||||
/**
|
||||
* Helper utility to build protocol buffer responses,
|
||||
|
@ -78,11 +80,13 @@ public final class ResponseConverter {
|
|||
* Get the results from a protocol buffer MultiResponse
|
||||
*
|
||||
* @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
|
||||
*/
|
||||
public static List<Object> getResults(
|
||||
final ClientProtos.MultiResponse proto) throws IOException {
|
||||
public static List<Object> getResults(final ClientProtos.MultiResponse proto,
|
||||
final CellScanner cells)
|
||||
throws IOException {
|
||||
List<Object> results = new ArrayList<Object>();
|
||||
List<ActionResult> resultList = proto.getResultList();
|
||||
for (int i = 0, n = resultList.size(); i < n; i++) {
|
||||
|
@ -90,13 +94,8 @@ public final class ResponseConverter {
|
|||
if (result.hasException()) {
|
||||
results.add(ProtobufUtil.toException(result.getException()));
|
||||
} else if (result.hasValue()) {
|
||||
ClientProtos.Result r = result.getValue();
|
||||
Object value = ProtobufUtil.toResult(r);
|
||||
if (value instanceof ClientProtos.Result) {
|
||||
results.add(ProtobufUtil.toResult((ClientProtos.Result)value));
|
||||
} else {
|
||||
results.add(value);
|
||||
}
|
||||
ClientProtos.Result value = result.getValue();
|
||||
results.add(ProtobufUtil.toResult(value, cells));
|
||||
} else {
|
||||
results.add(new Result());
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -55,8 +55,8 @@ public final class HConstants {
|
|||
/**
|
||||
* The first four bytes of Hadoop RPC connections
|
||||
*/
|
||||
public static final ByteBuffer RPC_HEADER = ByteBuffer.wrap("hrpc".getBytes());
|
||||
public static final byte CURRENT_VERSION = 5;
|
||||
public static final ByteBuffer RPC_HEADER = ByteBuffer.wrap("HBas".getBytes());
|
||||
public static final byte RPC_CURRENT_VERSION = 0;
|
||||
|
||||
// HFileBlock constants.
|
||||
|
||||
|
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -11,14 +11,14 @@ public final class MultiRowMutation {
|
|||
public interface MultiMutateRequestOrBuilder
|
||||
extends com.google.protobuf.MessageOrBuilder {
|
||||
|
||||
// repeated .Mutate mutationRequest = 1;
|
||||
java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate>
|
||||
// repeated .MutationProto mutationRequest = 1;
|
||||
java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto>
|
||||
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();
|
||||
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();
|
||||
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder getMutationRequestOrBuilder(
|
||||
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationRequestOrBuilder(
|
||||
int index);
|
||||
}
|
||||
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;
|
||||
}
|
||||
|
||||
// repeated .Mutate mutationRequest = 1;
|
||||
// repeated .MutationProto mutationRequest = 1;
|
||||
public static final int MUTATIONREQUEST_FIELD_NUMBER = 1;
|
||||
private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate> mutationRequest_;
|
||||
public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate> getMutationRequestList() {
|
||||
private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto> mutationRequest_;
|
||||
public java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto> getMutationRequestList() {
|
||||
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() {
|
||||
return mutationRequest_;
|
||||
}
|
||||
public int getMutationRequestCount() {
|
||||
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);
|
||||
}
|
||||
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateOrBuilder getMutationRequestOrBuilder(
|
||||
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationRequestOrBuilder(
|
||||
int index) {
|
||||
return mutationRequest_.get(index);
|
||||
}
|
||||
|
@ -393,7 +393,7 @@ public final class MultiRowMutation {
|
|||
break;
|
||||
}
|
||||
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);
|
||||
addMutationRequest(subBuilder.buildPartial());
|
||||
break;
|
||||
|
@ -404,20 +404,20 @@ public final class MultiRowMutation {
|
|||
|
||||
private int bitField0_;
|
||||
|
||||
// repeated .Mutate mutationRequest = 1;
|
||||
private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate> mutationRequest_ =
|
||||
// repeated .MutationProto mutationRequest = 1;
|
||||
private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto> mutationRequest_ =
|
||||
java.util.Collections.emptyList();
|
||||
private void ensureMutationRequestIsMutable() {
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
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) {
|
||||
return java.util.Collections.unmodifiableList(mutationRequest_);
|
||||
} else {
|
||||
|
@ -431,7 +431,7 @@ public final class MultiRowMutation {
|
|||
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) {
|
||||
return mutationRequest_.get(index);
|
||||
} else {
|
||||
|
@ -439,7 +439,7 @@ public final class MultiRowMutation {
|
|||
}
|
||||
}
|
||||
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 (value == null) {
|
||||
throw new NullPointerException();
|
||||
|
@ -453,7 +453,7 @@ public final class MultiRowMutation {
|
|||
return this;
|
||||
}
|
||||
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) {
|
||||
ensureMutationRequestIsMutable();
|
||||
mutationRequest_.set(index, builderForValue.build());
|
||||
|
@ -463,7 +463,7 @@ public final class MultiRowMutation {
|
|||
}
|
||||
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 (value == null) {
|
||||
throw new NullPointerException();
|
||||
|
@ -477,7 +477,7 @@ public final class MultiRowMutation {
|
|||
return this;
|
||||
}
|
||||
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 (value == null) {
|
||||
throw new NullPointerException();
|
||||
|
@ -491,7 +491,7 @@ public final class MultiRowMutation {
|
|||
return this;
|
||||
}
|
||||
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) {
|
||||
ensureMutationRequestIsMutable();
|
||||
mutationRequest_.add(builderForValue.build());
|
||||
|
@ -502,7 +502,7 @@ public final class MultiRowMutation {
|
|||
return this;
|
||||
}
|
||||
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) {
|
||||
ensureMutationRequestIsMutable();
|
||||
mutationRequest_.add(index, builderForValue.build());
|
||||
|
@ -513,7 +513,7 @@ public final class MultiRowMutation {
|
|||
return this;
|
||||
}
|
||||
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) {
|
||||
ensureMutationRequestIsMutable();
|
||||
super.addAll(values, mutationRequest_);
|
||||
|
@ -543,18 +543,18 @@ public final class MultiRowMutation {
|
|||
}
|
||||
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) {
|
||||
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) {
|
||||
if (mutationRequestBuilder_ == null) {
|
||||
return mutationRequest_.get(index); } else {
|
||||
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() {
|
||||
if (mutationRequestBuilder_ != null) {
|
||||
return mutationRequestBuilder_.getMessageOrBuilderList();
|
||||
|
@ -562,25 +562,25 @@ public final class MultiRowMutation {
|
|||
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(
|
||||
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) {
|
||||
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() {
|
||||
return getMutationRequestFieldBuilder().getBuilderList();
|
||||
}
|
||||
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() {
|
||||
if (mutationRequestBuilder_ == null) {
|
||||
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_,
|
||||
((bitField0_ & 0x00000001) == 0x00000001),
|
||||
getParentForChildren(),
|
||||
|
@ -1141,13 +1141,13 @@ public final class MultiRowMutation {
|
|||
descriptor;
|
||||
static {
|
||||
java.lang.String[] descriptorData = {
|
||||
"\n\026MultiRowMutation.proto\032\014Client.proto\"6" +
|
||||
"\n\022MultiMutateRequest\022 \n\017mutationRequest\030" +
|
||||
"\001 \003(\0132\007.Mutate\"\025\n\023MultiMutateResponse2R\n" +
|
||||
"\027MultiRowMutationService\0227\n\nmutateRows\022\023" +
|
||||
".MultiMutateRequest\032\024.MultiMutateRespons" +
|
||||
"eBF\n*org.apache.hadoop.hbase.protobuf.ge" +
|
||||
"neratedB\020MultiRowMutationH\001\210\001\001\240\001\001"
|
||||
"\n\026MultiRowMutation.proto\032\014Client.proto\"=" +
|
||||
"\n\022MultiMutateRequest\022\'\n\017mutationRequest\030" +
|
||||
"\001 \003(\0132\016.MutationProto\"\025\n\023MultiMutateResp" +
|
||||
"onse2R\n\027MultiRowMutationService\0227\n\nmutat" +
|
||||
"eRows\022\023.MultiMutateRequest\032\024.MultiMutate" +
|
||||
"ResponseBF\n*org.apache.hadoop.hbase.prot" +
|
||||
"obuf.generatedB\020MultiRowMutationH\001\210\001\001\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -51,7 +51,16 @@ message Get {
|
|||
}
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* A specific mutate inside a mutate request.
|
||||
* A specific mutation inside a mutate request.
|
||||
* 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 {
|
||||
required bytes row = 1;
|
||||
required MutateType mutateType = 2;
|
||||
message MutationProto {
|
||||
optional bytes row = 1;
|
||||
optional MutationType mutateType = 2;
|
||||
repeated ColumnValue columnValue = 3;
|
||||
repeated NameBytesPair attribute = 4;
|
||||
optional uint64 timestamp = 5;
|
||||
optional uint64 timestamp = 4;
|
||||
repeated NameBytesPair attribute = 5;
|
||||
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
|
||||
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;
|
||||
INCREMENT = 1;
|
||||
PUT = 2;
|
||||
|
@ -172,7 +191,7 @@ message Mutate {
|
|||
*/
|
||||
message MutateRequest {
|
||||
required RegionSpecifier region = 1;
|
||||
required Mutate mutate = 2;
|
||||
required MutationProto mutation = 2;
|
||||
optional Condition condition = 3;
|
||||
}
|
||||
|
||||
|
@ -281,7 +300,7 @@ message CoprocessorServiceResponse {
|
|||
* This is a union type - exactly one of the fields will be set.
|
||||
*/
|
||||
message MultiAction {
|
||||
optional Mutate mutate = 1;
|
||||
optional MutationProto mutation = 1;
|
||||
optional Get get = 2;
|
||||
}
|
||||
|
||||
|
|
|
@ -23,7 +23,7 @@ option java_generic_services = true;
|
|||
option optimize_for = SPEED;
|
||||
|
||||
message MultiMutateRequest {
|
||||
repeated Mutate mutationRequest = 1;
|
||||
repeated MutationProto mutationRequest = 1;
|
||||
}
|
||||
|
||||
message MultiMutateResponse {
|
||||
|
|
|
@ -15,123 +15,117 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* 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 "hbase.proto";
|
||||
|
||||
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
|
||||
option java_outer_classname = "RPCProtos";
|
||||
option java_generate_equals_and_hash = true;
|
||||
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 {
|
||||
required string effectiveUser = 1;
|
||||
optional string realUser = 2;
|
||||
}
|
||||
|
||||
// This is sent on connection setup after the connection preamble is sent.
|
||||
message ConnectionHeader {
|
||||
/** User Info beyond what is established at connection establishment
|
||||
* (applies to secure HBase setup)
|
||||
*/
|
||||
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"];
|
||||
// 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;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* The RPC request header
|
||||
*/
|
||||
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;
|
||||
// Optional Cell block Message. Included in client RequestHeader
|
||||
message CellBlockMeta {
|
||||
// Length of the following cell block. Could calculate it but convenient having it too hand.
|
||||
optional uint32 length = 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* The RPC response header
|
||||
*/
|
||||
message RpcResponseHeader {
|
||||
/** Echo back the callId the client sent */
|
||||
required uint32 callId = 1;
|
||||
/** 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 */
|
||||
// At the RPC layer, this message is used to carry
|
||||
// the server side exception to the RPC client.
|
||||
message ExceptionResponse {
|
||||
// Class name of the exception thrown from the server
|
||||
optional string exceptionClassName = 1;
|
||||
// Exception stack trace from the server side
|
||||
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;
|
||||
}
|
||||
|
|
|
@ -23,6 +23,33 @@ option java_outer_classname = "HBaseProtos";
|
|||
option java_generate_equals_and_hash = true;
|
||||
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
|
||||
* Inspired by the rest TableSchema
|
||||
|
@ -200,21 +227,6 @@ enum CompareType {
|
|||
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.
|
||||
* It doesn't have those transient parameters
|
||||
|
@ -224,7 +236,7 @@ message KeyValue {
|
|||
required bytes family = 2;
|
||||
required bytes qualifier = 3;
|
||||
optional uint64 timestamp = 4;
|
||||
optional KeyType keyType = 5;
|
||||
optional CellType keyType = 5;
|
||||
optional bytes value = 6;
|
||||
}
|
||||
|
||||
|
@ -288,4 +300,4 @@ message LongMsg {
|
|||
|
||||
message BigDecimalMsg {
|
||||
required bytes bigdecimalMsg = 1;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.client.Mutation;
|
|||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
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.MultiRowMutationService;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -301,7 +301,7 @@ public class MetaEditor {
|
|||
CoprocessorRpcChannel channel = table.coprocessorService(row);
|
||||
MultiMutateRequest.Builder mmrBuilder = MultiMutateRequest.newBuilder();
|
||||
for (Put put : puts) {
|
||||
mmrBuilder.addMutationRequest(ProtobufUtil.toMutate(MutateType.PUT, put));
|
||||
mmrBuilder.addMutationRequest(ProtobufUtil.toMutation(MutationType.PUT, put));
|
||||
}
|
||||
|
||||
MultiRowMutationService.BlockingInterface service =
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.exceptions.WrongRegionException;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
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.MultiMutateResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiRowMutationService;
|
||||
|
@ -86,9 +86,9 @@ CoprocessorService, Coprocessor {
|
|||
try {
|
||||
// set of rows to lock, sorted to avoid deadlocks
|
||||
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());
|
||||
for (Mutate m : mutateRequestList) {
|
||||
for (MutationProto m : mutateRequestList) {
|
||||
mutations.add(ProtobufUtil.toMutation(m));
|
||||
}
|
||||
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -23,36 +23,33 @@ import java.lang.reflect.InvocationTargetException;
|
|||
import java.lang.reflect.Method;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.IpcProtocol;
|
||||
import org.apache.hadoop.hbase.client.Operation;
|
||||
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
|
||||
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.security.HBasePolicyProvider;
|
||||
import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import com.google.protobuf.TextFormat;
|
||||
/**
|
||||
* The {@link RpcServerEngine} implementation for ProtoBuf-based RPCs.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
class ProtobufRpcServerEngine implements RpcServerEngine {
|
||||
private static final Log LOG =
|
||||
LogFactory.getLog("org.apache.hadoop.hbase.ipc.ProtobufRpcServerEngine");
|
||||
|
||||
ProtobufRpcServerEngine() {
|
||||
super();
|
||||
}
|
||||
|
@ -66,7 +63,6 @@ class ProtobufRpcServerEngine implements RpcServerEngine {
|
|||
metaHandlerCount, verbose, highPriorityLevel);
|
||||
}
|
||||
|
||||
|
||||
public static class Server extends HBaseServer {
|
||||
boolean verbose;
|
||||
Object instance;
|
||||
|
@ -111,10 +107,6 @@ class ProtobufRpcServerEngine implements RpcServerEngine {
|
|||
this.instance = instance;
|
||||
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(){
|
||||
if (!isSecurityEnabled ||
|
||||
|
@ -152,37 +144,20 @@ class ProtobufRpcServerEngine implements RpcServerEngine {
|
|||
* the return response has protobuf response payload. On failure, the
|
||||
* exception name and the stack trace are returned in the protobuf response.
|
||||
*/
|
||||
public Message call(Class<? extends IpcProtocol> protocol,
|
||||
RpcRequestBody rpcRequest, long receiveTime, MonitoredRPCHandler status)
|
||||
public Pair<Message, CellScanner> call(Class<? extends IpcProtocol> protocol,
|
||||
Method method, Message param, CellScanner cellScanner, long receiveTime,
|
||||
MonitoredRPCHandler status)
|
||||
throws IOException {
|
||||
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) {
|
||||
LOG.info("Call: protocol name=" + protocol.getName() +
|
||||
", method=" + methodName);
|
||||
LOG.info("callId: " + CurCall.get().id + " protocol: " + protocol.getName() +
|
||||
" method: " + method.getName());
|
||||
}
|
||||
|
||||
status.setRPC(rpcRequest.getMethodName(),
|
||||
new Object[]{rpcRequest.getRequest()}, receiveTime);
|
||||
status.setRPCPacket(rpcRequest);
|
||||
status.setRPC(method.getName(), new Object[]{param}, receiveTime);
|
||||
// TODO: Review after we add in encoded data blocks.
|
||||
status.setRPCPacket(param);
|
||||
status.resume("Servicing call");
|
||||
//get an instance of the method arg type
|
||||
Message protoType = getMethodArgType(method);
|
||||
Message param = protoType.newBuilderForType()
|
||||
.mergeFrom(rpcRequest.getRequest()).build();
|
||||
Message result;
|
||||
Object impl = null;
|
||||
if (protocol.isAssignableFrom(this.implementation)) {
|
||||
|
@ -190,57 +165,53 @@ class ProtobufRpcServerEngine implements RpcServerEngine {
|
|||
} else {
|
||||
throw new UnknownProtocolException(protocol);
|
||||
}
|
||||
|
||||
PayloadCarryingRpcController controller = null;
|
||||
long startTime = System.currentTimeMillis();
|
||||
if (method.getParameterTypes().length == 2) {
|
||||
// RpcController + Message in the method args
|
||||
// (generated code from RPC bits in .proto files have RpcController)
|
||||
result = (Message)method.invoke(impl, null, param);
|
||||
} else if (method.getParameterTypes().length == 1) {
|
||||
// Message (hand written code usually has only a single argument)
|
||||
result = (Message)method.invoke(impl, param);
|
||||
// Always create a controller. Some invocations may not pass data in but will pass
|
||||
// data out and they'll need a controller instance to carry it for them.
|
||||
controller = new PayloadCarryingRpcController(cellScanner);
|
||||
result = (Message)method.invoke(impl, controller, param);
|
||||
} else {
|
||||
throw new ServiceException("Too many parameters for method: ["
|
||||
+ method.getName() + "]" + ", allowed (at most): 2, Actual: "
|
||||
+ method.getParameterTypes().length);
|
||||
throw new ServiceException("Wrong number of parameters for method: [" +
|
||||
method.getName() + "]" + ", wanted: 2, actual: " + method.getParameterTypes().length);
|
||||
}
|
||||
int processingTime = (int) (System.currentTimeMillis() - startTime);
|
||||
int qTime = (int) (startTime-receiveTime);
|
||||
if (TRACELOG.isDebugEnabled()) {
|
||||
TRACELOG.debug("Call #" + CurCall.get().id +
|
||||
"; served=" + protocol.getSimpleName() + "#" + method.getName() +
|
||||
", queueTime=" + qTime +
|
||||
", processingTime=" + processingTime +
|
||||
", request=" + param.toString() +
|
||||
" response=" + result.toString());
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace(CurCall.get().toString() +
|
||||
" response: " + TextFormat.shortDebugString(result) +
|
||||
" served: " + protocol.getSimpleName() +
|
||||
" queueTime: " + qTime +
|
||||
" processingTime: " + processingTime);
|
||||
}
|
||||
metrics.dequeuedCall(qTime);
|
||||
metrics.processedCall(processingTime);
|
||||
|
||||
if (verbose) {
|
||||
log("Return: "+result, LOG);
|
||||
log("Return " + TextFormat.shortDebugString(result), LOG);
|
||||
}
|
||||
long responseSize = result.getSerializedSize();
|
||||
// log any RPC responses that are slower than the configured warn
|
||||
// response time or larger than configured warning size
|
||||
boolean tooSlow = (processingTime > warnResponseTime
|
||||
&& warnResponseTime > -1);
|
||||
boolean tooLarge = (responseSize > warnResponseSize
|
||||
&& warnResponseSize > -1);
|
||||
boolean tooSlow = (processingTime > warnResponseTime && warnResponseTime > -1);
|
||||
boolean tooLarge = (responseSize > warnResponseSize && warnResponseSize > -1);
|
||||
if (tooSlow || tooLarge) {
|
||||
// when tagging, we let TooLarge trump TooSmall to keep output simple
|
||||
// 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);
|
||||
buffer.append(methodName);
|
||||
buffer.append(method.getName());
|
||||
buffer.append("(");
|
||||
buffer.append(param.getClass().getName());
|
||||
buffer.append(")");
|
||||
logResponse(new Object[]{rpcRequest.getRequest()},
|
||||
methodName, buffer.toString(), (tooLarge ? "TooLarge" : "TooSlow"),
|
||||
logResponse(new Object[]{param},
|
||||
method.getName(), buffer.toString(), (tooLarge ? "TooLarge" : "TooSlow"),
|
||||
status.getClient(), startTime, processingTime, qTime,
|
||||
responseSize);
|
||||
}
|
||||
return result;
|
||||
return new Pair<Message, CellScanner>(result,
|
||||
controller != null? controller.cellScanner(): null);
|
||||
} catch (InvocationTargetException e) {
|
||||
Throwable target = e.getTargetException();
|
||||
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
|
||||
* client Operations.
|
||||
|
@ -361,10 +290,12 @@ class ProtobufRpcServerEngine implements RpcServerEngine {
|
|||
mapper.writeValueAsString(responseInfo));
|
||||
}
|
||||
}
|
||||
|
||||
protected static void log(String value, Log LOG) {
|
||||
String v = value;
|
||||
if (v != null && v.length() > 55)
|
||||
v = v.substring(0, 55)+"...";
|
||||
final int max = 100;
|
||||
if (v != null && v.length() > max)
|
||||
v = v.substring(0, max) + "...";
|
||||
LOG.info(v);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,16 +19,19 @@
|
|||
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.protobuf.Message;
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.net.InetSocketAddress;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.IpcProtocol;
|
||||
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 java.net.InetSocketAddress;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.protobuf.Message;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public interface RpcServer {
|
||||
|
@ -47,19 +50,19 @@ public interface RpcServer {
|
|||
InetSocketAddress getListenerAddress();
|
||||
|
||||
/** Called for each call.
|
||||
* @param method Method to invoke.
|
||||
* @param param parameter
|
||||
* @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
|
||||
*/
|
||||
Message call(Class<? extends IpcProtocol> protocol,
|
||||
RpcRequestBody param, long receiveTime, MonitoredRPCHandler status)
|
||||
throws IOException;
|
||||
Pair<Message, CellScanner> call(Class<? extends IpcProtocol> protocol, Method method,
|
||||
Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status)
|
||||
throws IOException;
|
||||
|
||||
void setErrorHandler(HBaseRPCErrorHandler handler);
|
||||
|
||||
void setQosFunction(Function<RpcRequestBody, Integer> newFunc);
|
||||
|
||||
void openServer();
|
||||
|
||||
void startThreads();
|
||||
|
@ -68,4 +71,6 @@ public interface RpcServer {
|
|||
* Returns the metrics instance for reporting RPC call statistics
|
||||
*/
|
||||
MetricsHBaseServer getMetrics();
|
||||
|
||||
public void setQosFunction(Function<Pair<RequestHeader, Message>, Integer> newFunc);
|
||||
}
|
|
@ -17,13 +17,13 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.mapreduce;
|
||||
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
|
||||
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.Serialization;
|
||||
import org.apache.hadoop.io.serializer.Serializer;
|
||||
|
@ -45,43 +45,41 @@ public class KeyValueSerialization implements Serialization<KeyValue> {
|
|||
}
|
||||
|
||||
public static class KeyValueDeserializer implements Deserializer<KeyValue> {
|
||||
private InputStream is;
|
||||
private DataInputStream dis;
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
this.is.close();
|
||||
this.dis.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public KeyValue deserialize(KeyValue ignore) throws IOException {
|
||||
// I can't overwrite the passed in KV, not from a proto kv, not just yet. TODO
|
||||
HBaseProtos.KeyValue proto =
|
||||
HBaseProtos.KeyValue.parseDelimitedFrom(this.is);
|
||||
return ProtobufUtil.toKeyValue(proto);
|
||||
return KeyValue.create(this.dis);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(InputStream is) throws IOException {
|
||||
this.is = is;
|
||||
this.dis = new DataInputStream(is);
|
||||
}
|
||||
}
|
||||
|
||||
public static class KeyValueSerializer implements Serializer<KeyValue> {
|
||||
private OutputStream os;
|
||||
private DataOutputStream dos;
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
this.os.close();
|
||||
this.dos.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(OutputStream os) throws IOException {
|
||||
this.os = os;
|
||||
this.dos = new DataOutputStream(os);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void serialize(KeyValue kv) throws IOException {
|
||||
ProtobufUtil.toKeyValue(kv).writeDelimitedTo(this.os);
|
||||
KeyValue.write(kv, this.dos);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -25,8 +25,8 @@ import org.apache.hadoop.hbase.client.Delete;
|
|||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
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.MutationProto;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
|
||||
import org.apache.hadoop.io.serializer.Deserializer;
|
||||
import org.apache.hadoop.io.serializer.Serialization;
|
||||
import org.apache.hadoop.io.serializer.Serializer;
|
||||
|
@ -57,7 +57,7 @@ public class MutationSerialization implements Serialization<Mutation> {
|
|||
|
||||
@Override
|
||||
public Mutation deserialize(Mutation mutation) throws IOException {
|
||||
Mutate proto = Mutate.parseDelimitedFrom(in);
|
||||
MutationProto proto = MutationProto.parseDelimitedFrom(in);
|
||||
return ProtobufUtil.toMutation(proto);
|
||||
}
|
||||
|
||||
|
@ -82,15 +82,15 @@ public class MutationSerialization implements Serialization<Mutation> {
|
|||
|
||||
@Override
|
||||
public void serialize(Mutation mutation) throws IOException {
|
||||
MutateType type;
|
||||
MutationType type;
|
||||
if (mutation instanceof Put) {
|
||||
type = MutateType.PUT;
|
||||
type = MutationType.PUT;
|
||||
} else if (mutation instanceof Delete) {
|
||||
type = MutateType.DELETE;
|
||||
type = MutationType.DELETE;
|
||||
} else {
|
||||
throw new IllegalArgumentException("Only Put and Delete are supported");
|
||||
}
|
||||
ProtobufUtil.toMutate(type, mutation).writeDelimitedTo(out);
|
||||
ProtobufUtil.toMutation(type, mutation).writeDelimitedTo(out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,7 +19,8 @@
|
|||
package org.apache.hadoop.hbase.monitoring;
|
||||
|
||||
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
|
||||
|
@ -37,8 +38,7 @@ public interface MonitoredRPCHandler extends MonitoredTask {
|
|||
public abstract boolean isRPCRunning();
|
||||
public abstract boolean isOperationRunning();
|
||||
|
||||
public abstract void setRPC(String methodName, Object [] params,
|
||||
long queueTime);
|
||||
public abstract void setRPCPacket(RpcRequestBody param);
|
||||
public abstract void setRPC(String methodName, Object [] params, long queueTime);
|
||||
public abstract void setRPCPacket(Message param);
|
||||
public abstract void setConnection(String clientAddress, int remotePort);
|
||||
}
|
||||
|
|
|
@ -18,20 +18,16 @@
|
|||
*/
|
||||
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.HashMap;
|
||||
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
|
||||
* handling frequent, short duration tasks. String concatenations and object
|
||||
|
@ -46,7 +42,7 @@ public class MonitoredRPCHandlerImpl extends MonitoredTaskImpl
|
|||
private long rpcStartTime;
|
||||
private String methodName = "";
|
||||
private Object [] params = {};
|
||||
private RpcRequestBody packet;
|
||||
private Message packet;
|
||||
|
||||
public MonitoredRPCHandlerImpl() {
|
||||
super();
|
||||
|
@ -201,7 +197,7 @@ public class MonitoredRPCHandlerImpl extends MonitoredTaskImpl
|
|||
* that it can later compute its size if asked for it.
|
||||
* @param param The protobuf received by the RPC for this call
|
||||
*/
|
||||
public void setRPCPacket(RpcRequestBody param) {
|
||||
public void setRPCPacket(Message param) {
|
||||
this.packet = param;
|
||||
}
|
||||
|
||||
|
@ -257,4 +253,4 @@ public class MonitoredRPCHandlerImpl extends MonitoredTaskImpl
|
|||
}
|
||||
return super.toString() + ", rpcMethod=" + getRPC();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignException;
|
||||
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.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
@ -205,8 +206,9 @@ public class ZKProcedureMemberRpcs implements ProcedureMemberRpcs {
|
|||
byte[] data = ZKUtil.getData(zkController.getWatcher(), path);
|
||||
LOG.debug("start proc data length is " + data.length);
|
||||
if (!ProtobufUtil.isPBMagicPrefix(data)) {
|
||||
String msg = "Data in for starting procuedure " + opName + " is illegally formatted. "
|
||||
+ "Killing the procedure.";
|
||||
String msg = "Data in for starting procuedure " + opName +
|
||||
" is illegally formatted (no pb magic). " +
|
||||
"Killing the procedure: " + Bytes.toString(data);
|
||||
LOG.error(msg);
|
||||
throw new IllegalArgumentException(msg);
|
||||
}
|
||||
|
|
|
@ -25,7 +25,6 @@ import java.lang.annotation.RetentionPolicy;
|
|||
import java.lang.management.ManagementFactory;
|
||||
import java.lang.management.MemoryUsage;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.Method;
|
||||
import java.net.BindException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.ArrayList;
|
||||
|
@ -58,14 +57,17 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Chore;
|
||||
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.CellScannable;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.HealthCheckChore;
|
||||
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.NoSuchColumnFamilyException;
|
||||
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.HBaseRPCErrorHandler;
|
||||
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.RpcClientEngine;
|
||||
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.MultiRequest;
|
||||
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.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.ScanResponse;
|
||||
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.RegionSpecifier;
|
||||
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.RegionServerReportRequest;
|
||||
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.cliffc.high_scale_lib.Counter;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import com.google.protobuf.TextFormat;
|
||||
|
||||
/**
|
||||
* 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
|
||||
this.numRetries = conf.getInt("hbase.client.retries.number", 10);
|
||||
this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY,
|
||||
10 * 1000);
|
||||
this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
|
||||
this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000);
|
||||
|
||||
this.sleeper = new Sleeper(this.msgInterval, this);
|
||||
|
@ -507,7 +508,7 @@ public class HRegionServer implements ClientProtocol,
|
|||
this.isa = this.rpcServer.getListenerAddress();
|
||||
|
||||
this.rpcServer.setErrorHandler(this);
|
||||
this.rpcServer.setQosFunction((qosFunction = new QosFunction()));
|
||||
this.rpcServer.setQosFunction((qosFunction = new QosFunction(this)));
|
||||
this.startcode = System.currentTimeMillis();
|
||||
|
||||
// login the zookeeper client principal (if using security)
|
||||
|
@ -566,152 +567,6 @@ public class HRegionServer implements ClientProtocol,
|
|||
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.
|
||||
*
|
||||
|
@ -1448,8 +1303,8 @@ public class HRegionServer implements ClientProtocol,
|
|||
Path logdir = new Path(rootDir, logName);
|
||||
if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
|
||||
|
||||
this.hlogForMeta = HLogFactory.createMetaHLog(this.fs.getBackingFs(),
|
||||
rootDir, logName, this.conf, getMetaWALActionListeners(),
|
||||
this.hlogForMeta = HLogFactory.createMetaHLog(this.fs.getBackingFs(),
|
||||
rootDir, logName, this.conf, getMetaWALActionListeners(),
|
||||
this.serverNameFromMasterPOV.toString());
|
||||
}
|
||||
return this.hlogForMeta;
|
||||
|
@ -1551,7 +1406,7 @@ public class HRegionServer implements ClientProtocol,
|
|||
".compactionChecker", uncaughtExceptionHandler);
|
||||
if (this.healthCheckChore != null) {
|
||||
Threads
|
||||
.setDaemonThreadRunning(this.healthCheckChore.getThread(), n + ".healthChecker",
|
||||
.setDaemonThreadRunning(this.healthCheckChore.getThread(), n + ".healthChecker",
|
||||
uncaughtExceptionHandler);
|
||||
}
|
||||
|
||||
|
@ -1645,17 +1500,17 @@ public class HRegionServer implements ClientProtocol,
|
|||
return getWAL(null);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("getWAL threw exception " + e);
|
||||
return null;
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public HLog getWAL(HRegionInfo regionInfo) throws IOException {
|
||||
//TODO: at some point this should delegate to the HLogFactory
|
||||
//currently, we don't care about the region as much as we care about the
|
||||
//currently, we don't care about the region as much as we care about the
|
||||
//table.. (hence checking the tablename below)
|
||||
//_ROOT_ and .META. regions have separate WAL.
|
||||
if (regionInfo != null &&
|
||||
//_ROOT_ and .META. regions have separate WAL.
|
||||
if (regionInfo != null &&
|
||||
regionInfo.isMetaTable()) {
|
||||
return getMetaWAL();
|
||||
}
|
||||
|
@ -1749,15 +1604,15 @@ public class HRegionServer implements ClientProtocol,
|
|||
if (cause != null) {
|
||||
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.newBuilder();
|
||||
ServerName sn =
|
||||
ServerName.parseVersionedServerName(this.serverNameFromMasterPOV.getVersionedBytes());
|
||||
builder.setServer(ProtobufUtil.toServerName(sn));
|
||||
builder.setErrorMessage(msg);
|
||||
hbaseMaster.reportRSFatalError(
|
||||
null,builder.build());
|
||||
hbaseMaster.reportRSFatalError(null, builder.build());
|
||||
}
|
||||
} catch (Throwable 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.
|
||||
*
|
||||
* @param controller the RPC controller
|
||||
* @param rpcc the RPC controller
|
||||
* @param request the mutate request
|
||||
* @throws ServiceException
|
||||
*/
|
||||
@Override
|
||||
public MutateResponse mutate(final RpcController controller,
|
||||
public MutateResponse mutate(final RpcController rpcc,
|
||||
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 {
|
||||
requestCount.increment();
|
||||
HRegion region = getRegion(request.getRegion());
|
||||
MutateResponse.Builder builder = MutateResponse.newBuilder();
|
||||
Mutate mutate = request.getMutate();
|
||||
MutationProto mutation = request.getMutation();
|
||||
if (!region.getRegionInfo().isMetaTable()) {
|
||||
cacheFlusher.reclaimMemStoreMemory();
|
||||
}
|
||||
Result r = null;
|
||||
Boolean processed = null;
|
||||
MutateType type = mutate.getMutateType();
|
||||
MutationType type = mutation.getMutateType();
|
||||
switch (type) {
|
||||
case APPEND:
|
||||
r = append(region, mutate);
|
||||
r = append(region, mutation, cellScanner);
|
||||
break;
|
||||
case INCREMENT:
|
||||
r = increment(region, mutate);
|
||||
r = increment(region, mutation, cellScanner);
|
||||
break;
|
||||
case PUT:
|
||||
Put put = ProtobufUtil.toPut(mutate);
|
||||
Put put = ProtobufUtil.toPut(mutation, cellScanner);
|
||||
if (request.hasCondition()) {
|
||||
Condition condition = request.getCondition();
|
||||
byte[] row = condition.getRow().toByteArray();
|
||||
|
@ -2859,7 +2720,7 @@ public class HRegionServer implements ClientProtocol,
|
|||
}
|
||||
break;
|
||||
case DELETE:
|
||||
Delete delete = ProtobufUtil.toDelete(mutate);
|
||||
Delete delete = ProtobufUtil.toDelete(mutation, cellScanner);
|
||||
if (request.hasCondition()) {
|
||||
Condition condition = request.getCondition();
|
||||
byte[] row = condition.getRow().toByteArray();
|
||||
|
@ -2890,10 +2751,15 @@ public class HRegionServer implements ClientProtocol,
|
|||
throw new DoNotRetryIOException(
|
||||
"Unsupported mutate type: " + type.name());
|
||||
}
|
||||
CellScannable cellsToReturn = null;
|
||||
if (processed != null) {
|
||||
builder.setProcessed(processed.booleanValue());
|
||||
} 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();
|
||||
} catch (IOException ie) {
|
||||
|
@ -3006,7 +2872,8 @@ public class HRegionServer implements ClientProtocol,
|
|||
if (rsh != null) {
|
||||
if (request.getNextCallSeq() != 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.
|
||||
rsh.nextCallSeq++;
|
||||
|
@ -3208,47 +3075,61 @@ public class HRegionServer implements ClientProtocol,
|
|||
/**
|
||||
* 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
|
||||
* @throws ServiceException
|
||||
*/
|
||||
@Override
|
||||
public MultiResponse multi(final RpcController controller,
|
||||
final MultiRequest request) throws ServiceException {
|
||||
public MultiResponse multi(final RpcController rpcc, final MultiRequest 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);
|
||||
List<CellScannable> cellsToReturn = null;
|
||||
try {
|
||||
HRegion region = getRegion(request.getRegion());
|
||||
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()) {
|
||||
List<Mutate> mutates = new ArrayList<Mutate>();
|
||||
// MultiAction is union type. Has a Get or a Mutate.
|
||||
for (ClientProtos.MultiAction actionUnion : request.getActionList()) {
|
||||
if (actionUnion.hasMutate()) {
|
||||
mutates.add(actionUnion.getMutate());
|
||||
if (actionUnion.hasMutation()) {
|
||||
mutations.add(actionUnion.getMutation());
|
||||
} else {
|
||||
throw new DoNotRetryIOException(
|
||||
"Unsupported atomic action type: " + actionUnion);
|
||||
throw new DoNotRetryIOException("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 {
|
||||
// Do a bunch of Actions.
|
||||
ActionResult.Builder resultBuilder = null;
|
||||
List<Mutate> mutates = new ArrayList<Mutate>();
|
||||
cellsToReturn = new ArrayList<CellScannable>(request.getActionCount());
|
||||
for (ClientProtos.MultiAction actionUnion : request.getActionList()) {
|
||||
requestCount.increment();
|
||||
this.requestCount.increment();
|
||||
ClientProtos.Result result = null;
|
||||
try {
|
||||
ClientProtos.Result result = null;
|
||||
if (actionUnion.hasGet()) {
|
||||
Get get = ProtobufUtil.toGet(actionUnion.getGet());
|
||||
Result r = region.get(get);
|
||||
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()) {
|
||||
Mutate mutate = actionUnion.getMutate();
|
||||
MutateType type = mutate.getMutateType();
|
||||
if (type != MutateType.PUT && type != MutateType.DELETE) {
|
||||
if (!mutates.isEmpty()) {
|
||||
doBatchOp(builder, region, mutates);
|
||||
mutates.clear();
|
||||
} else if (actionUnion.hasMutation()) {
|
||||
MutationProto mutation = actionUnion.getMutation();
|
||||
MutationType type = mutation.getMutateType();
|
||||
if (type != MutationType.PUT && type != MutationType.DELETE) {
|
||||
if (!mutations.isEmpty()) {
|
||||
doBatchOp(builder, region, mutations, cellScanner);
|
||||
mutations.clear();
|
||||
} else if (!region.getRegionInfo().isMetaTable()) {
|
||||
cacheFlusher.reclaimMemStoreMemory();
|
||||
}
|
||||
|
@ -3256,22 +3137,23 @@ public class HRegionServer implements ClientProtocol,
|
|||
Result r = null;
|
||||
switch (type) {
|
||||
case APPEND:
|
||||
r = append(region, mutate);
|
||||
r = append(region, mutation, cellScanner);
|
||||
break;
|
||||
case INCREMENT:
|
||||
r = increment(region, mutate);
|
||||
r = increment(region, mutation, cellScanner);
|
||||
break;
|
||||
case PUT:
|
||||
mutates.add(mutate);
|
||||
break;
|
||||
case DELETE:
|
||||
mutates.add(mutate);
|
||||
mutations.add(mutation);
|
||||
break;
|
||||
default:
|
||||
throw new DoNotRetryIOException("Unsupported mutate type: " + type.name());
|
||||
}
|
||||
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 {
|
||||
LOG.warn("Error: invalid action: " + actionUnion + ". "
|
||||
|
@ -3292,10 +3174,14 @@ public class HRegionServer implements ClientProtocol,
|
|||
builder.addResult(ResponseConverter.buildActionResult(ie));
|
||||
}
|
||||
}
|
||||
if (!mutates.isEmpty()) {
|
||||
doBatchOp(builder, region, mutates);
|
||||
if (!mutations.isEmpty()) {
|
||||
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();
|
||||
} catch (IOException ie) {
|
||||
throw new ServiceException(ie);
|
||||
|
@ -3758,15 +3644,16 @@ public class HRegionServer implements ClientProtocol,
|
|||
* Execute an append mutation.
|
||||
*
|
||||
* @param region
|
||||
* @param mutate
|
||||
* @param m
|
||||
* @param cellScanner
|
||||
* @return result to return to client if default operation should be
|
||||
* bypassed as indicated by RegionObserver, null otherwise
|
||||
* @throws IOException
|
||||
*/
|
||||
protected Result append(final HRegion region,
|
||||
final Mutate mutate) throws IOException {
|
||||
final MutationProto m, final CellScanner cellScanner) throws IOException {
|
||||
long before = EnvironmentEdgeManager.currentTimeMillis();
|
||||
Append append = ProtobufUtil.toAppend(mutate);
|
||||
Append append = ProtobufUtil.toAppend(m, cellScanner);
|
||||
Result r = null;
|
||||
if (region.getCoprocessorHost() != null) {
|
||||
r = region.getCoprocessorHost().preAppend(append);
|
||||
|
@ -3785,14 +3672,15 @@ public class HRegionServer implements ClientProtocol,
|
|||
* Execute an increment mutation.
|
||||
*
|
||||
* @param region
|
||||
* @param mutate
|
||||
* @param mutation
|
||||
* @return the Result
|
||||
* @throws IOException
|
||||
*/
|
||||
protected Result increment(final HRegion region,
|
||||
final Mutate mutate) throws IOException {
|
||||
protected Result increment(final HRegion region, final MutationProto mutation,
|
||||
final CellScanner cells)
|
||||
throws IOException {
|
||||
long before = EnvironmentEdgeManager.currentTimeMillis();
|
||||
Increment increment = ProtobufUtil.toIncrement(mutate);
|
||||
Increment increment = ProtobufUtil.toIncrement(mutation, cells);
|
||||
Result r = null;
|
||||
if (region.getCoprocessorHost() != null) {
|
||||
r = region.getCoprocessorHost().preIncrement(increment);
|
||||
|
@ -3812,12 +3700,12 @@ public class HRegionServer implements ClientProtocol,
|
|||
*
|
||||
* @param builder
|
||||
* @param region
|
||||
* @param mutates
|
||||
* @param mutations
|
||||
*/
|
||||
protected void doBatchOp(final MultiResponse.Builder builder,
|
||||
final HRegion region, final List<Mutate> mutates) {
|
||||
protected void doBatchOp(final MultiResponse.Builder builder, final HRegion region,
|
||||
final List<MutationProto> mutations, final CellScanner cells) {
|
||||
@SuppressWarnings("unchecked")
|
||||
Pair<Mutation, Integer>[] mutationsWithLocks = new Pair[mutates.size()];
|
||||
Pair<Mutation, Integer>[] mutationsWithLocks = new Pair[mutations.size()];
|
||||
long before = EnvironmentEdgeManager.currentTimeMillis();
|
||||
boolean batchContainsPuts = false, batchContainsDelete = false;
|
||||
try {
|
||||
|
@ -3825,21 +3713,20 @@ public class HRegionServer implements ClientProtocol,
|
|||
resultBuilder.setValue(ClientProtos.Result.newBuilder().build());
|
||||
ActionResult result = resultBuilder.build();
|
||||
int i = 0;
|
||||
for (Mutate m : mutates) {
|
||||
for (MutationProto m : mutations) {
|
||||
Mutation mutation;
|
||||
if (m.getMutateType() == MutateType.PUT) {
|
||||
mutation = ProtobufUtil.toPut(m);
|
||||
if (m.getMutateType() == MutationType.PUT) {
|
||||
mutation = ProtobufUtil.toPut(m, cells);
|
||||
batchContainsPuts = true;
|
||||
} else {
|
||||
mutation = ProtobufUtil.toDelete(m);
|
||||
mutation = ProtobufUtil.toDelete(m, cells);
|
||||
batchContainsDelete = true;
|
||||
}
|
||||
mutationsWithLocks[i++] = new Pair<Mutation, Integer>(mutation, null);
|
||||
builder.addResult(result);
|
||||
}
|
||||
|
||||
|
||||
requestCount.add(mutates.size());
|
||||
requestCount.add(mutations.size());
|
||||
if (!region.getRegionInfo().isMetaTable()) {
|
||||
cacheFlusher.reclaimMemStoreMemory();
|
||||
}
|
||||
|
@ -3871,7 +3758,7 @@ public class HRegionServer implements ClientProtocol,
|
|||
}
|
||||
} catch (IOException 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);
|
||||
}
|
||||
}
|
||||
|
@ -3888,25 +3775,27 @@ public class HRegionServer implements ClientProtocol,
|
|||
* Mutate a list of rows atomically.
|
||||
*
|
||||
* @param region
|
||||
* @param mutates
|
||||
* @param mutations
|
||||
* @param cellScanner if non-null, the mutation data -- the Cell content.
|
||||
* @throws IOException
|
||||
*/
|
||||
protected void mutateRows(final HRegion region,
|
||||
final List<Mutate> mutates) throws IOException {
|
||||
Mutate firstMutate = mutates.get(0);
|
||||
protected void mutateRows(final HRegion region, final List<MutationProto> mutations,
|
||||
final CellScanner cellScanner)
|
||||
throws IOException {
|
||||
MutationProto firstMutate = mutations.get(0);
|
||||
if (!region.getRegionInfo().isMetaTable()) {
|
||||
cacheFlusher.reclaimMemStoreMemory();
|
||||
}
|
||||
byte[] row = firstMutate.getRow().toByteArray();
|
||||
byte [] row = firstMutate.getRow().toByteArray();
|
||||
RowMutations rm = new RowMutations(row);
|
||||
for (Mutate mutate: mutates) {
|
||||
MutateType type = mutate.getMutateType();
|
||||
for (MutationProto mutate: mutations) {
|
||||
MutationType type = mutate.getMutateType();
|
||||
switch (mutate.getMutateType()) {
|
||||
case PUT:
|
||||
rm.add(ProtobufUtil.toPut(mutate));
|
||||
rm.add(ProtobufUtil.toPut(mutate, cellScanner));
|
||||
break;
|
||||
case DELETE:
|
||||
rm.add(ProtobufUtil.toDelete(mutate));
|
||||
rm.add(ProtobufUtil.toDelete(mutate, cellScanner));
|
||||
break;
|
||||
default:
|
||||
throw new DoNotRetryIOException(
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -279,7 +279,7 @@ public class IncrementCoalescer implements IncrementCoalescerMBean {
|
|||
LOG.error("FAILED_ICV: " + Bytes.toString(row.getTable()) + ", "
|
||||
+ Bytes.toStringBinary(row.getRowKey()) + ", "
|
||||
+ Bytes.toStringBinary(row.getFamily()) + ", "
|
||||
+ Bytes.toStringBinary(row.getQualifier()) + ", " + counter);
|
||||
+ Bytes.toStringBinary(row.getQualifier()) + ", " + counter, e);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hbase.util;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintWriter;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
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.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.ShutdownHook;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
|
||||
/**
|
||||
* 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)
|
||||
// with this, when we return the cluster is complete
|
||||
startTime = System.currentTimeMillis();
|
||||
final int maxwait = 200000;
|
||||
while (true) {
|
||||
JVMClusterUtil.MasterThread t = findActiveMaster(masters);
|
||||
if (t != null && t.master.isInitialized()) {
|
||||
return t.master.getServerName().toString();
|
||||
}
|
||||
if (System.currentTimeMillis() > startTime + 200000) {
|
||||
throw new RuntimeException("Master not initialized after 200 seconds");
|
||||
// REMOVE
|
||||
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 {
|
||||
Thread.sleep(100);
|
||||
|
@ -279,8 +290,6 @@ public class JVMClusterUtil {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
if (masters != null) {
|
||||
for (JVMClusterUtil.MasterThread t : masters) {
|
||||
while (t.master.isAlive() && !wasInterrupted) {
|
||||
|
@ -306,4 +315,4 @@ public class JVMClusterUtil {
|
|||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -38,6 +38,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.commons.logging.impl.Log4JLogger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
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.ZooKeeperConnectionException;
|
||||
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.HMaster;
|
||||
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.zookeeper.ZKTableReadOnly;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.*;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
|
@ -79,6 +83,9 @@ public class TestAdmin {
|
|||
|
||||
@BeforeClass
|
||||
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().setInt("hbase.regionserver.msginterval", 100);
|
||||
TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
|
||||
|
|
|
@ -23,14 +23,18 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.commons.logging.impl.Log4JLogger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
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.ScanResponse;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
@ -55,6 +59,9 @@ public class TestClientScannerRPCTimeout {
|
|||
|
||||
@BeforeClass
|
||||
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();
|
||||
conf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, rpcTimeout);
|
||||
conf.setStrings(HConstants.REGION_SERVER_IMPL, RegionServerWithScanTimeout.class.getName());
|
||||
|
|
|
@ -47,9 +47,9 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.commons.logging.impl.Log4JLogger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
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.CacheConfig;
|
||||
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.generated.ClientProtos.Mutate;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType;
|
||||
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.MultiRowMutation.MultiMutateRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutation.MultiRowMutationService;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
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.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
|
@ -120,6 +124,9 @@ public class TestFromClientSide {
|
|||
*/
|
||||
@BeforeClass
|
||||
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();
|
||||
conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
|
||||
MultiRowMutationEndpoint.class.getName());
|
||||
|
@ -4148,11 +4155,11 @@ public class TestFromClientSide {
|
|||
HTable t = TEST_UTIL.createTable(TABLENAME, FAMILY);
|
||||
Put p = new Put(ROW);
|
||||
p.add(FAMILY, QUALIFIER, VALUE);
|
||||
Mutate m1 = ProtobufUtil.toMutate(MutateType.PUT, p);
|
||||
MutationProto m1 = ProtobufUtil.toMutation(MutationType.PUT, p);
|
||||
|
||||
p = new Put(ROW1);
|
||||
p.add(FAMILY, QUALIFIER, VALUE);
|
||||
Mutate m2 = ProtobufUtil.toMutate(MutateType.PUT, p);
|
||||
MutationProto m2 = ProtobufUtil.toMutation(MutationType.PUT, p);
|
||||
|
||||
MultiMutateRequest.Builder mrmBuilder = MultiMutateRequest.newBuilder();
|
||||
mrmBuilder.addMutationRequest(m1);
|
||||
|
@ -4195,6 +4202,8 @@ public class TestFromClientSide {
|
|||
Delete d = new Delete(ROW);
|
||||
d.deleteColumns(FAMILY, QUALIFIERS[0]);
|
||||
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));
|
||||
r = t.get(g);
|
||||
assertEquals(0, Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIERS[1])));
|
||||
|
|
|
@ -39,4 +39,4 @@ public class TestFromClientSideWithCoprocessor extends TestFromClientSide {
|
|||
// We need more than one region server in this test
|
||||
TEST_UTIL.startMiniCluster(SLAVES);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -660,6 +660,8 @@ public class TestHCM {
|
|||
}
|
||||
}
|
||||
assertNotNull(otherRow);
|
||||
// If empty row, set it to first row.-f
|
||||
if (otherRow.length <= 0) otherRow = Bytes.toBytes("aaa");
|
||||
Put put2 = new Put(otherRow);
|
||||
put2.add(FAM_NAM, otherRow, otherRow);
|
||||
table.put(put2); //cache put2's location
|
||||
|
|
|
@ -68,6 +68,9 @@ public class TestMultiParallel {
|
|||
private static final int slaves = 2; // also used for testing HTable pool size
|
||||
|
||||
@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);
|
||||
HTable t = UTIL.createTable(Bytes.toBytes(TEST_TABLE), Bytes.toBytes(FAMILY));
|
||||
UTIL.createMultiRegions(t, Bytes.toBytes(FAMILY));
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -25,6 +25,7 @@ import java.util.List;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.commons.logging.impl.Log4JLogger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
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.exceptions.MasterNotRunningException;
|
||||
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.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.client.ScannerCallable;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import org.junit.AfterClass;
|
||||
|
@ -46,6 +50,7 @@ import org.junit.BeforeClass;
|
|||
import org.junit.Test;
|
||||
import static org.junit.Assert.*;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
|
@ -172,7 +177,9 @@ public class TestFilterWithScanLimits {
|
|||
|
||||
@BeforeClass
|
||||
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);
|
||||
initialize(TEST_UTIL.getConfiguration());
|
||||
}
|
||||
|
|
|
@ -44,6 +44,8 @@ import org.apache.log4j.spi.LoggingEvent;
|
|||
import org.junit.Test;
|
||||
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
|
||||
* 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(1).intValue());
|
||||
assertEquals(results.get(2).intValue(), delayReturnValue ? DELAYED :
|
||||
0xDEADBEEF);
|
||||
assertEquals(results.get(2).intValue(), delayReturnValue ? DELAYED : 0xDEADBEEF);
|
||||
} finally {
|
||||
clientEngine.close();
|
||||
}
|
||||
|
@ -182,7 +183,7 @@ public class TestDelayedRpc {
|
|||
}
|
||||
|
||||
public interface TestRpc extends IpcProtocol {
|
||||
TestResponse test(TestArg delay);
|
||||
TestResponse test(final Object rpcController, TestArg delay) throws ServiceException;
|
||||
}
|
||||
|
||||
private static class TestRpcImpl implements TestRpc {
|
||||
|
@ -201,7 +202,8 @@ public class TestDelayedRpc {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TestResponse test(final TestArg testArg) {
|
||||
public TestResponse test(final Object rpcController, final TestArg testArg)
|
||||
throws ServiceException {
|
||||
boolean delay = testArg.getDelay();
|
||||
TestResponse.Builder responseBuilder = TestResponse.newBuilder();
|
||||
if (!delay) {
|
||||
|
@ -243,9 +245,8 @@ public class TestDelayedRpc {
|
|||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
Integer result =
|
||||
new Integer(server.test(TestArg.newBuilder()
|
||||
.setDelay(delay).build()).getResponse());
|
||||
Integer result = new Integer(server.test(null, TestArg.newBuilder().setDelay(delay).
|
||||
build()).getResponse());
|
||||
if (results != null) {
|
||||
synchronized (results) {
|
||||
results.add(result);
|
||||
|
@ -276,7 +277,7 @@ public class TestDelayedRpc {
|
|||
int result = 0xDEADBEEF;
|
||||
|
||||
try {
|
||||
result = client.test(TestArg.newBuilder().setDelay(false).build()).getResponse();
|
||||
result = client.test(null, TestArg.newBuilder().setDelay(false).build()).getResponse();
|
||||
} catch (Exception e) {
|
||||
fail("No exception should have been thrown.");
|
||||
}
|
||||
|
@ -284,7 +285,7 @@ public class TestDelayedRpc {
|
|||
|
||||
boolean caughtException = false;
|
||||
try {
|
||||
result = client.test(TestArg.newBuilder().setDelay(true).build()).getResponse();
|
||||
result = client.test(null, TestArg.newBuilder().setDelay(true).build()).getResponse();
|
||||
} catch(Exception e) {
|
||||
// Exception thrown by server is enclosed in a RemoteException.
|
||||
if (e.getCause().getMessage().contains(
|
||||
|
@ -303,7 +304,7 @@ public class TestDelayedRpc {
|
|||
*/
|
||||
private static class FaultyTestRpc implements TestRpc {
|
||||
@Override
|
||||
public TestResponse test(TestArg arg) {
|
||||
public TestResponse test(Object rpcController, TestArg arg) {
|
||||
if (!arg.getDelay())
|
||||
return TestResponse.newBuilder().setResponse(UNDELAYED).build();
|
||||
Delayable call = HBaseServer.getCurrentCall();
|
||||
|
|
|
@ -25,21 +25,35 @@ import static org.mockito.Mockito.doThrow;
|
|||
import static org.mockito.Mockito.spy;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.Socket;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import javax.net.SocketFactory;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
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.HConstants;
|
||||
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.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RowMutations;
|
||||
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.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.util.StringUtils;
|
||||
import org.junit.Test;
|
||||
|
@ -53,6 +67,8 @@ import com.google.protobuf.Message;
|
|||
@Category(SmallTests.class)
|
||||
public class TestIPC {
|
||||
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 {
|
||||
TestRpcServer() throws IOException {
|
||||
|
@ -60,10 +76,53 @@ public class TestIPC {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Message call(Class<? extends IpcProtocol> protocol,
|
||||
RpcRequestBody param, long receiveTime, MonitoredRPCHandler status)
|
||||
public Pair<Message, CellScanner> call(Class<? extends IpcProtocol> protocol, Method method,
|
||||
Message param, final CellScanner cells, long receiveTime, MonitoredRPCHandler status)
|
||||
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();
|
||||
|
||||
TestRpcServer rpcServer = new TestRpcServer();
|
||||
rpcServer.start();
|
||||
|
||||
HBaseClient client = new HBaseClient(
|
||||
conf, HConstants.CLUSTER_ID_DEFAULT, spyFactory);
|
||||
InetSocketAddress address = rpcServer.getListenerAddress();
|
||||
|
||||
HBaseClient client = new HBaseClient(conf, HConstants.CLUSTER_ID_DEFAULT, spyFactory);
|
||||
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!");
|
||||
} catch (Exception e) {
|
||||
LOG.info("Caught expected exception: " + e.toString());
|
||||
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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,13 +22,15 @@ import java.net.InetSocketAddress;
|
|||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.IpcProtocol;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
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.EmptyRequestProto;
|
||||
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto;
|
||||
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.Test;
|
||||
import org.junit.Before;
|
||||
|
@ -40,9 +42,9 @@ import com.google.protobuf.ServiceException;
|
|||
|
||||
/**
|
||||
* Test for testing protocol buffer based RPC mechanism.
|
||||
* This test depends on test.proto definition of types in
|
||||
* This test depends on test.proto definition of types in
|
||||
* hbase-server/src/test/protobuf/test.proto
|
||||
* and protobuf service definition from
|
||||
* and protobuf service definition from
|
||||
* hbase-server/src/test/protobuf/test_rpc_service.proto
|
||||
*/
|
||||
@Category(MediumTests.class)
|
||||
|
@ -53,8 +55,7 @@ public class TestProtoBufRpc {
|
|||
private static Configuration conf;
|
||||
private static RpcServer server;
|
||||
|
||||
public interface TestRpcService
|
||||
extends TestProtobufRpcProto.BlockingInterface, IpcProtocol {
|
||||
public interface TestRpcService extends TestProtobufRpcProto.BlockingInterface, IpcProtocol {
|
||||
public long VERSION = 1;
|
||||
}
|
||||
|
||||
|
@ -83,18 +84,20 @@ public class TestProtoBufRpc {
|
|||
@Before
|
||||
public void setUp() throws IOException { // Setup server for both protocols
|
||||
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
|
||||
PBServerImpl serverImpl = new PBServerImpl();
|
||||
// Get RPC server for server side implementation
|
||||
server = HBaseServerRPC.getServer(TestRpcService.class,serverImpl,
|
||||
new Class[]{TestRpcService.class},
|
||||
new Class[]{TestRpcService.class},
|
||||
ADDRESS, PORT, 10, 10, true, conf, 0);
|
||||
addr = server.getListenerAddress();
|
||||
server.start();
|
||||
}
|
||||
|
||||
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
server.stop();
|
||||
|
@ -105,14 +108,13 @@ public class TestProtoBufRpc {
|
|||
ProtobufRpcClientEngine clientEngine =
|
||||
new ProtobufRpcClientEngine(conf, HConstants.CLUSTER_ID_DEFAULT);
|
||||
try {
|
||||
TestRpcService client = clientEngine.getProxy(TestRpcService.class, addr, conf, 10000);
|
||||
TestRpcService client = clientEngine.getProxy(TestRpcService.class, addr, conf, 100000);
|
||||
// Test ping method
|
||||
EmptyRequestProto emptyRequest = EmptyRequestProto.newBuilder().build();
|
||||
client.ping(null, emptyRequest);
|
||||
|
||||
// Test echo method
|
||||
EchoRequestProto echoRequest = EchoRequestProto.newBuilder()
|
||||
.setMessage("hello").build();
|
||||
EchoRequestProto echoRequest = EchoRequestProto.newBuilder().setMessage("hello").build();
|
||||
EchoResponseProto echoResponse = client.echo(null, echoRequest);
|
||||
Assert.assertEquals(echoResponse.getMessage(), "hello");
|
||||
|
||||
|
@ -126,4 +128,4 @@ public class TestProtoBufRpc {
|
|||
clientEngine.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,11 +31,11 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.ColumnValue.QualifierValue;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.DeleteType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Mutate.MutateType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.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.DeleteType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
@ -88,9 +88,9 @@ public class TestProtobufUtil {
|
|||
*/
|
||||
@Test
|
||||
public void testAppend() throws IOException {
|
||||
Mutate.Builder mutateBuilder = Mutate.newBuilder();
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
|
||||
mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
|
||||
mutateBuilder.setMutateType(MutateType.APPEND);
|
||||
mutateBuilder.setMutateType(MutationType.APPEND);
|
||||
mutateBuilder.setTimestamp(111111);
|
||||
ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
|
||||
valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
|
||||
|
@ -103,29 +103,27 @@ public class TestProtobufUtil {
|
|||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
mutateBuilder.addColumnValue(valueBuilder.build());
|
||||
|
||||
Mutate proto = mutateBuilder.build();
|
||||
MutationProto proto = mutateBuilder.build();
|
||||
// default fields
|
||||
assertEquals(true, proto.getWriteToWAL());
|
||||
|
||||
// set the default value for equal comparison
|
||||
mutateBuilder = Mutate.newBuilder(proto);
|
||||
mutateBuilder = MutationProto.newBuilder(proto);
|
||||
mutateBuilder.setWriteToWAL(true);
|
||||
|
||||
Append append = ProtobufUtil.toAppend(proto);
|
||||
Append append = ProtobufUtil.toAppend(proto, null);
|
||||
|
||||
// append always use the latest timestamp,
|
||||
// add the timestamp to the original mutate
|
||||
long timestamp = append.getTimeStamp();
|
||||
mutateBuilder.setTimestamp(timestamp);
|
||||
for (ColumnValue.Builder column:
|
||||
mutateBuilder.getColumnValueBuilderList()) {
|
||||
for (ColumnValue.Builder column: mutateBuilder.getColumnValueBuilderList()) {
|
||||
for (QualifierValue.Builder qualifier:
|
||||
column.getQualifierValueBuilderList()) {
|
||||
qualifier.setTimestamp(timestamp);
|
||||
}
|
||||
}
|
||||
assertEquals(mutateBuilder.build(),
|
||||
ProtobufUtil.toMutate(MutateType.APPEND, append));
|
||||
assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(MutationType.APPEND, append));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -135,9 +133,9 @@ public class TestProtobufUtil {
|
|||
*/
|
||||
@Test
|
||||
public void testDelete() throws IOException {
|
||||
Mutate.Builder mutateBuilder = Mutate.newBuilder();
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
|
||||
mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
|
||||
mutateBuilder.setMutateType(MutateType.DELETE);
|
||||
mutateBuilder.setMutateType(MutationType.DELETE);
|
||||
mutateBuilder.setTimestamp(111111);
|
||||
ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
|
||||
valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
|
||||
|
@ -152,12 +150,12 @@ public class TestProtobufUtil {
|
|||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
mutateBuilder.addColumnValue(valueBuilder.build());
|
||||
|
||||
Mutate proto = mutateBuilder.build();
|
||||
MutationProto proto = mutateBuilder.build();
|
||||
// default fields
|
||||
assertEquals(true, proto.getWriteToWAL());
|
||||
|
||||
// set the default value for equal comparison
|
||||
mutateBuilder = Mutate.newBuilder(proto);
|
||||
mutateBuilder = MutationProto.newBuilder(proto);
|
||||
mutateBuilder.setWriteToWAL(true);
|
||||
|
||||
Delete delete = ProtobufUtil.toDelete(proto);
|
||||
|
@ -172,7 +170,7 @@ public class TestProtobufUtil {
|
|||
}
|
||||
}
|
||||
assertEquals(mutateBuilder.build(),
|
||||
ProtobufUtil.toMutate(MutateType.DELETE, delete));
|
||||
ProtobufUtil.toMutation(MutationType.DELETE, delete));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -182,9 +180,9 @@ public class TestProtobufUtil {
|
|||
*/
|
||||
@Test
|
||||
public void testIncrement() throws IOException {
|
||||
Mutate.Builder mutateBuilder = Mutate.newBuilder();
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
|
||||
mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
|
||||
mutateBuilder.setMutateType(MutateType.INCREMENT);
|
||||
mutateBuilder.setMutateType(MutationType.INCREMENT);
|
||||
ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
|
||||
valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
|
||||
QualifierValue.Builder qualifierBuilder = QualifierValue.newBuilder();
|
||||
|
@ -196,16 +194,16 @@ public class TestProtobufUtil {
|
|||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
mutateBuilder.addColumnValue(valueBuilder.build());
|
||||
|
||||
Mutate proto = mutateBuilder.build();
|
||||
MutationProto proto = mutateBuilder.build();
|
||||
// default fields
|
||||
assertEquals(true, proto.getWriteToWAL());
|
||||
|
||||
// set the default value for equal comparison
|
||||
mutateBuilder = Mutate.newBuilder(proto);
|
||||
mutateBuilder = MutationProto.newBuilder(proto);
|
||||
mutateBuilder.setWriteToWAL(true);
|
||||
|
||||
Increment increment = ProtobufUtil.toIncrement(proto);
|
||||
assertEquals(mutateBuilder.build(), ProtobufUtil.toMutate(increment));
|
||||
Increment increment = ProtobufUtil.toIncrement(proto, null);
|
||||
assertEquals(mutateBuilder.build(), ProtobufUtil.toMutation(increment));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -215,9 +213,9 @@ public class TestProtobufUtil {
|
|||
*/
|
||||
@Test
|
||||
public void testPut() throws IOException {
|
||||
Mutate.Builder mutateBuilder = Mutate.newBuilder();
|
||||
MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
|
||||
mutateBuilder.setRow(ByteString.copyFromUtf8("row"));
|
||||
mutateBuilder.setMutateType(MutateType.PUT);
|
||||
mutateBuilder.setMutateType(MutationType.PUT);
|
||||
mutateBuilder.setTimestamp(111111);
|
||||
ColumnValue.Builder valueBuilder = ColumnValue.newBuilder();
|
||||
valueBuilder.setFamily(ByteString.copyFromUtf8("f1"));
|
||||
|
@ -231,12 +229,12 @@ public class TestProtobufUtil {
|
|||
valueBuilder.addQualifierValue(qualifierBuilder.build());
|
||||
mutateBuilder.addColumnValue(valueBuilder.build());
|
||||
|
||||
Mutate proto = mutateBuilder.build();
|
||||
MutationProto proto = mutateBuilder.build();
|
||||
// default fields
|
||||
assertEquals(true, proto.getWriteToWAL());
|
||||
|
||||
// set the default value for equal comparison
|
||||
mutateBuilder = Mutate.newBuilder(proto);
|
||||
mutateBuilder = MutationProto.newBuilder(proto);
|
||||
mutateBuilder.setWriteToWAL(true);
|
||||
|
||||
Put put = ProtobufUtil.toPut(proto);
|
||||
|
@ -255,7 +253,7 @@ public class TestProtobufUtil {
|
|||
}
|
||||
}
|
||||
assertEquals(mutateBuilder.build(),
|
||||
ProtobufUtil.toMutate(MutateType.PUT, put));
|
||||
ProtobufUtil.toMutation(MutationType.PUT, put));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -18,23 +18,21 @@
|
|||
*/
|
||||
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.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
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.
|
||||
|
|
|
@ -22,37 +22,25 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
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.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.experimental.categories.Category;
|
||||
|
||||
|
@ -69,7 +57,6 @@ public class TestJoinedScanners {
|
|||
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
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_joined = Bytes.toBytes("joined");
|
||||
private static final byte[] col_name = Bytes.toBytes("a");
|
||||
|
|
|
@ -28,58 +28,56 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
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.GetRequest;
|
||||
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.RegionSpecifierType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RpcRequestBody;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer.QosFunction;
|
||||
import org.junit.BeforeClass;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.Message;
|
||||
/**
|
||||
* Tests that verify certain RPCs get a higher QoS.
|
||||
*/
|
||||
@Category(MediumTests.class)
|
||||
public class TestPriorityRpc {
|
||||
static HRegionServer regionServer = null;
|
||||
static QosFunction qosFunction = null;
|
||||
@BeforeClass
|
||||
public static void onetimeSetup() {
|
||||
private HRegionServer regionServer = null;
|
||||
private QosFunction qosFunction = null;
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
regionServer =
|
||||
HRegionServer.constructRegionServer(HRegionServer.class, conf);
|
||||
regionServer = HRegionServer.constructRegionServer(HRegionServer.class, conf);
|
||||
qosFunction = regionServer.getQosFunction();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQosFunctionForMeta() throws IOException {
|
||||
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
|
||||
//uses one of the known argument classes (known argument classes are
|
||||
//listed in HRegionServer.QosFunction.knownArgumentClasses)
|
||||
rpcRequestBuilder = RpcRequestBody.newBuilder();
|
||||
rpcRequestBuilder.setMethodName("foo");
|
||||
headerBuilder.setMethodName("foo");
|
||||
|
||||
GetRequest.Builder getRequestBuilder = GetRequest.newBuilder();
|
||||
RegionSpecifier.Builder regionSpecifierBuilder = RegionSpecifier.newBuilder();
|
||||
regionSpecifierBuilder.setType(RegionSpecifierType.REGION_NAME);
|
||||
ByteString name =
|
||||
ByteString.copyFrom(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
|
||||
ByteString name = ByteString.copyFrom(HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
|
||||
regionSpecifierBuilder.setValue(name);
|
||||
RegionSpecifier regionSpecifier = regionSpecifierBuilder.build();
|
||||
getRequestBuilder.setRegion(regionSpecifier);
|
||||
Get.Builder getBuilder = Get.newBuilder();
|
||||
getBuilder.setRow(ByteString.copyFrom("somerow".getBytes()));
|
||||
getRequestBuilder.setGet(getBuilder.build());
|
||||
rpcRequestBuilder.setRequest(getRequestBuilder.build().toByteString());
|
||||
rpcRequestBuilder.setRequestClassName(GetRequest.class.getCanonicalName());
|
||||
RpcRequestBody rpcRequest = rpcRequestBuilder.build();
|
||||
GetRequest getRequest = getRequestBuilder.build();
|
||||
RequestHeader header = headerBuilder.build();
|
||||
HRegion mockRegion = Mockito.mock(HRegion.class);
|
||||
HRegionServer mockRS = Mockito.mock(HRegionServer.class);
|
||||
HRegionInfo mockRegionInfo = Mockito.mock(HRegionInfo.class);
|
||||
|
@ -87,7 +85,8 @@ public class TestPriorityRpc {
|
|||
Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
|
||||
Mockito.when(mockRegionInfo.isMetaTable()).thenReturn(true);
|
||||
qosFunction.setRegionServer(mockRS);
|
||||
assertTrue (qosFunction.apply(rpcRequest) == HConstants.HIGH_QOS);
|
||||
assertTrue (qosFunction.apply(new Pair<RequestHeader, Message>(header, getRequest)) ==
|
||||
HConstants.HIGH_QOS);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -96,51 +95,53 @@ public class TestPriorityRpc {
|
|||
//known argument classes (it uses one random request class)
|
||||
//(known argument classes are listed in
|
||||
//HRegionServer.QosFunction.knownArgumentClasses)
|
||||
RpcRequestBody.Builder rpcRequestBuilder = RpcRequestBody.newBuilder();
|
||||
rpcRequestBuilder.setMethodName("foo");
|
||||
rpcRequestBuilder.setRequestClassName(GetOnlineRegionRequest.class.getCanonicalName());
|
||||
RpcRequestBody rpcRequest = rpcRequestBuilder.build();
|
||||
RequestHeader.Builder headerBuilder = RequestHeader.newBuilder();
|
||||
headerBuilder.setMethodName("foo");
|
||||
RequestHeader header = headerBuilder.build();
|
||||
QosFunction qosFunc = regionServer.getQosFunction();
|
||||
assertTrue (qosFunc.apply(rpcRequest) == HConstants.NORMAL_QOS);
|
||||
assertTrue (qosFunc.apply(new Pair<RequestHeader, Message>(header, null)) ==
|
||||
HConstants.NORMAL_QOS);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQosFunctionForScanMethod() throws IOException {
|
||||
RpcRequestBody.Builder rpcRequestBuilder = RpcRequestBody.newBuilder();
|
||||
rpcRequestBuilder.setMethodName("scan");
|
||||
RequestHeader.Builder headerBuilder = RequestHeader.newBuilder();
|
||||
headerBuilder.setMethodName("scan");
|
||||
RequestHeader header = headerBuilder.build();
|
||||
|
||||
//build an empty scan request
|
||||
ScanRequest.Builder scanBuilder = ScanRequest.newBuilder();
|
||||
ByteString requestBody = scanBuilder.build().toByteString();
|
||||
rpcRequestBuilder.setRequest(requestBody);
|
||||
RpcRequestBody rpcRequest = rpcRequestBuilder.build();
|
||||
assertTrue (qosFunction.apply(rpcRequest) == HConstants.NORMAL_QOS);
|
||||
ScanRequest scanRequest = scanBuilder.build();
|
||||
HRegion mockRegion = Mockito.mock(HRegion.class);
|
||||
HRegionServer mockRS = Mockito.mock(HRegionServer.class);
|
||||
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
|
||||
scanBuilder = ScanRequest.newBuilder();
|
||||
scanBuilder.setScannerId(12345);
|
||||
requestBody = scanBuilder.build().toByteString();
|
||||
rpcRequestBuilder.setRequest(requestBody);
|
||||
rpcRequestBuilder.setRequestClassName(ScanRequest.class.getCanonicalName());
|
||||
rpcRequest = rpcRequestBuilder.build();
|
||||
scanRequest = scanBuilder.build();
|
||||
//mock out a high priority type handling and see the QoS returned
|
||||
HRegionServer mockRS = Mockito.mock(HRegionServer.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(mockRegionScanner.getRegionInfo()).thenReturn(mockRegionInfo);
|
||||
Mockito.when(mockRS.getRegion((RegionSpecifier)Mockito.any())).thenReturn(mockRegion);
|
||||
Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
|
||||
Mockito.when(mockRegionInfo.isMetaTable()).thenReturn(true);
|
||||
Mockito.when(mockRegionInfo.isMetaRegion()).thenReturn(true);
|
||||
|
||||
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
|
||||
Mockito.when(mockRegionInfo.isMetaTable()).thenReturn(false);
|
||||
assertTrue (qosFunction.apply(rpcRequest) == HConstants.NORMAL_QOS);
|
||||
Mockito.when(mockRegionInfo.isMetaRegion()).thenReturn(false);
|
||||
assertTrue (qosFunction.apply(new Pair<RequestHeader, Message>(header, scanRequest)) ==
|
||||
HConstants.NORMAL_QOS);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import java.util.concurrent.CountDownLatch;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.commons.logging.impl.Log4JLogger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
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.exceptions.SnapshotCreationException;
|
||||
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.HTable;
|
||||
import org.apache.hadoop.hbase.client.ScannerCallable;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
|
||||
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.HBaseFsck;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
|
@ -83,6 +88,9 @@ public class TestFlushSnapshotFromClient {
|
|||
*/
|
||||
@BeforeClass
|
||||
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());
|
||||
UTIL.startMiniCluster(NUM_RS);
|
||||
}
|
||||
|
|
|
@ -248,6 +248,8 @@ public class TestThriftServer {
|
|||
|
||||
Thread.sleep(1000);
|
||||
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 );
|
||||
|
||||
|
||||
|
|
|
@ -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.EnvironmentEdgeManagerTestHelper;
|
||||
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.TCompactProtocol;
|
||||
import org.apache.thrift.protocol.TProtocol;
|
||||
|
|
Loading…
Reference in New Issue